You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Lucas Brutschy <lb...@confluent.io.INVALID> on 2023/01/02 16:18:19 UTC

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

Hi Nick,

I'm just starting to read up on the whole discussion about KIP-892 and
KIP-844. Thanks a lot for your work on this, I do think
`WriteBatchWithIndex` may be the way to go here. I do have some
questions about the latest draft.

 A) If I understand correctly, you propose to put a bound on the
(native) memory consumed by each task. However, I wonder if this is
sufficient if we have temporary imbalances in the cluster. For
example, depending on the timing of rebalances during a cluster
restart, it could happen that a single streams node is assigned a lot
more tasks than expected. With your proposed change, this would mean
that the memory required by this one node could be a multiple of what
is required during normal operation. I wonder if it wouldn't be safer
to put a global bound on the memory use, across all tasks.
 B) Generally, the memory concerns still give me the feeling that this
should not be enabled by default for all users in a minor release.
 C) In section "Transaction Management": the sentence "A similar
analogue will be created to automatically manage `Segment`
transactions.". Maybe this is just me lacking some background, 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 you to find out why the KIP-844 PoC is 20x slower (e.g.
by providing a flame graph), that would be quite interesting.

Cheers,
Lucas

On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <ni...@gmail.com> wrote:
>
> Hi everyone,
>
> I've updated the KIP with a more detailed design, which reflects the
> implementation I've been working on:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>
> This new design should address the outstanding points already made in the
> thread.
>
> Please let me know if there are areas that are unclear or need more
> clarification.
>
> I have a (nearly) working implementation. I'm confident that the remaining
> work (making Segments behave) will not impact the documented design.
>
> Regards,
>
> Nick
>
> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <co...@littlehorse.io> wrote:
>
> > Nick,
> >
> > Thank you for the reply; that makes sense. I was hoping that, since reading
> > uncommitted records from IQ in EOS isn't part of the documented API, maybe
> > you *wouldn't* have to wait for the next major release to make that change;
> > but given 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 <ni...@gmail.com>
> > wrote:
> >
> > > Hi Colt,
> > >
> > > 10: Yes, I agree it's not ideal. I originally intended to try to keep the
> > > behaviour unchanged as much as possible, otherwise we'd have to wait for
> > a
> > > major version release to land these changes.
> > > 20: Good point, ALOS doesn't need the same level of guarantee, and the
> > > typically longer commit intervals would be problematic when reading only
> > > "committed" records.
> > >
> > > I've been away for 5 days recovering from minor surgery, but I spent a
> > > considerable amount of that time working through ideas for possible
> > > solutions in my head. I think your suggestion of keeping ALOS as-is, but
> > > buffering writes for EOS is the right path forwards, although I have a
> > > solution that both expands on this, and provides for some more formal
> > > guarantees.
> > >
> > > Essentially, adding support to KeyValueStores for "Transactions", with
> > > clearly defined IsolationLevels. Using "Read Committed" when under EOS,
> > and
> > > "Read Uncommitted" under ALOS.
> > >
> > > The nice thing about this approach is that it gives us much more clearly
> > > defined isolation behaviour that can be properly documented to ensure
> > users
> > > know what to expect.
> > >
> > > I'm still working out the kinks in the design, and will update the KIP
> > when
> > > I have something. The main struggle is trying to implement this without
> > > making any major changes to the existing interfaces or breaking existing
> > > implementations, because currently everything expects to operate directly
> > > on a StateStore, and not a Transaction of that store. I think I'm getting
> > > close, although sadly I won't be able to progress much until next week
> > due
> > > to some work commitments.
> > >
> > > Regards,
> > > Nick
> > >
> > > On Thu, 1 Dec 2022 at 00:01, Colt McNealy <co...@littlehorse.io> wrote:
> > >
> > > > Nick,
> > > >
> > > > Thank you for the explanation, and also for the updated KIP. I am quite
> > > > eager for this improvement to be released as it would greatly reduce
> > the
> > > > operational difficulties of EOS streams apps.
> > > >
> > > > Two questions:
> > > >
> > > > 10)
> > > > >When reading records, we will use the
> > > > WriteBatchWithIndex#getFromBatchAndDB
> > > >  and WriteBatchWithIndex#newIteratorWithBase utilities in order to
> > ensure
> > > > that uncommitted writes are available to query.
> > > > Why do extra work to enable the reading of uncommitted writes during
> > IQ?
> > > > Code complexity aside, reading uncommitted writes is, in my opinion, a
> > > > minor flaw in EOS IQ; it would be very nice to have the guarantee that,
> > > > with EOS, IQ only reads committed records. In order to avoid dirty
> > reads,
> > > > one currently must query a standby replica (but this still doesn't
> > fully
> > > > guarantee monotonic reads).
> > > >
> > > > 20) Is it also necessary to enable this optimization on ALOS stores?
> > The
> > > > motivation of KIP-844 was mainly to reduce the need to restore state
> > from
> > > > scratch on unclean EOS shutdowns; with ALOS it was acceptable to accept
> > > > that there may have been uncommitted writes on disk. On a side note, if
> > > you
> > > > enable this type of store on ALOS processors, the community would
> > > > 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 <ni...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi everyone,
> > > > >
> > > > > I've drastically reduced the scope of this KIP to no longer include
> > the
> > > > > StateStore management of checkpointing. This can be added as a KIP
> > > later
> > > > on
> > > > > to further optimize the consistency and performance of state stores.
> > > > >
> > > > > I've also added a section discussing some of the concerns 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 2022 at 16:36, Nick Telford <ni...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > Hi Colt,
> > > > > >
> > > > > > I didn't do any profiling, but the 844 implementation:
> > > > > >
> > > > > >    - Writes uncommitted records to a temporary RocksDB instance
> > > > > >       - Since tombstones need to be flagged, all record values are
> > > > > >       prefixed with a value/tombstone marker. This necessitates a
> > > > memory
> > > > > copy.
> > > > > >    - On-commit, iterates all records in this temporary instance and
> > > > > >    writes them to the main RocksDB store.
> > > > > >    - While iterating, the value/tombstone marker needs to be parsed
> > > and
> > > > > >    the real value extracted. This necessitates another memory copy.
> > > > > >
> > > > > > My guess is that the cost of iterating the temporary RocksDB store
> > is
> > > > the
> > > > > > major factor, with the 2 extra memory copies per-Record
> > contributing
> > > a
> > > > > > significant amount too.
> > > > > >
> > > > > > Regards,
> > > > > > Nick
> > > > > >
> > > > > > On Mon, 28 Nov 2022 at 16:12, Colt McNealy <co...@littlehorse.io>
> > > > wrote:
> > > > > >
> > > > > >> Hi all,
> > > > > >>
> > > > > >> Out of curiosity, why does the performance of the store degrade so
> > > > > >> significantly with the 844 implementation? I wouldn't be too
> > > surprised
> > > > > by
> > > > > >> a
> > > > > >> 50-60% drop (caused by each record being written twice), but 96%
> > is
> > > > > >> extreme.
> > > > > >>
> > > > > >> The only thing I can think of which could create such a bottleneck
> > > > 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 Nick Telford <
> > > nick.telford@gmail.com>
> > > > > >> wrote:
> > > > > >>
> > > > > >> > Hi everyone,
> > > > > >> >
> > > > > >> > I've updated the KIP to resolve all the points that have been
> > > raised
> > > > > so
> > > > > >> > far, with one exception: the ALOS default commit interval of 5
> > > > minutes
> > > > > >> is
> > > > > >> > likely to cause WriteBatchWithIndex memory to grow too large.
> > > > > >> >
> > > > > >> > There's a couple of different things I can think of to solve
> > this:
> > > > > >> >
> > > > > >> >    - We already have a memory/record limit in the KIP to prevent
> > > OOM
> > > > > >> >    errors. Should we choose a default value for these? My
> > concern
> > > > here
> > > > > >> is
> > > > > >> > that
> > > > > >> >    anything we choose might seem rather arbitrary. We could
> > change
> > > > > >> >    its behaviour such that under ALOS, it only triggers the
> > commit
> > > > of
> > > > > >> the
> > > > > >> >    StateStore, but under EOS, it triggers a commit of the Kafka
> > > > > >> > transaction.
> > > > > >> >    - We could introduce a separate `checkpoint.interval.ms` to
> > > > allow
> > > > > >> ALOS
> > > > > >> >    to commit the StateStores more frequently than the general
> > > > > >> >    commit.interval.ms? My concern here is that the semantics of
> > > > this
> > > > > >> > config
> > > > > >> >    would depend 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 <
> > > nick.telford@gmail.com>
> > > > > >> wrote:
> > > > > >> >
> > > > > >> > > Hi Alex,
> > > > > >> > >
> > > > > >> > > Thanks for the feedback.
> > > > > >> > >
> > > > > >> > > I've updated the discussion of OOM issues by describing how
> > > we'll
> > > > > >> handle
> > > > > >> > > it. Here's the new text:
> > > > > >> > >
> > > > > >> > > To mitigate this, we will automatically force a Task commit if
> > > the
> > > > > >> total
> > > > > >> > >> uncommitted records returned by
> > > > > >> > >> StateStore#approximateNumUncommittedEntries()  exceeds a
> > > > threshold,
> > > > > >> > >> configured by max.uncommitted.state.entries.per.task; or the
> > > > total
> > > > > >> > >> memory used for buffering uncommitted records returned by
> > > > > >> > >> StateStore#approximateNumUncommittedBytes() exceeds the
> > > threshold
> > > > > >> > >> configured by max.uncommitted.state.bytes.per.task. This will
> > > > > roughly
> > > > > >> > >> bound the memory required per-Task for buffering uncommitted
> > > > > records,
> > > > > >> > >> irrespective of the commit.interval.ms, and will effectively
> > > > bound
> > > > > >> the
> > > > > >> > >> number of records that will need to be restored in the event
> > > of a
> > > > > >> > failure.
> > > > > >> > >>
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > These limits will be checked in StreamTask#process and a
> > > premature
> > > > > >> commit
> > > > > >> > >> will be requested via Task#requestCommit().
> > > > > >> > >>
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > Note that these new methods provide default implementations
> > that
> > > > > >> ensure
> > > > > >> > >> existing custom stores and non-transactional stores (e.g.
> > > > > >> > >> InMemoryKeyValueStore) do not force any early commits.
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > I've chosen to have the StateStore expose approximations of
> > its
> > > > > buffer
> > > > > >> > > size/count instead of opaquely requesting a commit in order to
> > > > > >> delegate
> > > > > >> > the
> > > > > >> > > decision making to the Task itself. This enables Tasks to look
> > > at
> > > > > >> *all*
> > > > > >> > of
> > > > > >> > > their StateStores, and determine whether an early commit is
> > > > > necessary.
> > > > > >> > > Notably, it enables pre-Task thresholds, instead of per-Store,
> > > > which
> > > > > >> > > prevents Tasks with many StateStores from using much more
> > memory
> > > > > than
> > > > > >> > Tasks
> > > > > >> > > with one StateStore. This makes sense, since commits are done
> > > > > by-Task,
> > > > > >> > not
> > > > > >> > > by-Store.
> > > > > >> > >
> > > > > >> > > Prizes* for anyone who can come up with a better name for the
> > > new
> > > > > >> config
> > > > > >> > > properties!
> > > > > >> > >
> > > > > >> > > Thanks for pointing out the potential performance issues of
> > > WBWI.
> > > > > From
> > > > > >> > the
> > > > > >> > > benchmarks that user posted[1], it looks like WBWI still
> > > performs
> > > > > >> > > considerably better than individual puts, which is the
> > existing
> > > > > >> design,
> > > > > >> > so
> > > > > >> > > I'd actually expect a performance boost from WBWI, just not as
> > > > great
> > > > > >> as
> > > > > >> > > we'd get from a plain WriteBatch. This does suggest that a
> > good
> > > > > >> > > optimization would be to use a regular WriteBatch for
> > > restoration
> > > > > (in
> > > > > >> > > RocksDBStore#restoreBatch), since we know that those records
> > > will
> > > > > >> never
> > > > > >> > be
> > > > > >> > > queried before they're committed.
> > > > > >> > >
> > > > > >> > > 1:
> > > > > >> >
> > > > >
> > > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > > >> > >
> > > > > >> > > * Just kidding, no prizes, sadly.
> > > > > >> > >
> > > > > >> > > On Wed, 23 Nov 2022 at 12:28, Alexander Sorokoumov
> > > > > >> > > <as...@confluent.io.invalid> wrote:
> > > > > >> > >
> > > > > >> > >> 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 to keep the current checkpointing as a
> > > > > fallback
> > > > > >> > >> mechanism. We want to keep existing users' workflows intact
> > if
> > > we
> > > > > >> can. A
> > > > > >> > >> non-intrusive way would be to add a separate StateStore
> > method,
> > > > > say,
> > > > > >> > >> StateStore#managesCheckpointing(), that controls whether the
> > > > state
> > > > > >> store
> > > > > >> > >> implementation owns checkpointing.
> > > > > >> > >>
> > > > > >> > >> I think that a solution to the transactional writes should
> > > > address
> > > > > >> the
> > > > > >> > >> OOMEs. One possible way to address that is to wire
> > StateStore's
> > > > > >> commit
> > > > > >> > >> request by adding, say, StateStore#commitNeeded that is
> > checked
> > > > in
> > > > > >> > >> StreamTask#commitNeeded via the corresponding
> > > > > ProcessorStateManager.
> > > > > >> > With
> > > > > >> > >> that change, RocksDBStore will have to track the current
> > > > > transaction
> > > > > >> > size
> > > > > >> > >> and request a commit when the size goes over a (configurable)
> > > > > >> threshold.
> > > > > >> > >>
> > > > > >> > >> AFAIU WriteBatchWithIndex might perform significantly slower
> > > than
> > > > > >> > non-txn
> > > > > >> > >> puts as the batch size grows [1]. We should have a
> > > configuration
> > > > to
> > > > > >> fall
> > > > > >> > >> back to the current behavior (and/or disable txn stores for
> > > ALOS)
> > > > > >> unless
> > > > > >> > >> the benchmarks show negligible overhead for longer commits /
> > > > > >> > large-enough
> > > > > >> > >> batch sizes.
> > > > > >> > >>
> > > > > >> > >> If you prefer to keep the KIP smaller, I would rather cut out
> > > > > >> > >> state-store-managed checkpointing rather than proper OOMe
> > > > handling
> > > > > >> and
> > > > > >> > >> being able to switch to non-txn behavior. The checkpointing
> > 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 configuration parts as well as review and test the
> > > > patch.
> > > > > >> > >>
> > > > > >> > >> Best,
> > > > > >> > >> Alex
> > > > > >> > >>
> > > > > >> > >>
> > > > > >> > >> 1. https://github.com/facebook/rocksdb/issues/608
> > > > > >> > >>
> > > > > >> > >> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
> > > > > nick.telford@gmail.com
> > > > > >> >
> > > > > >> > >> wrote:
> > > > > >> > >>
> > > > > >> > >> > Hi John,
> > > > > >> > >> >
> > > > > >> > >> > Thanks for the review and feedback!
> > > > > >> > >> >
> > > > > >> > >> > 1. Custom Stores: I've been mulling over this problem
> > myself.
> > > > As
> > > > > it
> > > > > >> > >> stands,
> > > > > >> > >> > custom stores would essentially lose checkpointing with no
> > > > > >> indication
> > > > > >> > >> that
> > > > > >> > >> > they're expected to make changes, besides a line in the
> > > release
> > > > > >> > notes. I
> > > > > >> > >> > agree that the best solution would be to provide a default
> > > that
> > > > > >> > >> checkpoints
> > > > > >> > >> > to a file. The one thing I would change is that the
> > > > checkpointing
> > > > > >> is
> > > > > >> > to
> > > > > >> > >> a
> > > > > >> > >> > store-local file, instead of a per-Task file. This way the
> > > > > >> StateStore
> > > > > >> > >> still
> > > > > >> > >> > technically owns its own checkpointing (via a default
> > > > > >> implementation),
> > > > > >> > >> and
> > > > > >> > >> > the StateManager/Task execution engine doesn't need to know
> > > > > >> anything
> > > > > >> > >> about
> > > > > >> > >> > checkpointing, which greatly simplifies some of the logic.
> > > > > >> > >> >
> > > > > >> > >> > 2. OOME errors: The main reasons why I didn't explore a
> > > > solution
> > > > > to
> > > > > >> > >> this is
> > > > > >> > >> > a) to keep this KIP as simple as possible, and b) because
> > I'm
> > > > not
> > > > > >> > >> exactly
> > > > > >> > >> > how to signal that a Task should commit prematurely. I'm
> > > > > confident
> > > > > >> > it's
> > > > > >> > >> > possible, and I think it's worth adding a section on
> > handling
> > > > > this.
> > > > > >> > >> Besides
> > > > > >> > >> > my proposal to force an early commit once memory usage
> > > reaches
> > > > a
> > > > > >> > >> threshold,
> > > > > >> > >> > is there any other approach that you might suggest for
> > > tackling
> > > > > >> this
> > > > > >> > >> > problem?
> > > > > >> > >> >
> > > > > >> > >> > 3. ALOS: I can add in an explicit paragraph, but my
> > > assumption
> > > > is
> > > > > >> that
> > > > > >> > >> > since transactional behaviour comes at little/no cost, that
> > > it
> > > > > >> should
> > > > > >> > be
> > > > > >> > >> > available by default on all stores, irrespective of the
> > > > > processing
> > > > > >> > mode.
> > > > > >> > >> > While ALOS doesn't use transactions, the Task itself still
> > > > > >> "commits",
> > > > > >> > so
> > > > > >> > >> > the behaviour should be correct under ALOS too. I'm not
> > > > convinced
> > > > > >> that
> > > > > >> > >> it's
> > > > > >> > >> > worth having both transactional/non-transactional stores
> > > > > >> available, as
> > > > > >> > >> it
> > > > > >> > >> > would considerably increase the complexity of the codebase,
> > > for
> > > > > >> very
> > > > > >> > >> little
> > > > > >> > >> > benefit.
> > > > > >> > >> >
> > > > > >> > >> > 4. Method deprecation: Are you referring to
> > > > > >> StateStore#getPosition()?
> > > > > >> > >> As I
> > > > > >> > >> > understand it, Position contains the position of the
> > *source*
> > > > > >> topics,
> > > > > >> > >> > whereas the commit offsets would be the *changelog*
> > offsets.
> > > So
> > > > > >> it's
> > > > > >> > >> still
> > > > > >> > >> > necessary to retain the Position data, as well as the
> > > changelog
> > > > > >> > offsets.
> > > > > >> > >> > What I meant in the KIP is that Position offsets are
> > > currently
> > > > > >> stored
> > > > > >> > >> in a
> > > > > >> > >> > file, and since we can atomically store metadata along with
> > > the
> > > > > >> record
> > > > > >> > >> > batch we commit to RocksDB, we can move our Position
> > offsets
> > > in
> > > > > to
> > > > > >> > this
> > > > > >> > >> > metadata too, and gain the same transactional guarantees
> > that
> > > > we
> > > > > >> will
> > > > > >> > >> for
> > > > > >> > >> > changelog offsets, ensuring that the Position offsets are
> > > > > >> consistent
> > > > > >> > >> with
> > > > > >> > >> > the records that are read from the database.
> > > > > >> > >> >
> > > > > >> > >> > Regards,
> > > > > >> > >> > Nick
> > > > > >> > >> >
> > > > > >> > >> > On Tue, 22 Nov 2022 at 16:25, John Roesler <
> > > > vvcephei@apache.org>
> > > > > >> > wrote:
> > > > > >> > >> >
> > > > > >> > >> > > Thanks for publishing this alternative, Nick!
> > > > > >> > >> > >
> > > > > >> > >> > > The benchmark you mentioned in the KIP-844 discussion
> > seems
> > > > > like
> > > > > >> a
> > > > > >> > >> > > compelling reason to revisit the built-in
> > transactionality
> > > > > >> > mechanism.
> > > > > >> > >> I
> > > > > >> > >> > > also appreciate you analysis, showing that for most use
> > > > cases,
> > > > > >> the
> > > > > >> > >> write
> > > > > >> > >> > > batch approach should be just fine.
> > > > > >> > >> > >
> > > > > >> > >> > > There are a couple of points that would hold me back from
> > > > > >> approving
> > > > > >> > >> this
> > > > > >> > >> > > KIP right now:
> > > > > >> > >> > >
> > > > > >> > >> > > 1. Loss of coverage for custom stores.
> > > > > >> > >> > > The fact that you can plug in a (relatively) simple
> > > > > >> implementation
> > > > > >> > of
> > > > > >> > >> the
> > > > > >> > >> > > XStateStore interfaces and automagically get a
> > distributed
> > > > > >> database
> > > > > >> > >> out
> > > > > >> > >> > of
> > > > > >> > >> > > it is a significant benefit of Kafka Streams. I'd hate to
> > > > lose
> > > > > >> it,
> > > > > >> > so
> > > > > >> > >> it
> > > > > >> > >> > > would be better to spend some time and come up with a way
> > > to
> > > > > >> > preserve
> > > > > >> > >> > that
> > > > > >> > >> > > property. For example, can we provide a default
> > > > implementation
> > > > > of
> > > > > >> > >> > > `commit(..)` that re-implements the existing
> > > checkpoint-file
> > > > > >> > >> approach? Or
> > > > > >> > >> > > perhaps add an `isTransactional()` flag to the state
> > store
> > > > > >> interface
> > > > > >> > >> so
> > > > > >> > >> > > that the runtime can decide whether to continue to manage
> > > > > >> checkpoint
> > > > > >> > >> > files
> > > > > >> > >> > > vs delegating transactionality to the stores?
> > > > > >> > >> > >
> > > > > >> > >> > > 2. Guarding against OOME
> > > > > >> > >> > > I appreciate your analysis, but I don't think it's
> > > sufficient
> > > > > to
> > > > > >> say
> > > > > >> > >> that
> > > > > >> > >> > > we will solve the memory problem later if it becomes
> > > > necessary.
> > > > > >> The
> > > > > >> > >> > > experience leading to that situation would be quite bad:
> > > > > Imagine,
> > > > > >> > you
> > > > > >> > >> > > upgrade to AK 3.next, your tests pass, so you deploy to
> > > > > >> production.
> > > > > >> > >> That
> > > > > >> > >> > > night, you get paged because your app is now crashing
> > with
> > > > > >> OOMEs. As
> > > > > >> > >> with
> > > > > >> > >> > > all OOMEs, you'll have a really hard time finding the
> > root
> > > > > cause,
> > > > > >> > and
> > > > > >> > >> > once
> > > > > >> > >> > > you do, you won't have a clear path to resolve the issue.
> > > You
> > > > > >> could
> > > > > >> > >> only
> > > > > >> > >> > > tune down the commit interval and cache buffer size until
> > > you
> > > > > >> stop
> > > > > >> > >> > getting
> > > > > >> > >> > > crashes.
> > > > > >> > >> > >
> > > > > >> > >> > > FYI, I know of multiple cases where people run EOS with
> > > much
> > > > > >> larger
> > > > > >> > >> > commit
> > > > > >> > >> > > intervals to get better batching than the default, so I
> > > don't
> > > > > >> think
> > > > > >> > >> this
> > > > > >> > >> > > pathological case would be as rare as you suspect.
> > > > > >> > >> > >
> > > > > >> > >> > > Given that we already have the rudiments of an idea of
> > what
> > > > we
> > > > > >> could
> > > > > >> > >> do
> > > > > >> > >> > to
> > > > > >> > >> > > prevent this downside, we should take the time to design
> > a
> > > > > >> solution.
> > > > > >> > >> We
> > > > > >> > >> > owe
> > > > > >> > >> > > it to our users to ensure that awesome new features don't
> > > > come
> > > > > >> with
> > > > > >> > >> > bitter
> > > > > >> > >> > > pills unless we can't avoid it.
> > > > > >> > >> > >
> > > > > >> > >> > > 3. ALOS mode.
> > > > > >> > >> > > On the other hand, I didn't see an indication of how
> > stores
> > > > > will
> > > > > >> be
> > > > > >> > >> > > handled under ALOS (aka non-EOS) mode. Theoretically, the
> > > > > >> > >> > transactionality
> > > > > >> > >> > > of the store and the processing mode are orthogonal. A
> > > > > >> transactional
> > > > > >> > >> > store
> > > > > >> > >> > > would serve ALOS just as well as a non-transactional one
> > > (if
> > > > > not
> > > > > >> > >> better).
> > > > > >> > >> > > Under ALOS, though, the default commit interval is five
> > > > > minutes,
> > > > > >> so
> > > > > >> > >> the
> > > > > >> > >> > > memory issue is far more pressing.
> > > > > >> > >> > >
> > > > > >> > >> > > As I see it, we have several options to resolve this
> > point.
> > > > We
> > > > > >> could
> > > > > >> > >> > > demonstrate that transactional stores work just fine for
> > > ALOS
> > > > > >> and we
> > > > > >> > >> can
> > > > > >> > >> > > therefore just swap over unconditionally. We could also
> > > > disable
> > > > > >> the
> > > > > >> > >> > > transactional mechanism under ALOS so that stores operate
> > > > just
> > > > > >> the
> > > > > >> > >> same
> > > > > >> > >> > as
> > > > > >> > >> > > they do today when run in ALOS mode. Finally, we could do
> > > the
> > > > > >> same
> > > > > >> > as
> > > > > >> > >> in
> > > > > >> > >> > > KIP-844 and make transactional stores opt-in (it'd be
> > > better
> > > > to
> > > > > >> > avoid
> > > > > >> > >> the
> > > > > >> > >> > > extra opt-in mechanism, but it's a good
> > > get-out-of-jail-free
> > > > > >> card).
> > > > > >> > >> > >
> > > > > >> > >> > > 4. (minor point) Deprecation of methods
> > > > > >> > >> > >
> > > > > >> > >> > > You mentioned that the new `commit` 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?
> > > > > >> > >> > >
> > > > > >> > >> > > Thanks again for the KIP! It's really nice that you and
> > > Alex
> > > > > will
> > > > > >> > get
> > > > > >> > >> the
> > > > > >> > >> > > chance to collaborate on both directions so that we can
> > get
> > > > the
> > > > > >> best
> > > > > >> > >> > > outcome for Streams and its users.
> > > > > >> > >> > >
> > > > > >> > >> > > -John
> > > > > >> > >> > >
> > > > > >> > >> > >
> > > > > >> > >> > > On 2022/11/21 15:02:15 Nick Telford wrote:
> > > > > >> > >> > > > 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
> > > > > >> > >> > > > <
> > > > > >> > >> > >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > > >> > >> > > >,
> > > > > >> > >> > > > so that it can be discussed/reviewed separately from
> > > > KIP-844.
> > > > > >> > >> > > >
> > > > > >> > >> > > > Alex: I'm especially interested in what you think!
> > > > > >> > >> > > >
> > > > > >> > >> > > > I have a nearly complete implementation of the changes
> > > > > >> outlined in
> > > > > >> > >> this
> > > > > >> > >> > > > KIP, please let me know if you'd like me to push them
> > for
> > > > > >> review
> > > > > >> > in
> > > > > >> > >> > > advance
> > > > > >> > >> > > > of a vote.
> > > > > >> > >> > > >
> > > > > >> > >> > > > Regards,
> > > > > >> > >> > > >
> > > > > >> > >> > > > Nick
> > > > > >> > >> > > >
> > > > > >> > >> > >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >

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

Posted by Nick Telford <ni...@gmail.com>.
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-892: Transactional Semantics for StateStores

Posted by Walker Carlson <wc...@confluent.io.INVALID>.
Hey Nick,

I just read through the KIP. It seems like this is something we really need
in streams.

The only questions I have would be related to the implementation and are
not relevant the the KIP process. I think you are good to restart the Vote
thread. I would be willing to give this a +1 binding as is.

best,
Walker



On Tue, Apr 16, 2024 at 2:15 PM Nick Telford <ni...@gmail.com> wrote:

> OK, I've adjusted the KIP to use "enable.transactional.statestores" that
> now unconditionally defaults to "false".
>
> This required a fairly significant rewrite of the KIP to remove all the
> references to isolation level, so you might need to re-read the entire KIP.
>
> I did not update the references to isolation level in the "Rejected
> Alternatives" section, because it still seemed relevant there.
>
> If there are no other concerns, I'll bring the KIP to a vote.
>
> Thanks for all your input!
>
> Nick
>
> On Tue, 16 Apr 2024 at 17:33, Sophie Blee-Goldman <so...@responsive.dev>
> wrote:
>
> > Yeah I had missed the internal Streams configs when I first looked
> > and was going to suggest the "enable.x.y" format based on the client
> > configs until I noticed we had internal configs of the form x.y.enabled
> >
> > But I fully agree that it's silly to align ourselves with internal config
> > names
> > and we should just follow the established pattern in public configs.
> >
> > So yeah, I'm on board with enable.transactional.stores
> >
> > ...I think that's the last open question then?
> >
> > On Mon, Apr 15, 2024 at 5:32 AM Lucas Brutschy
> > <lb...@confluent.io.invalid> wrote:
> >
> > > Hi Nick,
> > >
> > > the java clients do have some properties that are of the form
> > > `enable....` (`enable.auto.comit` and `enable.idempotance`), so I
> > > would go with this form. State updater and processing thread configs
> > > use the `....enabled` form, but they are internal and could be changed
> > > to the `enable...` form for consistency. But, not intending to start a
> > > big discussion here. Both options are fine with me.
> > >
> > > Thanks for the updates!
> > >
> > >
> > > On Sun, Apr 14, 2024 at 10:58 PM Sophie Blee-Goldman
> > > <so...@responsive.dev> wrote:
> > > >
> > > > Makes sense to me! Regarding the new config name -- doesn't seem like
> > > > we have any *public *configs in StreamsConfig that set a standard
> yet,
> > so
> > > > imo we're free to pick whatever we think sounds good.
> > > >
> > > > However we do have a few internal configs that follow the pattern
> > > > *feature.name.enabled* so if we want to play it safe and adhere to
> the
> > > > established pattern, we should call it
> > > "transactional.statestores.enabled"
> > > >
> > > > I also personally think x.y.enabled sounds better than enable.x.y,
> but
> > > > honestly I don't feel too strongly either way. I don't think it's
> worth
> > > > prolonging
> > > > the discussion any further over
> > > >
> > > > I'll take a look at KIP-892 as soon as I can since it's a prereq for
> > this
> > > > one.
> > > > Thanks for the updates!
> > > >
> > > >
> > > > On Thu, Apr 11, 2024 at 12:02 PM Nick Telford <
> nick.telford@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi everyone,
> > > > >
> > > > > After various off-list discussions, it's become clear there are
> still
> > > some
> > > > > contentious points to this KIP.
> > > > >
> > > > > 1.
> > > > > We agreed to change the "default.state.isolation.level" config
> > > property to
> > > > > be a transactional state store feature toggle. What should we name
> > > this?
> > > > > "enable.transactional.statestores"? What's the convention for this
> > > kind of
> > > > > flag?
> > > > >
> > > > > 2.
> > > > > The "atomic checkpointing" section has been broken out into
> KIP-1035
> > > (with
> > > > > one significant change, and otherwise just more explanation). This
> is
> > > now a
> > > > > hard dependency of KIP-892.
> > > > >
> > > > > Regards,
> > > > > Nick
> > > > >
> > > > > On Mon, 6 Nov 2023 at 17:00, Nick Telford <ni...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Hi everyone,
> > > > > >
> > > > > > Sorry for the delay.
> > > > > >
> > > > > > I've updated the KIP based on our discussion. I've also added
> some
> > > > > details
> > > > > > on "Transactional support under READ_UNCOMMITTED" and
> "Query-time
> > > > > > Isolation Levels" in the "Rejected Alternatives" section, to
> > > highlight
> > > > > the
> > > > > > potential to improve things in a future KIP.
> > > > > >
> > > > > > If there are no more requested changes or objections, I'll kick
> off
> > > the
> > > > > > vote thread tomorrow, approximately 24 hours from now.
> > > > > >
> > > > > > While I'd love for this KIP to make it into 3.7.0, I can't commit
> > to
> > > it
> > > > > > being code-complete by the December 20th deadline. While the
> > branch I
> > > > > have
> > > > > > is broadly working, there are areas that will likely require
> > > improvement,
> > > > > > and others that are untested:
> > > > > >
> > > > > > - Untested outside of unit tests: Windowed stores, Versioned
> > stores,
> > > > > > Global stores, IQv2. They should all work as expected, but the
> > > Position
> > > > > > file logic might be a bit janky.
> > > > > > - Requires improvement: Position data handling. Very clunky right
> > > now.
> > > > > > - Requires implementation: The "feature flag", where we disable
> > > > > > transactionality under EOS+READ_UNCOMMITTED. I've started this,
> but
> > > it's
> > > > > > far from simple. Plus this will require fleshing out the test
> suite
> > > > > quite a
> > > > > > bit.
> > > > > >
> > > > > > For the curious, my active development branch has been rebased
> > > against
> > > > > > 3.6.0: https://github.com/nicktelford/kafka/tree/KIP-892-3.6.0
> > > > > >
> > > > > > Regards,
> > > > > > Nick
> > > > > >
> > > > > > On Mon, 30 Oct 2023 at 23:32, Sophie Blee-Goldman <
> > > sophie@responsive.dev
> > > > > >
> > > > > > wrote:
> > > > > >
> > > > > >> Hey Nick, sounds like things are moving along here. I think
> you're
> > > > > already
> > > > > >> aware, but just as a reminder to us all, the KIP freeze deadline
> > > for 3.7
> > > > > >> is
> > > > > >> approaching and is currently set for Nov 18. I think we can all
> > > agree
> > > > > that
> > > > > >> it would be great to have this KIP accepted by then, and it
> sounds
> > > like
> > > > > >> much of the implementation is already completed, in which case
> > this
> > > > > might
> > > > > >> be able to make it into 3.7.
> > > > > >>
> > > > > >> It sounds like this discussion is wrapping up so I just want to
> > > > > summarize
> > > > > >> my understanding of the current plan regarding configs since it
> > > doesn't
> > > > > >> seem like the KIP has been updated to include this yet.
> > > > > >>
> > > > > >> Basically we're all in agreement to go with option #1 that you
> > > presented
> > > > > >> earlier, right? Assuming so, I just want to clear up some
> details
> > > around
> > > > > >> the default behavior. What happens if:
> > > > > >> 1. User sets EOS and sets READ_UNCOMMITTED: txn state stores
> will
> > be
> > > > > >> disabled/feature-flagged
> > > > > >> 2. User sets EOS and does not set anything for the isolation
> > level:
> > > > > >> although the default is READ_UNCOMMITTED, enabling EOS will
> change
> > > the
> > > > > >> default to READ_COMMITTED and txn state stores will be used
> > > > > >>
> > > > > >> As for the future, it sounds like when READ_UNCOMMITTED mode is
> > > > > >> implemented, we will basically just remove this "feature flag"
> and
> > > txn
> > > > > >> state stores will always be used for all EOS cases. EOS users
> will
> > > be
> > > > > able
> > > > > >> to configure the isolation level independently, although it will
> > > still
> > > > > >> default to READ_COMMITTED when EOS is enabled and it wasn't
> > > explicitly
> > > > > >> configured.
> > > > > >>
> > > > > >> Is this understanding correct? I think this was the right choice
> > > > > >> regardless, as it will give people a way to disable the txn
> stores
> > > in an
> > > > > >> emergency -- as a project we went a long time with little
> pressure
> > > to
> > > > > >> feature flag things, and our users paid the cost for that. Even
> if
> > > we
> > > > > >> managed to ship something without bugs, it was often only after
> an
> > > > > intense
> > > > > >> period of successive blocker bugs that delayed the entire
> released
> > > for
> > > > > >> weeks. Other times, major bugs slipped through and some versions
> > > became
> > > > > >> unusable for certain use cases. So having some way to disable
> the
> > > txn
> > > > > >> state
> > > > > >> stores when EOS is used feels like a good strategy, since you
> just
> > > never
> > > > > >> know what might get through all the testing we do.
> > > > > >>
> > > > > >> If this ends up slipping to 4.0 and you manage to implement the
> > > > > >> READ_UNCOMMITTED mode within the same release, I think it's
> worth
> > > > > >> considering to add in an additional feature flag, even if it's
> > just
> > > a
> > > > > >> backdoor internal config (eg as we did in KIP-441 with the
> > internal
> > > task
> > > > > >> assignor config). But for now let's assume the first release
> this
> > > KIP
> > > > > >> appears in will have the behavior as described above, with
> > > > > >> READ_UNCOMMITTED
> > > > > >> mode acting as a feature flag
> > > > > >>
> > > > > >> Nick -- please let us know when you've updated the KIP to
> clarify
> > > the
> > > > > >> config behavior, and are ready for a vote!
> > > > > >>
> > > > > >>
> > > > > >> On Sun, Oct 29, 2023 at 12:02 PM Colt McNealy <
> > colt@littlehorse.io>
> > > > > >> wrote:
> > > > > >>
> > > > > >> > Guozhang—I agree, I am in favor of moving forward with the KIP
> > now
> > > > > that
> > > > > >> the
> > > > > >> > Transactional State Stores will be behind a feature flag.
> > > > > >> >
> > > > > >> > Nick—I just did a bit more light testing of your branch
> > > > > `KIP-892-3.5.0`
> > > > > >> > with your most recent changes. I couldn't detect a performance
> > > > > >> difference
> > > > > >> > versus trunk (in the past there was a slight degradation of
> > > > > performance
> > > > > >> on
> > > > > >> > the restoration path, but that has been fixed). I don't
> believe
> > > that
> > > > > >> your
> > > > > >> > branch has the state updater thread enabled, so I didn't test
> > that
> > > > > path
> > > > > >> too
> > > > > >> > heavily.
> > > > > >> >
> > > > > >> > As expected, however, our internal correctness tests failed
> due
> > > to the
> > > > > >> IQ
> > > > > >> > read-your-own-writes issue we discussed previously. The
> > community
> > > as a
> > > > > >> > whole would vastly benefit from this KIP getting over the
> finish
> > > line
> > > > > in
> > > > > >> > 3.7.0, and so long as it is behind a feature flag so that we
> at
> > > > > >> LittleHorse
> > > > > >> > can still guarantee RYOW for our users, I think it's purely a
> > win
> > > 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 <
> > > > > >> guozhang.wang.us@gmail.com
> > > > > >> > >
> > > > > >> > wrote:
> > > > > >> >
> > > > > >> > > I'd agree with you guys that as long as we are in agreement
> > > about
> > > > > the
> > > > > >> > > configuration semantics, that would be a big win to move
> > > forward for
> > > > > >> > > this KIP. As for the TaskCorruptedException handling like
> > wiping
> > > > > state
> > > > > >> > > stores, we can discuss that in the PR rather than in the
> KIP.
> > > > > >> > >
> > > > > >> > > Just to clarify, I'm onboard with the latest proposal, and
> > > probably
> > > > > we
> > > > > >> > > can move on for voting on this KIP now?
> > > > > >> > >
> > > > > >> > > Guozhang
> > > > > >> > >
> > > > > >> > > On Thu, Oct 19, 2023 at 5:33 AM Bruno Cadonna <
> > > cadonna@apache.org>
> > > > > >> > wrote:
> > > > > >> > > >
> > > > > >> > > > Hi Nick,
> > > > > >> > > >
> > > > > >> > > > What you and Lucas wrote about the different
> configurations
> > of
> > > > > >> ALOS/EOS
> > > > > >> > > > and READ_COMMITTED/READ_UNCOMMITTED make sense to me. My
> > > earlier
> > > > > >> > > > concerns about changelogs diverging from the content of
> the
> > > local
> > > > > >> state
> > > > > >> > > > stores turned out to not apply. So I think, we can move on
> > > with
> > > > > >> those
> > > > > >> > > > configurations.
> > > > > >> > > >
> > > > > >> > > > Regarding the TaskCorruptedException and wiping out the
> > state
> > > > > stores
> > > > > >> > > > under EOS, couldn't we abort the transaction on the state
> > > store
> > > > > and
> > > > > >> > > > close the task dirty? If the Kafka transaction was indeed
> > > > > committed,
> > > > > >> > the
> > > > > >> > > > store would restore the missing part from the changelog
> > > topic. If
> > > > > >> the
> > > > > >> > > > Kafka transaction was not committed, changelog topic and
> > state
> > > > > store
> > > > > >> > are
> > > > > >> > > > in-sync.
> > > > > >> > > >
> > > > > >> > > > In any case, IMO those are implementation details that we
> do
> > > not
> > > > > >> need
> > > > > >> > to
> > > > > >> > > > discuss and solve in the KIP discussion. We can solve them
> > on
> > > the
> > > > > >> PR.
> > > > > >> > > > The important thing is that the processing guarantees
> hold.
> > > > > >> > > >
> > > > > >> > > > Best,
> > > > > >> > > > Bruno
> > > > > >> > > >
> > > > > >> > > > On 10/18/23 3:56 PM, Nick Telford wrote:
> > > > > >> > > > > Hi Lucas,
> > > > > >> > > > >
> > > > > >> > > > > TaskCorruptedException is how Streams signals that the
> > Task
> > > > > state
> > > > > >> > > needs to
> > > > > >> > > > > be wiped, so we can't retain that exception without also
> > > wiping
> > > > > >> state
> > > > > >> > > on
> > > > > >> > > > > timeouts.
> > > > > >> > > > >
> > > > > >> > > > > Regards,
> > > > > >> > > > > Nick
> > > > > >> > > > >
> > > > > >> > > > > On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <
> > > > > >> lbrutschy@confluent.io
> > > > > >> > > .invalid>
> > > > > >> > > > > wrote:
> > > > > >> > > > >
> > > > > >> > > > >> Hi Nick,
> > > > > >> > > > >>
> > > > > >> > > > >> I think indeed the better behavior would be to retry
> > > > > >> > commitTransaction
> > > > > >> > > > >> until we risk running out of time to meet `
> > > > > max.poll.interval.ms
> > > > > >> `.
> > > > > >> > > > >>
> > > > > >> > > > >> However, if it's handled as a `TaskCorruptedException`
> at
> > > the
> > > > > >> > moment,
> > > > > >> > > > >> I would do the same in this KIP, and leave exception
> > > handling
> > > > > >> > > > >> improvements to future work. 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 <
> > > > > >> > nick.telford@gmail.com>
> > > > > >> > > > >> wrote:
> > > > > >> > > > >>>
> > > > > >> > > > >>> Hi Lucas,
> > > > > >> > > > >>>
> > > > > >> > > > >>> Yeah, this is pretty much the direction I'm thinking
> of
> > > going
> > > > > in
> > > > > >> > > now. You
> > > > > >> > > > >>> make an interesting point about committing on-error
> > under
> > > > > >> > > > >>> ALOS/READ_COMMITTED, although I haven't had a chance
> to
> > > think
> > > > > >> > > through the
> > > > > >> > > > >>> implications yet.
> > > > > >> > > > >>>
> > > > > >> > > > >>> Something that I ran into earlier this week is an
> issue
> > > with
> > > > > the
> > > > > >> > new
> > > > > >> > > > >>> handling of TimeoutException. Without TX stores,
> > > > > >> TimeoutException
> > > > > >> > > under
> > > > > >> > > > >> EOS
> > > > > >> > > > >>> throws a TaskCorruptedException, which wipes the
> stores.
> > > > > >> However,
> > > > > >> > > with TX
> > > > > >> > > > >>> stores, TimeoutException is now just bubbled up and
> > dealt
> > > with
> > > > > >> as
> > > > > >> > it
> > > > > >> > > is
> > > > > >> > > > >>> under ALOS. The problem arises when the
> > > > > >> Producer#commitTransaction
> > > > > >> > > call
> > > > > >> > > > >>> times out: Streams attempts to ignore the error and
> > > continue
> > > > > >> > > producing,
> > > > > >> > > > >>> which causes the next call to Producer#send to throw
> > > > > >> > > > >>> "IllegalStateException: Cannot attempt operation
> `send`
> > > > > because
> > > > > >> the
> > > > > >> > > > >>> previous call to `commitTransaction` timed out and
> must
> > be
> > > > > >> > retried".
> > > > > >> > > > >>>
> > > > > >> > > > >>> I'm not sure what we should do here: retrying the
> > > > > >> commitTransaction
> > > > > >> > > seems
> > > > > >> > > > >>> logical, but what if it times out again? Where do we
> > draw
> > > the
> > > > > >> line
> > > > > >> > > and
> > > > > >> > > > >>> shutdown the instance?
> > > > > >> > > > >>>
> > > > > >> > > > >>> Regards,
> > > > > >> > > > >>> Nick
> > > > > >> > > > >>>
> > > > > >> > > > >>> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <
> > > > > >> > lbrutschy@confluent.io
> > > > > >> > > > >> .invalid>
> > > > > >> > > > >>> wrote:
> > > > > >> > > > >>>
> > > > > >> > > > >>>> Hi all,
> > > > > >> > > > >>>>
> > > > > >> > > > >>>> I think I liked your suggestion of allowing EOS with
> > > > > >> > > READ_UNCOMMITTED,
> > > > > >> > > > >>>> but keep wiping the state on error, and I'd vote for
> > this
> > > > > >> solution
> > > > > >> > > > >>>> when introducing `default.state.isolation.level`.
> This
> > > way,
> > > > > >> we'd
> > > > > >> > > have
> > > > > >> > > > >>>> the most low-risk roll-out of this feature (no
> behavior
> > > > > change
> > > > > >> > > without
> > > > > >> > > > >>>> reconfiguration), with the possibility of switching
> to
> > > the
> > > > > most
> > > > > >> > > sane /
> > > > > >> > > > >>>> battle-tested default settings in 4.0. Essentially,
> > we'd
> > > > > have a
> > > > > >> > > > >>>> feature flag but call it
> > `default.state.isolation.level`
> > > and
> > > > > >> don't
> > > > > >> > > > >>>> have to deprecate it later.
> > > > > >> > > > >>>>
> > > > > >> > > > >>>> So the possible configurations would then be this:
> > > > > >> > > > >>>>
> > > > > >> > > > >>>> 1. ALOS/READ_UNCOMMITTED (default) = processing uses
> > > > > >> direct-to-DB,
> > > > > >> > > IQ
> > > > > >> > > > >>>> reads from DB.
> > > > > >> > > > >>>> 2. ALOS/READ_COMMITTED = processing uses WriteBatch,
> IQ
> > > reads
> > > > > >> from
> > > > > >> > > > >>>> WriteBatch/DB. Flush on error (see note below).
> > > > > >> > > > >>>> 3. EOS/READ_UNCOMMITTED (default) = processing uses
> > > > > >> direct-to-DB,
> > > > > >> > IQ
> > > > > >> > > > >>>> reads from DB. Wipe state on error.
> > > > > >> > > > >>>> 4. EOS/READ_COMMITTED = processing uses WriteBatch,
> IQ
> > > reads
> > > > > >> from
> > > > > >> > > > >>>> WriteBatch/DB.
> > > > > >> > > > >>>>
> > > > > >> > > > >>>> I believe the feature is important enough that we
> will
> > > see
> > > > > good
> > > > > >> > > > >>>> adoption even without changing the default. In 4.0,
> > when
> > > we
> > > > > >> have
> > > > > >> > > seen
> > > > > >> > > > >>>> this being adopted and is battle-tested, we make
> > > > > READ_COMMITTED
> > > > > >> > the
> > > > > >> > > > >>>> default for EOS, or even READ_COMITTED always the
> > > default,
> > > > > >> > depending
> > > > > >> > > > >>>> on our experiences. And we could add a clever
> > > implementation
> > > > > of
> > > > > >> > > > >>>> READ_UNCOMITTED with WriteBatches later.
> > > > > >> > > > >>>>
> > > > > >> > > > >>>> The only smell here is that
> > > `default.state.isolation.level`
> > > > > >> > wouldn't
> > > > > >> > > > >>>> be purely an IQ setting, but it would also (slightly)
> > > change
> > > > > >> the
> > > > > >> > > > >>>> behavior of the processing, but that seems
> unavoidable
> > as
> > > > > long
> > > > > >> as
> > > > > >> > we
> > > > > >> > > > >>>> haven't solve READ_UNCOMITTED IQ with WriteBatches.
> > > > > >> > > > >>>>
> > > > > >> > > > >>>> Minor: As for Bruno's point 4, I think if we are
> > > concerned
> > > > > >> about
> > > > > >> > > this
> > > > > >> > > > >>>> 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).
> > > > > >> > > > >>>>
> > > > > >> > > > >>>> Cheers,
> > > > > >> > > > >>>> Lucas
> > > > > >> > > > >>>>
> > > > > >> > > > >>>> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <
> > > > > >> > > nick.telford@gmail.com>
> > > > > >> > > > >>>> wrote:
> > > > > >> > > > >>>>>
> > > > > >> > > > >>>>> Hi Guozhang,
> > > > > >> > > > >>>>>
> > > > > >> > > > >>>>> The KIP as it stands introduces a new configuration,
> > > > > >> > > > >>>>> default.state.isolation.level, which is independent
> of
> > > > > >> > > > >> processing.mode.
> > > > > >> > > > >>>>> It's intended that this new configuration be used to
> > > > > >> configure a
> > > > > >> > > > >> global
> > > > > >> > > > >>>> IQ
> > > > > >> > > > >>>>> isolation level in the short term, with a future KIP
> > > > > >> introducing
> > > > > >> > > the
> > > > > >> > > > >>>>> capability to change the isolation level on a
> > per-query
> > > > > basis,
> > > > > >> > > > >> falling
> > > > > >> > > > >>>> back
> > > > > >> > > > >>>>> to the "default" defined by this config. That's why
> I
> > > called
> > > > > >> it
> > > > > >> > > > >>>> "default",
> > > > > >> > > > >>>>> for future-proofing.
> > > > > >> > > > >>>>>
> > > > > >> > > > >>>>> However, it currently includes the caveat that
> > > > > >> READ_UNCOMMITTED
> > > > > >> > is
> > > > > >> > > > >> not
> > > > > >> > > > >>>>> available under EOS. I think this is the coupling
> you
> > > are
> > > > > >> > alluding
> > > > > >> > > > >> to?
> > > > > >> > > > >>>>>
> > > > > >> > > > >>>>> This isn't intended to be a restriction of the API,
> > but
> > > is
> > > > > >> > > currently
> > > > > >> > > > >> a
> > > > > >> > > > >>>>> technical limitation. However, after discussing with
> > > some
> > > > > >> users
> > > > > >> > > about
> > > > > >> > > > >>>>> use-cases that would require READ_UNCOMMITTED under
> > > EOS, I'm
> > > > > >> > > > >> inclined to
> > > > > >> > > > >>>>> remove that clause and put in the necessary work to
> > make
> > > > > that
> > > > > >> > > > >> combination
> > > > > >> > > > >>>>> possible now.
> > > > > >> > > > >>>>>
> > > > > >> > > > >>>>> I currently see two possible approaches:
> > > > > >> > > > >>>>>
> > > > > >> > > > >>>>>     1. Disable TX StateStores internally when the
> > > > > >> IsolationLevel
> > > > > >> > is
> > > > > >> > > > >>>>>     READ_UNCOMMITTED and the processing.mode is EOS.
> > > This is
> > > > > >> more
> > > > > >> > > > >>>> difficult
> > > > > >> > > > >>>>>     than it sounds, as there are many assumptions
> > being
> > > made
> > > > > >> > > > >> throughout
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>     internals about the guarantees StateStores
> > provide.
> > > It
> > > > > >> would
> > > > > >> > > > >>>> definitely add
> > > > > >> > > > >>>>>     a lot of extra "if (read_uncommitted && eos)"
> > > branches,
> > > > > >> > > > >> complicating
> > > > > >> > > > >>>>>     maintenance and testing.
> > > > > >> > > > >>>>>     2. Invest the time *now* to make
> READ_UNCOMMITTED
> > > of EOS
> > > > > >> > > > >> StateStores
> > > > > >> > > > >>>>>     possible. I have some ideas on how this could be
> > > > > achieved,
> > > > > >> > but
> > > > > >> > > > >> they
> > > > > >> > > > >>>> would
> > > > > >> > > > >>>>>     need testing and could introduce some additional
> > > issues.
> > > > > >> The
> > > > > >> > > > >> benefit
> > > > > >> > > > >>>> of
> > > > > >> > > > >>>>>     this approach is that it would make query-time
> > > > > >> > IsolationLevels
> > > > > >> > > > >> much
> > > > > >> > > > >>>> simpler
> > > > > >> > > > >>>>>     to implement in the future.
> > > > > >> > > > >>>>>
> > > > > >> > > > >>>>> Unfortunately, both will require considerable work
> > that
> > > will
> > > > > >> > > further
> > > > > >> > > > >>>> delay
> > > > > >> > > > >>>>> this KIP, which was the reason I placed the
> > restriction
> > > in
> > > > > the
> > > > > >> > KIP
> > > > > >> > > > >> in the
> > > > > >> > > > >>>>> first place.
> > > > > >> > > > >>>>>
> > > > > >> > > > >>>>> Regards,
> > > > > >> > > > >>>>> Nick
> > > > > >> > > > >>>>>
> > > > > >> > > > >>>>> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
> > > > > >> > > > >> guozhang.wang.us@gmail.com>
> > > > > >> > > > >>>>> wrote:
> > > > > >> > > > >>>>>
> > > > > >> > > > >>>>>> Hello Nick,
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>>> First of all, thanks a lot for the great effort
> > you've
> > > put
> > > > > in
> > > > > >> > > > >> driving
> > > > > >> > > > >>>>>> this KIP! I really like it coming through finally,
> as
> > > many
> > > > > >> > people
> > > > > >> > > > >> in
> > > > > >> > > > >>>>>> the community have raised this. At the same time I
> > > honestly
> > > > > >> > feel a
> > > > > >> > > > >> bit
> > > > > >> > > > >>>>>> ashamed for not putting enough of my time
> supporting
> > > it and
> > > > > >> > > > >> pushing it
> > > > > >> > > > >>>>>> through the finish line (you raised this KIP
> almost a
> > > year
> > > > > >> ago).
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>>> I briefly passed through the DISCUSS thread so far,
> > not
> > > > > sure
> > > > > >> > I've
> > > > > >> > > > >> 100
> > > > > >> > > > >>>>>> percent digested all the bullet points. But with
> the
> > > goal
> > > > > of
> > > > > >> > > > >> trying to
> > > > > >> > > > >>>>>> help take it through the finish line in mind, I'd
> > want
> > > to
> > > > > >> throw
> > > > > >> > > > >>>>>> thoughts on top of my head only on the point #4
> above
> > > > > which I
> > > > > >> > felt
> > > > > >> > > > >> may
> > > > > >> > > > >>>>>> be the main hurdle for the current KIP to drive to
> a
> > > > > >> consensus
> > > > > >> > > now.
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>>> The general question I asked myself is, whether we
> > > want to
> > > > > >> > couple
> > > > > >> > > > >> "IQ
> > > > > >> > > > >>>>>> reading mode" with "processing mode". While
> > > technically I
> > > > > >> tend
> > > > > >> > to
> > > > > >> > > > >>>>>> agree with you that, it's feels like a bug if some
> > > single
> > > > > >> user
> > > > > >> > > > >> chose
> > > > > >> > > > >>>>>> "EOS" for processing mode while choosing "read
> > > uncommitted"
> > > > > >> for
> > > > > >> > IQ
> > > > > >> > > > >>>>>> reading mode, at the same time, I'm thinking if
> it's
> > > > > possible
> > > > > >> > that
> > > > > >> > > > >>>>>> there could be two different persons (or even two
> > > teams)
> > > > > that
> > > > > >> > > > >> would be
> > > > > >> > > > >>>>>> using the stream API to build the app, and the IQ
> API
> > > to
> > > > > >> query
> > > > > >> > the
> > > > > >> > > > >>>>>> running state of the app. I know this is less of a
> > > > > technical
> > > > > >> > thing
> > > > > >> > > > >> but
> > > > > >> > > > >>>>>> rather a more design stuff, but if it could be ever
> > the
> > > > > case,
> > > > > >> > I'm
> > > > > >> > > > >>>>>> wondering if the personale using the IQ API knows
> > > about the
> > > > > >> > risks
> > > > > >> > > > >> of
> > > > > >> > > > >>>>>> using read uncommitted but still chose so for the
> > > favor of
> > > > > >> > > > >>>>>> performance, no matter if the underlying stream
> > > processing
> > > > > >> mode
> > > > > >> > > > >>>>>> configured by another personale is EOS or not. In
> > that
> > > > > >> regard,
> > > > > >> > I'm
> > > > > >> > > > >>>>>> leaning towards a "leaving the door open, and close
> > it
> > > > > later
> > > > > >> if
> > > > > >> > we
> > > > > >> > > > >>>>>> found it's a bad idea" aspect with a configuration
> > > 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 can
> > still
> > > > > >> enforce
> > > > > >> > > > >>>>>> coupling them.
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>>> Again, just my 2c here. Thanks again for the great
> > > patience
> > > > > >> and
> > > > > >> > > > >>>>>> diligence on this KIP.
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>>> Guozhang
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>>> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
> > > > > >> > > > >> nick.telford@gmail.com>
> > > > > >> > > > >>>>>> wrote:
> > > > > >> > > > >>>>>>>
> > > > > >> > > > >>>>>>> Hi Bruno,
> > > > > >> > > > >>>>>>>
> > > > > >> > > > >>>>>>> 4.
> > > > > >> > > > >>>>>>> I'll hold off on making that change until we have
> a
> > > > > >> consensus
> > > > > >> > as
> > > > > >> > > > >> to
> > > > > >> > > > >>>> what
> > > > > >> > > > >>>>>>> configuration to use to control all of this, as
> > it'll
> > > be
> > > > > >> > > > >> affected by
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>> decision on EOS isolation levels.
> > > > > >> > > > >>>>>>>
> > > > > >> > > > >>>>>>> 5.
> > > > > >> > > > >>>>>>> Done. I've chosen "committedOffsets".
> > > > > >> > > > >>>>>>>
> > > > > >> > > > >>>>>>> Regards,
> > > > > >> > > > >>>>>>> Nick
> > > > > >> > > > >>>>>>>
> > > > > >> > > > >>>>>>> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <
> > > > > >> > cadonna@apache.org>
> > > > > >> > > > >>>> wrote:
> > > > > >> > > > >>>>>>>
> > > > > >> > > > >>>>>>>> Hi Nick,
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>>> 1.
> > > > > >> > > > >>>>>>>> Yeah, you are probably right that it does not
> make
> > > too
> > > > > much
> > > > > >> > > > >> sense.
> > > > > >> > > > >>>>>>>> Thanks for the clarification!
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>>> 4.
> > > > > >> > > > >>>>>>>> Yes, sorry for the back and forth, but I think
> for
> > > the
> > > > > >> sake of
> > > > > >> > > > >> the
> > > > > >> > > > >>>> KIP
> > > > > >> > > > >>>>>>>> it is better to let the ALOS behavior as it is
> for
> > > now
> > > > > due
> > > > > >> to
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>>>> possible issues you would run into. Maybe we can
> > > find a
> > > > > >> > > > >> solution
> > > > > >> > > > >>>> in the
> > > > > >> > > > >>>>>>>> future. Now the question returns to whether we
> > really
> > > > > need
> > > > > >> > > > >>>>>>>> default.state.isolation.level. Maybe the config
> > > could be
> > > > > >> the
> > > > > >> > > > >>>> feature
> > > > > >> > > > >>>>>>>> flag Sophie requested.
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>>> 5.
> > > > > >> > > > >>>>>>>> There 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
> > > > > >> committedOffsetFor().
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>>> Best,
> > > > > >> > > > >>>>>>>> Bruno
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>>> On 10/13/23 3:21 PM, Nick Telford wrote:
> > > > > >> > > > >>>>>>>>> Hi Bruno,
> > > > > >> > > > >>>>>>>>>
> > > > > >> > > > >>>>>>>>> Thanks for getting back to me.
> > > > > >> > > > >>>>>>>>>
> > > > > >> > > > >>>>>>>>> 1.
> > > > > >> > > > >>>>>>>>> I think this should be possible. Are you
> thinking
> > > of the
> > > > > >> > > > >>>> situation
> > > > > >> > > > >>>>>> where
> > > > > >> > > > >>>>>>>> a
> > > > > >> > > > >>>>>>>>> user may downgrade to a previous version of
> Kafka
> > > > > >> Streams? In
> > > > > >> > > > >>>> that
> > > > > >> > > > >>>>>> case,
> > > > > >> > > > >>>>>>>>> sadly, the RocksDBStore would get wiped by the
> > older
> > > > > >> version
> > > > > >> > > > >> of
> > > > > >> > > > >>>> Kafka
> > > > > >> > > > >>>>>>>>> Streams anyway, because that version wouldn't
> > > understand
> > > > > >> the
> > > > > >> > > > >>>> extra
> > > > > >> > > > >>>>>> column
> > > > > >> > > > >>>>>>>>> family (that holds offsets), so the missing
> > Position
> > > > > file
> > > > > >> > > > >> would
> > > > > >> > > > >>>>>>>>> automatically get rebuilt when the store is
> > rebuilt
> > > from
> > > > > >> the
> > > > > >> > > > >>>>>> changelog.
> > > > > >> > > > >>>>>>>>> Are there other situations than downgrade where
> a
> > > > > >> > > > >> transactional
> > > > > >> > > > >>>> store
> > > > > >> > > > >>>>>>>> could
> > > > > >> > > > >>>>>>>>> be replaced by a non-transactional one? I can't
> > > think of
> > > > > >> any.
> > > > > >> > > > >>>>>>>>>
> > > > > >> > > > >>>>>>>>> 2.
> > > > > >> > > > >>>>>>>>> Ahh yes, the Test Plan - my Kryptonite! This
> > section
> > > > > >> > > > >> definitely
> > > > > >> > > > >>>>>> needs to
> > > > > >> > > > >>>>>>>> be
> > > > > >> > > > >>>>>>>>> fleshed out. I'll work on that. How much detail
> do
> > > you
> > > > > >> need?
> > > > > >> > > > >>>>>>>>>
> > > > > >> > > > >>>>>>>>> 3.
> > > > > >> > > > >>>>>>>>> See my previous email discussing this.
> > > > > >> > > > >>>>>>>>>
> > > > > >> > > > >>>>>>>>> 4.
> > > > > >> > > > >>>>>>>>> Hmm, this is an interesting point. Are you
> > > suggesting
> > > > > that
> > > > > >> > > > >> under
> > > > > >> > > > >>>> ALOS
> > > > > >> > > > >>>>>>>>> READ_COMMITTED should not be supported?
> > > > > >> > > > >>>>>>>>>
> > > > > >> > > > >>>>>>>>> Regards,
> > > > > >> > > > >>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>
> > > > > >> > > > >>>>>>>>> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
> > > > > >> > > > >> cadonna@apache.org>
> > > > > >> > > > >>>>>> wrote:
> > > > > >> > > > >>>>>>>>>
> > > > > >> > > > >>>>>>>>>> Hi Nick,
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>> I think the KIP is converging!
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>> 1.
> > > > > >> > > > >>>>>>>>>> I am wondering whether it makes sense to write
> > the
> > > > > >> position
> > > > > >> > > > >> file
> > > > > >> > > > >>>>>> during
> > > > > >> > > > >>>>>>>>>> close as we do for the checkpoint file, so that
> > in
> > > case
> > > > > >> the
> > > > > >> > > > >>>> state
> > > > > >> > > > >>>>>> store
> > > > > >> > > > >>>>>>>>>> is replaced with a non-transactional state
> store
> > > the
> > > > > >> > > > >>>>>> non-transactional
> > > > > >> > > > >>>>>>>>>> state store finds the position file. I think,
> > this
> > > is
> > > > > not
> > > > > >> > > > >>>> strictly
> > > > > >> > > > >>>>>>>>>> needed, but would be a nice behavior instead of
> > > just
> > > > > >> > > > >> deleting
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>>>>> position file.
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>> 2.
> > > > > >> > > > >>>>>>>>>> The test plan does not mention integration
> tests.
> > > Do
> > > > > you
> > > > > >> not
> > > > > >> > > > >>>> need to
> > > > > >> > > > >>>>>>>>>> extend existing ones and add new ones. Also for
> > > > > upgrading
> > > > > >> > > > >> and
> > > > > >> > > > >>>>>>>>>> downgrading you might need integration and/or
> > > system
> > > > > >> tests.
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>> 3.
> > > > > >> > > > >>>>>>>>>> I think Sophie made a point. Although, IQ
> reading
> > > from
> > > > > >> > > > >>>> uncommitted
> > > > > >> > > > >>>>>> data
> > > > > >> > > > >>>>>>>>>> under EOS might be considered a bug by some
> > people.
> > > > > Thus,
> > > > > >> > > > >> your
> > > > > >> > > > >>>> KIP
> > > > > >> > > > >>>>>> would
> > > > > >> > > > >>>>>>>>>> fix a bug rather than changing the intended
> > > behavior.
> > > > > >> > > > >> However, I
> > > > > >> > > > >>>>>> also
> > > > > >> > > > >>>>>>>>>> see that a feature flag would help users that
> > rely
> > > on
> > > > > >> this
> > > > > >> > > > >> buggy
> > > > > >> > > > >>>>>>>>>> behavior (at least until AK 4.0).
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>> 4.
> > > > > >> > > > >>>>>>>>>> This is related to the previous point. I assume
> > > that
> > > > > the
> > > > > >> > > > >>>> difference
> > > > > >> > > > >>>>>>>>>> between READ_COMMITTED and READ_UNCOMMITTED for
> > > ALOS is
> > > > > >> > > > >> that in
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>>>>> former you enable transactions on the state
> store
> > > and
> > > > > in
> > > > > >> the
> > > > > >> > > > >>>> latter
> > > > > >> > > > >>>>>> you
> > > > > >> > > > >>>>>>>>>> disable them. If my assumption is correct, I
> > think
> > > that
> > > > > >> is
> > > > > >> > > > >> an
> > > > > >> > > > >>>> issue.
> > > > > >> > > > >>>>>>>>>> Let's assume under ALOS Streams fails over a
> > > couple of
> > > > > >> times
> > > > > >> > > > >>>> more or
> > > > > >> > > > >>>>>>>>>> less at the same step in processing after
> value 3
> > > is
> > > > > >> added
> > > > > >> > > > >> to an
> > > > > >> > > > >>>>>>>>>> aggregation but the offset of the corresponding
> > > input
> > > > > >> record
> > > > > >> > > > >>>> was not
> > > > > >> > > > >>>>>>>>>> committed. Without transactions disabled, the
> > > > > aggregation
> > > > > >> > > > >> value
> > > > > >> > > > >>>>>> would
> > > > > >> > > > >>>>>>>>>> increase by 3 for each failover. With
> > transactions
> > > > > >> enabled,
> > > > > >> > > > >>>> value 3
> > > > > >> > > > >>>>>>>>>> would only be added to the aggregation once
> when
> > > the
> > > > > >> offset
> > > > > >> > > > >> of
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>> input
> > > > > >> > > > >>>>>>>>>> record is committed and the transaction finally
> > > > > >> completes.
> > > > > >> > > > >> So
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>>>>> content of the state store would change
> depending
> > > on
> > > > > the
> > > > > >> > > > >>>>>> configuration
> > > > > >> > > > >>>>>>>>>> for IQ. IMO, the content of the state store
> > should
> > > be
> > > > > >> > > > >>>> independent
> > > > > >> > > > >>>>>> from
> > > > > >> > > > >>>>>>>>>> IQ. Given this issue, I propose to not use
> > > transactions
> > > > > >> with
> > > > > >> > > > >>>> ALOS at
> > > > > >> > > > >>>>>>>>>> all. I was a big proponent of using
> transactions
> > > with
> > > > > >> ALOS,
> > > > > >> > > > >> but
> > > > > >> > > > >>>> I
> > > > > >> > > > >>>>>>>>>> realized that transactions with ALOS is not as
> > > easy as
> > > > > >> > > > >> enabling
> > > > > >> > > > >>>>>>>>>> transactions on state stores. Another aspect
> that
> > > is
> > > > > >> > > > >>>> problematic is
> > > > > >> > > > >>>>>> that
> > > > > >> > > > >>>>>>>>>> the changelog topic which actually replicates
> the
> > > state
> > > > > >> > > > >> store
> > > > > >> > > > >>>> is not
> > > > > >> > > > >>>>>>>>>> transactional under ALOS. Thus, it might happen
> > > that
> > > > > the
> > > > > >> > > > >> state
> > > > > >> > > > >>>>>> store and
> > > > > >> > > > >>>>>>>>>> the changelog differ in their content. All of
> > this
> > > is
> > > > > >> maybe
> > > > > >> > > > >>>> solvable
> > > > > >> > > > >>>>>>>>>> somehow, but for the sake of this KIP, I would
> > > leave it
> > > > > >> for
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>> future.
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>> Best,
> > > > > >> > > > >>>>>>>>>> Bruno
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>> On 10/12/23 10:32 PM, Sophie Blee-Goldman
> wrote:
> > > > > >> > > > >>>>>>>>>>> Hey Nick! First of all thanks for taking up
> this
> > > > > awesome
> > > > > >> > > > >>>> feature,
> > > > > >> > > > >>>>>> I'm
> > > > > >> > > > >>>>>>>>>> sure
> > > > > >> > > > >>>>>>>>>>> every single
> > > > > >> > > > >>>>>>>>>>> Kafka Streams user and dev would agree that it
> > is
> > > > > sorely
> > > > > >> > > > >>>> needed.
> > > > > >> > > > >>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>> I've just been catching up on the KIP and
> > > surrounding
> > > > > >> > > > >>>> discussion,
> > > > > >> > > > >>>>>> so
> > > > > >> > > > >>>>>>>>>> please
> > > > > >> > > > >>>>>>>>>>> forgive me
> > > > > >> > > > >>>>>>>>>>> for any misunderstandings or
> misinterpretations
> > > of the
> > > > > >> > > > >> current
> > > > > >> > > > >>>>>> plan and
> > > > > >> > > > >>>>>>>>>>> don't hesitate to
> > > > > >> > > > >>>>>>>>>>> correct me.
> > > > > >> > > > >>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>> Before I jump in, I just want to say that
> having
> > > seen
> > > > > >> this
> > > > > >> > > > >>>> drag on
> > > > > >> > > > >>>>>> for
> > > > > >> > > > >>>>>>>> so
> > > > > >> > > > >>>>>>>>>>> long, my singular
> > > > > >> > > > >>>>>>>>>>> goal in responding is to help this KIP past a
> > > > > perceived
> > > > > >> > > > >>>> impasse so
> > > > > >> > > > >>>>>> we
> > > > > >> > > > >>>>>>>> can
> > > > > >> > > > >>>>>>>>>>> finally move on
> > > > > >> > > > >>>>>>>>>>> to voting and implementing it. Long
> discussions
> > > are to
> > > > > >> be
> > > > > >> > > > >>>> expected
> > > > > >> > > > >>>>>> for
> > > > > >> > > > >>>>>>>>>>> major features like
> > > > > >> > > > >>>>>>>>>>> this but it's completely on us as the Streams
> > > devs to
> > > > > >> make
> > > > > >> > > > >> sure
> > > > > >> > > > >>>>>> there
> > > > > >> > > > >>>>>>>> is
> > > > > >> > > > >>>>>>>>>> an
> > > > > >> > > > >>>>>>>>>>> end in sight
> > > > > >> > > > >>>>>>>>>>> for any ongoing discussion.
> > > > > >> > > > >>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>> With that said, it's my understanding that the
> > > KIP as
> > > > > >> > > > >> currently
> > > > > >> > > > >>>>>>>> proposed
> > > > > >> > > > >>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>> just not tenable
> > > > > >> > > > >>>>>>>>>>> for Kafka Streams, and would prevent some EOS
> > > users
> > > > > from
> > > > > >> > > > >>>> upgrading
> > > > > >> > > > >>>>>> to
> > > > > >> > > > >>>>>>>> the
> > > > > >> > > > >>>>>>>>>>> version it
> > > > > >> > > > >>>>>>>>>>> first appears in. Given that we can't predict
> or
> > > > > >> guarantee
> > > > > >> > > > >>>> whether
> > > > > >> > > > >>>>>> any
> > > > > >> > > > >>>>>>>> of
> > > > > >> > > > >>>>>>>>>>> the followup KIPs
> > > > > >> > > > >>>>>>>>>>> would be completed in the same release cycle
> as
> > > this
> > > > > >> one,
> > > > > >> > > > >> we
> > > > > >> > > > >>>> need
> > > > > >> > > > >>>>>> to
> > > > > >> > > > >>>>>>>> make
> > > > > >> > > > >>>>>>>>>>> sure that the
> > > > > >> > > > >>>>>>>>>>> feature is either compatible with all current
> > > users or
> > > > > >> else
> > > > > >> > > > >>>>>>>>>> feature-flagged
> > > > > >> > > > >>>>>>>>>>> so that they may
> > > > > >> > > > >>>>>>>>>>> opt in/out.
> > > > > >> > > > >>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>> Therefore, IIUC we need to have either (or
> both)
> > > of
> > > > > >> these
> > > > > >> > > > >> as
> > > > > >> > > > >>>>>>>>>>> fully-implemented config options:
> > > > > >> > > > >>>>>>>>>>> 1. default.state.isolation.level
> > > > > >> > > > >>>>>>>>>>> 2. enable.transactional.state.stores
> > > > > >> > > > >>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>> This way EOS users for whom read_committed
> > > semantics
> > > > > are
> > > > > >> > > > >> not
> > > > > >> > > > >>>>>> viable can
> > > > > >> > > > >>>>>>>>>>> still upgrade,
> > > > > >> > > > >>>>>>>>>>> and either use the isolation.level config to
> > > leverage
> > > > > >> the
> > > > > >> > > > >> new
> > > > > >> > > > >>>> txn
> > > > > >> > > > >>>>>> state
> > > > > >> > > > >>>>>>>>>>> stores without sacrificing
> > > > > >> > > > >>>>>>>>>>> their application semantics, or else simply
> keep
> > > the
> > > > > >> > > > >>>> transactional
> > > > > >> > > > >>>>>>>> state
> > > > > >> > > > >>>>>>>>>>> stores disabled until we
> > > > > >> > > > >>>>>>>>>>> are able to fully implement the isolation
> level
> > > > > >> > > > >> configuration
> > > > > >> > > > >>>> at
> > > > > >> > > > >>>>>> either
> > > > > >> > > > >>>>>>>>>> an
> > > > > >> > > > >>>>>>>>>>> application or query level.
> > > > > >> > > > >>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>> Frankly you are the expert here and know much
> > more
> > > > > about
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>> tradeoffs
> > > > > >> > > > >>>>>>>> in
> > > > > >> > > > >>>>>>>>>>> both semantics and
> > > > > >> > > > >>>>>>>>>>> effort level of implementing one of these
> > configs
> > > vs
> > > > > the
> > > > > >> > > > >>>> other. In
> > > > > >> > > > >>>>>> my
> > > > > >> > > > >>>>>>>>>>> opinion, either option would
> > > > > >> > > > >>>>>>>>>>> be fine and I would leave the decision of
> which
> > > one to
> > > > > >> > > > >> include
> > > > > >> > > > >>>> in
> > > > > >> > > > >>>>>> this
> > > > > >> > > > >>>>>>>>>> KIP
> > > > > >> > > > >>>>>>>>>>> completely up to you.
> > > > > >> > > > >>>>>>>>>>> I just don't see a way for the KIP to proceed
> > > without
> > > > > >> some
> > > > > >> > > > >>>>>> variation of
> > > > > >> > > > >>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>> above that would allow
> > > > > >> > > > >>>>>>>>>>> EOS users to opt-out of read_committed.
> > > > > >> > > > >>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>> (If it's all the same to you, I would
> recommend
> > > 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 that gets released.
> > > > > >> > > > >>>>>>>>>>> So personally, my choice would be to add the
> > > feature
> > > > > >> flag
> > > > > >> > > > >> and
> > > > > >> > > > >>>>>> leave it
> > > > > >> > > > >>>>>>>>>> off
> > > > > >> > > > >>>>>>>>>>> by default. If all 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
> > > > > >> > > > >>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > > > > >> > > > >>>>>> nick.telford@gmail.com>
> > > > > >> > > > >>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>> Hi Bruno,
> > > > > >> > > > >>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>> Agreed, I can live with that for now.
> > > > > >> > > > >>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>> In an effort to keep the scope of this KIP
> from
> > > > > >> > > > >> expanding, I'm
> > > > > >> > > > >>>>>> leaning
> > > > > >> > > > >>>>>>>>>>>> towards just providing a configurable
> > > > > >> > > > >>>>>> default.state.isolation.level
> > > > > >> > > > >>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>> removing IsolationLevel from the
> > > StateStoreContext.
> > > > > >> This
> > > > > >> > > > >>>> would be
> > > > > >> > > > >>>>>>>>>>>> compatible with adding support for query-time
> > > > > >> > > > >> IsolationLevels
> > > > > >> > > > >>>> in
> > > > > >> > > > >>>>>> the
> > > > > >> > > > >>>>>>>>>>>> future, whilst providing a way for users to
> > > select an
> > > > > >> > > > >>>> isolation
> > > > > >> > > > >>>>>> level
> > > > > >> > > > >>>>>>>>>> now.
> > > > > >> > > > >>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>> The big problem with this, however, is that
> if
> > a
> > > user
> > > > > >> > > > >> selects
> > > > > >> > > > >>>>>>>>>>>> processing.mode
> > > > > >> > > > >>>>>>>>>>>> = "exactly-once(-v2|-beta)", and
> > > > > >> > > > >>>> default.state.isolation.level =
> > > > > >> > > > >>>>>>>>>>>> "READ_UNCOMMITTED", we need to guarantee that
> > the
> > > > > data
> > > > > >> > > > >> isn't
> > > > > >> > > > >>>>>> written
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>> disk until commit() is called, but we also
> need
> > > to
> > > > > >> permit
> > > > > >> > > > >> IQ
> > > > > >> > > > >>>>>> threads
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>> read from the ongoing transaction.
> > > > > >> > > > >>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>> A simple solution would be to (temporarily)
> > > forbid
> > > > > this
> > > > > >> > > > >>>>>> combination of
> > > > > >> > > > >>>>>>>>>>>> configuration, and have
> > > default.state.isolation.level
> > > > > >> > > > >>>>>> automatically
> > > > > >> > > > >>>>>>>>>> switch
> > > > > >> > > > >>>>>>>>>>>> to READ_COMMITTED when processing.mode is
> > > anything
> > > > > >> other
> > > > > >> > > > >> than
> > > > > >> > > > >>>>>>>>>>>> at-least-once. Do you think this would be
> > > acceptable?
> > > > > >> > > > >>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>> In a later KIP, we can add support for
> > query-time
> > > > > >> > > > >> isolation
> > > > > >> > > > >>>>>> levels and
> > > > > >> > > > >>>>>>>>>>>> solve this particular problem there, which
> > would
> > > > > relax
> > > > > >> > > > >> this
> > > > > >> > > > >>>>>>>> restriction.
> > > > > >> > > > >>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>> Regards,
> > > > > >> > > > >>>>>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna
> <
> > > > > >> > > > >>>> cadonna@apache.org>
> > > > > >> > > > >>>>>>>>>> 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,
> > > > > >> > > > >>>>>>>>>>>>> Bruno
> > > > > >> > > > >>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > > > > >> > > > >>>>>>>>>>>>>> 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 <
> > > > > >> > > > >>>>>> nick.telford@gmail.com
> > > > > >> > > > >>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> Hi everyone,
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> I agree that having IsolationLevel be
> > > determined
> > > > > at
> > > > > >> > > > >>>> query-time
> > > > > >> > > > >>>>>> is
> > > > > >> > > > >>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>> ideal design, but there are a few sticking
> > > points:
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> 1.
> > > > > >> > > > >>>>>>>>>>>>>>> There needs to be some way to communicate
> > the
> > > > > >> > > > >>>> IsolationLevel
> > > > > >> > > > >>>>>> down
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>> RocksDBStore itself, so that the query can
> > > respect
> > > > > >> it.
> > > > > >> > > > >>>> Since
> > > > > >> > > > >>>>>> stores
> > > > > >> > > > >>>>>>>>>>>> are
> > > > > >> > > > >>>>>>>>>>>>>>> "layered" in functionality (i.e.
> > > > > ChangeLoggingStore,
> > > > > >> > > > >>>>>> MeteredStore,
> > > > > >> > > > >>>>>>>>>>>>> etc.),
> > > > > >> > > > >>>>>>>>>>>>>>> we need some way to deliver that
> information
> > > to
> > > > > the
> > > > > >> > > > >> bottom
> > > > > >> > > > >>>>>> layer.
> > > > > >> > > > >>>>>>>> For
> > > > > >> > > > >>>>>>>>>>>>> IQv2,
> > > > > >> > > > >>>>>>>>>>>>>>> we can use the existing State#query()
> > method,
> > > but
> > > > > >> IQv1
> > > > > >> > > > >> has
> > > > > >> > > > >>>> no
> > > > > >> > > > >>>>>> way
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>> do
> > > > > >> > > > >>>>>>>>>>>>>>> this.
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> A simple approach, which would potentially
> > > open up
> > > > > >> > > > >> other
> > > > > >> > > > >>>>>> options,
> > > > > >> > > > >>>>>>>>>>>> would
> > > > > >> > > > >>>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>> to add something like:
> > > ReadOnlyKeyValueStore<K, V>
> > > > > >> > > > >>>>>>>>>>>>>>> readOnlyView(IsolationLevel
> isolationLevel)
> > to
> > > > > >> > > > >>>>>>>> ReadOnlyKeyValueStore
> > > > > >> > > > >>>>>>>>>>>>> (and
> > > > > >> > > > >>>>>>>>>>>>>>> similar to ReadOnlyWindowStore,
> > > > > >> ReadOnlySessionStore,
> > > > > >> > > > >>>> etc.).
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> 2.
> > > > > >> > > > >>>>>>>>>>>>>>> As mentioned above, RocksDB WriteBatches
> are
> > > not
> > > > > >> > > > >>>> thread-safe,
> > > > > >> > > > >>>>>> which
> > > > > >> > > > >>>>>>>>>>>>> causes
> > > > > >> > > > >>>>>>>>>>>>>>> a problem if we want to provide
> > > READ_UNCOMMITTED
> > > > > >> > > > >>>> Iterators. I
> > > > > >> > > > >>>>>> also
> > > > > >> > > > >>>>>>>>>>>> had a
> > > > > >> > > > >>>>>>>>>>>>>>> look at RocksDB Transactions[1], but they
> > > solve a
> > > > > >> very
> > > > > >> > > > >>>>>> different
> > > > > >> > > > >>>>>>>>>>>>> problem,
> > > > > >> > > > >>>>>>>>>>>>>>> and have the same thread-safety issue.
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> One possible approach that I mentioned is
> > > chaining
> > > > > >> > > > >>>>>> WriteBatches:
> > > > > >> > > > >>>>>>>>>> every
> > > > > >> > > > >>>>>>>>>>>>>>> time a new Interactive Query is received
> > (i.e.
> > > > > >> > > > >>>> readOnlyView,
> > > > > >> > > > >>>>>> see
> > > > > >> > > > >>>>>>>>>>>> above,
> > > > > >> > > > >>>>>>>>>>>>>>> is called) we "freeze" the existing
> > > WriteBatch,
> > > > > and
> > > > > >> > > > >> start a
> > > > > >> > > > >>>>>> new one
> > > > > >> > > > >>>>>>>>>>>> for
> > > > > >> > > > >>>>>>>>>>>>> new
> > > > > >> > > > >>>>>>>>>>>>>>> writes. The Interactive Query queries the
> > > "chain"
> > > > > of
> > > > > >> > > > >>>> previous
> > > > > >> > > > >>>>>>>>>>>>> WriteBatches
> > > > > >> > > > >>>>>>>>>>>>>>> + the underlying database; while the
> > > StreamThread
> > > > > >> > > > >> starts
> > > > > >> > > > >>>>>> writing to
> > > > > >> > > > >>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>> *new* WriteBatch. On-commit, the
> > StreamThread
> > > > > would
> > > > > >> > > > >> write
> > > > > >> > > > >>>> *all*
> > > > > >> > > > >>>>>>>>>>>>>>> WriteBatches in the chain to the database
> > > (that
> > > > > have
> > > > > >> > > > >> not
> > > > > >> > > > >>>> yet
> > > > > >> > > > >>>>>> been
> > > > > >> > > > >>>>>>>>>>>>> written).
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> WriteBatches would be closed/freed only
> when
> > > they
> > > > > >> have
> > > > > >> > > > >> been
> > > > > >> > > > >>>>>> both
> > > > > >> > > > >>>>>>>>>>>>>>> committed, and all open Interactive
> Queries
> > on
> > > > > them
> > > > > >> > > > >> have
> > > > > >> > > > >>>> been
> > > > > >> > > > >>>>>>>> closed.
> > > > > >> > > > >>>>>>>>>>>>> This
> > > > > >> > > > >>>>>>>>>>>>>>> would require some reference counting.
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> Obviously a drawback of this approach is
> the
> > > > > >> potential
> > > > > >> > > > >> for
> > > > > >> > > > >>>>>>>> increased
> > > > > >> > > > >>>>>>>>>>>>>>> memory usage: if an Interactive Query is
> > > > > long-lived,
> > > > > >> > > > >> for
> > > > > >> > > > >>>>>> example by
> > > > > >> > > > >>>>>>>>>>>>> doing a
> > > > > >> > > > >>>>>>>>>>>>>>> full scan over a large database, or even
> > just
> > > > > >> pausing
> > > > > >> > > > >> in
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>> middle
> > > > > >> > > > >>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>> an
> > > > > >> > > > >>>>>>>>>>>>>>> iteration, then the existing chain of
> > > WriteBatches
> > > > > >> > > > >> could be
> > > > > >> > > > >>>>>> kept
> > > > > >> > > > >>>>>>>>>>>> around
> > > > > >> > > > >>>>>>>>>>>>> for
> > > > > >> > > > >>>>>>>>>>>>>>> a long time, potentially forever.
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> --
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> A.
> > > > > >> > > > >>>>>>>>>>>>>>> Going off on a tangent, it looks like in
> > > addition
> > > > > to
> > > > > >> > > > >>>> supporting
> > > > > >> > > > >>>>>>>>>>>>>>> READ_COMMITTED queries, we could go
> further
> > > and
> > > > > >> support
> > > > > >> > > > >>>>>>>>>>>> REPEATABLE_READ
> > > > > >> > > > >>>>>>>>>>>>>>> queries (i.e. where subsequent reads to
> the
> > > same
> > > > > key
> > > > > >> > > > >> in the
> > > > > >> > > > >>>>>> same
> > > > > >> > > > >>>>>>>>>>>>>>> Interactive Query are guaranteed to yield
> > the
> > > same
> > > > > >> > > > >> value)
> > > > > >> > > > >>>> by
> > > > > >> > > > >>>>>> making
> > > > > >> > > > >>>>>>>>>>>> use
> > > > > >> > > > >>>>>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>> RocksDB Snapshots[2]. These are fairly
> > > > > lightweight,
> > > > > >> so
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>>>>>> performance
> > > > > >> > > > >>>>>>>>>>>>>>> impact is likely to be negligible, but
> they
> > do
> > > > > >> require
> > > > > >> > > > >>>> that the
> > > > > >> > > > >>>>>>>>>>>>> Interactive
> > > > > >> > > > >>>>>>>>>>>>>>> Query session can be explicitly closed.
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> This could be achieved if we made the
> above
> > > > > >> > > > >> readOnlyView
> > > > > >> > > > >>>>>> interface
> > > > > >> > > > >>>>>>>>>>>> look
> > > > > >> > > > >>>>>>>>>>>>>>> more like:
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueView<K, V>
> > > implements
> > > > > >> > > > >>>>>>>>>>>> ReadOnlyKeyValueStore<K,
> > > > > >> > > > >>>>>>>>>>>>>>> V>, AutoCloseable {}
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > > > > >> > > > >>>>>>>>>>>>>>>         ...
> > > > > >> > > > >>>>>>>>>>>>>>>         ReadOnlyKeyValueView<K, V>
> > > > > >> > > > >>>> readOnlyView(IsolationLevel
> > > > > >> > > > >>>>>>>>>>>>> isolationLevel);
> > > > > >> > > > >>>>>>>>>>>>>>> }
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> But this would be a breaking change, as
> > > existing
> > > > > >> IQv1
> > > > > >> > > > >>>> queries
> > > > > >> > > > >>>>>> are
> > > > > >> > > > >>>>>>>>>>>>>>> guaranteed to never call store.close(),
> and
> > > > > >> therefore
> > > > > >> > > > >> these
> > > > > >> > > > >>>>>> would
> > > > > >> > > > >>>>>>>>>> leak
> > > > > >> > > > >>>>>>>>>>>>>>> memory under REPEATABLE_READ.
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> B.
> > > > > >> > > > >>>>>>>>>>>>>>> One thing that's notable: MyRocks states
> > that
> > > they
> > > > > >> > > > >> support
> > > > > >> > > > >>>>>>>>>>>>> READ_COMMITTED
> > > > > >> > > > >>>>>>>>>>>>>>> and REPEATABLE_READ, but they make no
> > mention
> > > of
> > > > > >> > > > >>>>>>>>>>>> READ_UNCOMMITTED[3][4].
> > > > > >> > > > >>>>>>>>>>>>>>> This could be because doing so is
> > technically
> > > > > >> > > > >>>>>> difficult/impossible
> > > > > >> > > > >>>>>>>>>>>> using
> > > > > >> > > > >>>>>>>>>>>>>>> the primitives available in RocksDB.
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> --
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> Lucas, to address your points:
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> U1.
> > > > > >> > > > >>>>>>>>>>>>>>> It's only "SHOULD" to permit alternative
> > (i.e.
> > > > > >> > > > >> non-RocksDB)
> > > > > >> > > > >>>>>>>>>>>>>>> implementations of StateStore that do not
> > > support
> > > > > >> > > > >> atomic
> > > > > >> > > > >>>>>> writes.
> > > > > >> > > > >>>>>>>>>>>>> Obviously
> > > > > >> > > > >>>>>>>>>>>>>>> in those cases, the guarantees Kafka
> Streams
> > > > > >> > > > >>>> provides/expects
> > > > > >> > > > >>>>>> would
> > > > > >> > > > >>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>> relaxed. Do you think we should require
> all
> > > > > >> > > > >>>> implementations to
> > > > > >> > > > >>>>>>>>>> support
> > > > > >> > > > >>>>>>>>>>>>>>> atomic writes?
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> U2.
> > > > > >> > > > >>>>>>>>>>>>>>> Stores can support multiple
> IsolationLevels.
> > > As
> > > > > >> we've
> > > > > >> > > > >>>> discussed
> > > > > >> > > > >>>>>>>>>> above,
> > > > > >> > > > >>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>> ideal scenario would be to specify the
> > > > > >> IsolationLevel
> > > > > >> > > > >> at
> > > > > >> > > > >>>>>>>> query-time.
> > > > > >> > > > >>>>>>>>>>>>>>> Failing that, I think the second-best
> > > approach is
> > > > > to
> > > > > >> > > > >>>> define the
> > > > > >> > > > >>>>>>>>>>>>>>> IsolationLevel for *all* queries based on
> > the
> > > > > >> > > > >>>> processing.mode,
> > > > > >> > > > >>>>>>>> which
> > > > > >> > > > >>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>> what the default
> > > > > StateStoreContext#isolationLevel()
> > > > > >> > > > >>>> achieves.
> > > > > >> > > > >>>>>> Would
> > > > > >> > > > >>>>>>>>>>>> you
> > > > > >> > > > >>>>>>>>>>>>>>> prefer an alternative?
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> While the existing implementation is
> > > equivalent to
> > > > > >> > > > >>>>>>>> READ_UNCOMMITTED,
> > > > > >> > > > >>>>>>>>>>>>> this
> > > > > >> > > > >>>>>>>>>>>>>>> can yield unexpected results/errors under
> > > EOS, if
> > > > > a
> > > > > >> > > > >>>>>> transaction is
> > > > > >> > > > >>>>>>>>>>>>> rolled
> > > > > >> > > > >>>>>>>>>>>>>>> back. While this would be a change in
> > > behaviour
> > > > > for
> > > > > >> > > > >> users,
> > > > > >> > > > >>>> it
> > > > > >> > > > >>>>>> would
> > > > > >> > > > >>>>>>>>>>>> look
> > > > > >> > > > >>>>>>>>>>>>>>> more like a bug fix than a breaking
> change.
> > > That
> > > > > >> said,
> > > > > >> > > > >> we
> > > > > >> > > > >>>>>> *could*
> > > > > >> > > > >>>>>>>>>> make
> > > > > >> > > > >>>>>>>>>>>>> it
> > > > > >> > > > >>>>>>>>>>>>>>> configurable, and default to the existing
> > > > > behaviour
> > > > > >> > > > >>>>>>>>>> (READ_UNCOMMITTED)
> > > > > >> > > > >>>>>>>>>>>>>>> instead of inferring it from the
> > > processing.mode?
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> N1, N2.
> > > > > >> > > > >>>>>>>>>>>>>>> These were only primitives to avoid boxing
> > > costs,
> > > > > >> but
> > > > > >> > > > >> since
> > > > > >> > > > >>>>>> this is
> > > > > >> > > > >>>>>>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>> a
> > > > > >> > > > >>>>>>>>>>>>>>> performance sensitive area, it should be
> > fine
> > > to
> > > > > >> > > > >> change if
> > > > > >> > > > >>>>>> that's
> > > > > >> > > > >>>>>>>>>>>>> desirable.
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> N3.
> > > > > >> > > > >>>>>>>>>>>>>>> It's because the store "manages its own
> > > offsets",
> > > > > >> which
> > > > > >> > > > >>>>>> includes
> > > > > >> > > > >>>>>>>> both
> > > > > >> > > > >>>>>>>>>>>>>>> committing the offset, *and providing it*
> > via
> > > > > >> > > > >>>>>> getCommittedOffset().
> > > > > >> > > > >>>>>>>>>>>>>>> Personally, I think "managesOffsets"
> conveys
> > > this
> > > > > >> best,
> > > > > >> > > > >>>> but I
> > > > > >> > > > >>>>>> don't
> > > > > >> > > > >>>>>>>>>>>> mind
> > > > > >> > > > >>>>>>>>>>>>>>> changing it if the nomenclature is
> unclear.
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> Sorry for the massive emails/essays!
> > > > > >> > > > >>>>>>>>>>>>>>> --
> > > > > >> > > > >>>>>>>>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> 1:
> > > > > >> > > > >> https://github.com/facebook/rocksdb/wiki/Transactions
> > > > > >> > > > >>>>>>>>>>>>>>> 2:
> > > > > >> https://github.com/facebook/rocksdb/wiki/Snapshot
> > > > > >> > > > >>>>>>>>>>>>>>> 3:
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>
> > > > > https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > > > >> > > > >>>>>>>>>>>>>>> 4:
> > > > > >> > > > >>>>
> > > https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas
> > Brutschy
> > > > > >> > > > >>>>>>>>>>>>>>> <lb...@confluent.io.invalid> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>> Hi Nick,
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>> since I last read it in April, the KIP
> has
> > > become
> > > > > >> much
> > > > > >> > > > >>>>>> cleaner and
> > > > > >> > > > >>>>>>>>>>>>>>>> easier to read. Great work!
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>> It feels to me the last big open point is
> > > whether
> > > > > >> we
> > > > > >> > > > >> can
> > > > > >> > > > >>>>>> implement
> > > > > >> > > > >>>>>>>>>>>>>>>> isolation level as a query parameter. I
> > > > > understand
> > > > > >> > > > >> that
> > > > > >> > > > >>>> there
> > > > > >> > > > >>>>>> are
> > > > > >> > > > >>>>>>>>>>>>>>>> implementation concerns, but as Colt
> says,
> > it
> > > > > would
> > > > > >> > > > >> be a
> > > > > >> > > > >>>> great
> > > > > >> > > > >>>>>>>>>>>>>>>> addition, and would also simplify the
> > > migration
> > > > > >> path
> > > > > >> > > > >> for
> > > > > >> > > > >>>> this
> > > > > >> > > > >>>>>>>>>> change.
> > > > > >> > > > >>>>>>>>>>>>>>>> Is the implementation problem you
> mentioned
> > > > > caused
> > > > > >> by
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>>>> WriteBatch
> > > > > >> > > > >>>>>>>>>>>>>>>> not having a notion of a snapshot, as the
> > > > > >> underlying
> > > > > >> > > > >> DB
> > > > > >> > > > >>>>>> iterator
> > > > > >> > > > >>>>>>>>>>>> does?
> > > > > >> > > > >>>>>>>>>>>>>>>> In that case, I am not sure a chain of
> > > > > WriteBatches
> > > > > >> > > > >> as you
> > > > > >> > > > >>>>>> propose
> > > > > >> > > > >>>>>>>>>>>>>>>> would fully solve the problem, but maybe
> I
> > > didn't
> > > > > >> dig
> > > > > >> > > > >>>> enough
> > > > > >> > > > >>>>>> into
> > > > > >> > > > >>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> details to fully understand it.
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>> If it's not possible to implement it now,
> > > would
> > > > > it
> > > > > >> be
> > > > > >> > > > >> an
> > > > > >> > > > >>>>>> option to
> > > > > >> > > > >>>>>>>>>>>>>>>> make sure in this KIP that we do not
> fully
> > > close
> > > > > >> the
> > > > > >> > > > >> door
> > > > > >> > > > >>>> on
> > > > > >> > > > >>>>>>>>>>>> per-query
> > > > > >> > > > >>>>>>>>>>>>>>>> isolation levels in the interface, as it
> > may
> > > be
> > > > > >> > > > >> possible
> > > > > >> > > > >>>> to
> > > > > >> > > > >>>>>>>>>> implement
> > > > > >> > > > >>>>>>>>>>>>>>>> the missing primitives in RocksDB or
> Speedb
> > > in
> > > > > the
> > > > > >> > > > >> future.
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>> Understanding:
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>> * U1) Why is it only "SHOULD" for
> > > > > changelogOffsets
> > > > > >> to
> > > > > >> > > > >> be
> > > > > >> > > > >>>>>> persisted
> > > > > >> > > > >>>>>>>>>>>>>>>> atomically with the records?
> > > > > >> > > > >>>>>>>>>>>>>>>> * U2) Don't understand the default
> > > implementation
> > > > > >> of
> > > > > >> > > > >>>>>>>>>>>> `isolationLevel`.
> > > > > >> > > > >>>>>>>>>>>>>>>> The isolation level should be a property
> of
> > > the
> > > > > >> > > > >> underlying
> > > > > >> > > > >>>>>> store,
> > > > > >> > > > >>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>> not be defined by the default config?
> > > Existing
> > > > > >> stores
> > > > > >> > > > >>>> probably
> > > > > >> > > > >>>>>>>> don't
> > > > > >> > > > >>>>>>>>>>>>>>>> guarantee READ_COMMITTED, so the default
> > > should
> > > > > be
> > > > > >> to
> > > > > >> > > > >>>> return
> > > > > >> > > > >>>>>>>>>>>>>>>> READ_UNCOMMITTED.
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>> Nits:
> > > > > >> > > > >>>>>>>>>>>>>>>> * N1) Could `getComittedOffset` use an
> > > > > >> `OptionalLong`
> > > > > >> > > > >>>> return
> > > > > >> > > > >>>>>> type,
> > > > > >> > > > >>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> avoid the `null`?
> > > > > >> > > > >>>>>>>>>>>>>>>> * N2) Could
> `apporixmateNumUncomittedBytes`
> > > use
> > > > > an
> > > > > >> > > > >>>>>> `OptionalLong`
> > > > > >> > > > >>>>>>>>>>>>>>>> return type, to avoid the `-1`?
> > > > > >> > > > >>>>>>>>>>>>>>>> * N3) I don't understand why
> > `managesOffsets`
> > > > > uses
> > > > > >> the
> > > > > >> > > > >>>>>> 'manage'
> > > > > >> > > > >>>>>>>>>> verb,
> > > > > >> > > > >>>>>>>>>>>>>>>> whereas all other methods use the
> "commits"
> > > verb.
> > > > > >> I'd
> > > > > >> > > > >>>> suggest
> > > > > >> > > > >>>>>>>>>>>>>>>> `commitsOffsets`.
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>> Either way, it feels this KIP is very
> close
> > > to
> > > > > the
> > > > > >> > > > >> finish
> > > > > >> > > > >>>>>> line,
> > > > > >> > > > >>>>>>>> I'm
> > > > > >> > > > >>>>>>>>>>>>>>>> looking forward to seeing this in
> > production!
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>> Cheers,
> > > > > >> > > > >>>>>>>>>>>>>>>> Lucas
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt
> > McNealy
> > > <
> > > > > >> > > > >>>>>> colt@littlehorse.io
> > > > > >> > > > >>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>> Making IsolationLevel a query-time
> > > constraint,
> > > > > >> > > > >> rather
> > > > > >> > > > >>>> than
> > > > > >> > > > >>>>>>>> linking
> > > > > >> > > > >>>>>>>>>>>> it
> > > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>> the processing.guarantee.
> > > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>> As I understand it, would this allow
> even
> > a
> > > user
> > > > > >> of
> > > > > >> > > > >> EOS
> > > > > >> > > > >>>> to
> > > > > >> > > > >>>>>>>> control
> > > > > >> > > > >>>>>>>>>>>>>>>> whether
> > > > > >> > > > >>>>>>>>>>>>>>>>> reading committed or uncommitted
> records?
> > > If so,
> > > > > >> I am
> > > > > >> > > > >>>> highly
> > > > > >> > > > >>>>>> in
> > > > > >> > > > >>>>>>>>>>>> favor
> > > > > >> > > > >>>>>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>> this.
> > > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>> I know that I was one of the early
> people
> > to
> > > > > point
> > > > > >> > > > >> out
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>>> current
> > > > > >> > > > >>>>>>>>>>>>>>>>> shortcoming that IQ reads uncommitted
> > > records,
> > > > > but
> > > > > >> > > > >> just
> > > > > >> > > > >>>> this
> > > > > >> > > > >>>>>>>>>>>> morning I
> > > > > >> > > > >>>>>>>>>>>>>>>>> realized a pattern we use which means
> that
> > > (for
> > > > > >> > > > >> certain
> > > > > >> > > > >>>>>> queries)
> > > > > >> > > > >>>>>>>>>> our
> > > > > >> > > > >>>>>>>>>>>>>>>> system
> > > > > >> > > > >>>>>>>>>>>>>>>>> needs to be able to read uncommitted
> > > records,
> > > > > >> which
> > > > > >> > > > >> is
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>>> current
> > > > > >> > > > >>>>>>>>>>>>>>>> behavior
> > > > > >> > > > >>>>>>>>>>>>>>>>> of Kafka Streams in EOS.***
> > > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>> If IsolationLevel being a query-time
> > > decision
> > > > > >> allows
> > > > > >> > > > >> for
> > > > > >> > > > >>>>>> this,
> > > > > >> > > > >>>>>>>> then
> > > > > >> > > > >>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>> would be amazing. I would also vote that
> > the
> > > > > >> default
> > > > > >> > > > >>>> behavior
> > > > > >> > > > >>>>>>>>>> should
> > > > > >> > > > >>>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>> for
> > > > > >> > > > >>>>>>>>>>>>>>>>> reading uncommitted records, because it
> is
> > > > > totally
> > > > > >> > > > >>>> possible
> > > > > >> > > > >>>>>> for a
> > > > > >> > > > >>>>>>>>>>>>> valid
> > > > > >> > > > >>>>>>>>>>>>>>>>> application to depend on that behavior,
> > and
> > > > > >> breaking
> > > > > >> > > > >> it
> > > > > >> > > > >>>> in a
> > > > > >> > > > >>>>>>>> minor
> > > > > >> > > > >>>>>>>>>>>>>>>> release
> > > > > >> > > > >>>>>>>>>>>>>>>>> might be a bit strong.
> > > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>> *** (Note, for the curious reader....)
> Our
> > > > > >> > > > >> use-case/query
> > > > > >> > > > >>>>>> pattern
> > > > > >> > > > >>>>>>>>>>>> is a
> > > > > >> > > > >>>>>>>>>>>>>>>> bit
> > > > > >> > > > >>>>>>>>>>>>>>>>> complex, but reading "uncommitted"
> records
> > > is
> > > > > >> > > > >> actually
> > > > > >> > > > >>>> safe
> > > > > >> > > > >>>>>> in
> > > > > >> > > > >>>>>>>> our
> > > > > >> > > > >>>>>>>>>>>>> case
> > > > > >> > > > >>>>>>>>>>>>>>>>> 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 soon as the
> processor
> > > > > >> processes
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>> result.
> > > > > >> > > > >>>>>>>> If
> > > > > >> > > > >>>>>>>>>>>>> they
> > > > > >> > > > >>>>>>>>>>>>>>>>> can't read uncommitted records, they may
> > > > > receive a
> > > > > >> > > > >> "201 -
> > > > > >> > > > >>>>>>>> Created"
> > > > > >> > > > >>>>>>>>>>>>>>>>> response, immediately followed by a
> "404 -
> > > 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 <
> > > > > >> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> > > > > >> > > > >>>>>>>>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>> Addendum:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>> I think we would also face the same
> > problem
> > > > > with
> > > > > >> the
> > > > > >> > > > >>>>>> approach
> > > > > >> > > > >>>>>>>> John
> > > > > >> > > > >>>>>>>>>>>>>>>> outlined
> > > > > >> > > > >>>>>>>>>>>>>>>>>> earlier (using the record cache as a
> > > > > transaction
> > > > > >> > > > >> buffer
> > > > > >> > > > >>>> and
> > > > > >> > > > >>>>>>>>>>>> flushing
> > > > > >> > > > >>>>>>>>>>>>>>>> it
> > > > > >> > > > >>>>>>>>>>>>>>>>>> straight to SST files). This is because
> > the
> > > > > >> record
> > > > > >> > > > >> cache
> > > > > >> > > > >>>>>> (the
> > > > > >> > > > >>>>>>>>>>>>>>>> ThreadCache
> > > > > >> > > > >>>>>>>>>>>>>>>>>> class) is not thread-safe, so every
> > commit
> > > > > would
> > > > > >> > > > >>>> invalidate
> > > > > >> > > > >>>>>> open
> > > > > >> > > > >>>>>>>>>> IQ
> > > > > >> > > > >>>>>>>>>>>>>>>>>> Iterators in the same way that RocksDB
> > > > > >> WriteBatches
> > > > > >> > > > >> do.
> > > > > >> > > > >>>>>>>>>>>>>>>>>> --
> > > > > >> > > > >>>>>>>>>>>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick
> > > Telford <
> > > > > >> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> > > > > >> > > > >>>>>>>>>>>>>>>>>> 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.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> 2. Making IsolationLevel a query-time
> > > > > >> constraint,
> > > > > >> > > > >>>> rather
> > > > > >> > > > >>>>>> than
> > > > > >> > > > >>>>>>>>>>>>>>>> linking it
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> to the processing.guarantee.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> There's a wrinkle that makes this a
> > > challenge:
> > > > > >> > > > >>>> Interactive
> > > > > >> > > > >>>>>>>>>> Queries
> > > > > >> > > > >>>>>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> open an Iterator, when using
> > transactions
> > > and
> > > > > >> > > > >>>>>> READ_UNCOMMITTED.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> The problem is that under
> > > READ_UNCOMMITTED,
> > > > > >> queries
> > > > > >> > > > >>>> need
> > > > > >> > > > >>>>>> to be
> > > > > >> > > > >>>>>>>>>>>> able
> > > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> read records from the currently
> > > uncommitted
> > > > > >> > > > >> transaction
> > > > > >> > > > >>>>>> buffer
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> (WriteBatch). This includes for
> > Iterators,
> > > > > which
> > > > > >> > > > >> should
> > > > > >> > > > >>>>>> iterate
> > > > > >> > > > >>>>>>>>>>>>>>>> both the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> transaction buffer and underlying
> > database
> > > > > >> (using
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> The issue is that when the
> StreamThread
> > > > > >> commits, it
> > > > > >> > > > >>>> writes
> > > > > >> > > > >>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> current
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch to RocksDB *and then clears
> > the
> > > > > >> > > > >> WriteBatch*.
> > > > > >> > > > >>>>>>>> Clearing
> > > > > >> > > > >>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch while an Interactive Query
> > > holds an
> > > > > >> open
> > > > > >> > > > >>>>>> Iterator on
> > > > > >> > > > >>>>>>>>>> it
> > > > > >> > > > >>>>>>>>>>>>>>>> will
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> invalidate the Iterator. Worse, it
> turns
> > > out
> > > > > >> that
> > > > > >> > > > >>>> Iterators
> > > > > >> > > > >>>>>>>> over
> > > > > >> > > > >>>>>>>>>> a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch become invalidated not just
> > > when
> > > > > the
> > > > > >> > > > >>>> WriteBatch
> > > > > >> > > > >>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>> cleared,
> > > > > >> > > > >>>>>>>>>>>>>>>>>> but
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> also when the Iterators' current key
> > > receives
> > > > > a
> > > > > >> new
> > > > > >> > > > >>>> write.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> Now that I'm writing this, I remember
> > that
> > > > > this
> > > > > >> is
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>> major
> > > > > >> > > > >>>>>>>>>>>> reason
> > > > > >> > > > >>>>>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>> I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> switched the original design from
> > having a
> > > > > >> > > > >> query-time
> > > > > >> > > > >>>>>>>>>>>>>>>> IsolationLevel to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> having the IsolationLevel linked to
> the
> > > > > >> > > > >>>> transactionality
> > > > > >> > > > >>>>>> of the
> > > > > >> > > > >>>>>>>>>>>>>>>> stores
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> themselves.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> It *might* be possible to resolve
> this,
> > by
> > > > > >> having a
> > > > > >> > > > >>>>>> "chain" of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatches, with the StreamThread
> > > switching
> > > > > >> to a
> > > > > >> > > > >> new
> > > > > >> > > > >>>>>>>> WriteBatch
> > > > > >> > > > >>>>>>>>>>>>>>>>>> whenever
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> a new Interactive Query attempts to
> read
> > > from
> > > > > >> the
> > > > > >> > > > >>>>>> database, but
> > > > > >> > > > >>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>> could
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> cause some performance problems/memory
> > > > > pressure
> > > > > >> > > > >> when
> > > > > >> > > > >>>>>> subjected
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>> a
> > > > > >> > > > >>>>>>>>>>>>>>>> high
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> Interactive Query load. It would also
> > > reduce
> > > > > the
> > > > > >> > > > >>>>>> efficiency of
> > > > > >> > > > >>>>>>>>>>>>>>>>>> WriteBatches
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> on-commit, as we'd have to write N
> > > > > WriteBatches,
> > > > > >> > > > >> where
> > > > > >> > > > >>>> N
> > > > > >> > > > >>>>>> is the
> > > > > >> > > > >>>>>>>>>>>>>>>> number of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> Interactive Queries since the last
> > commit.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> I realise this is getting into the
> weeds
> > > of
> > > > > the
> > > > > >> > > > >>>>>> implementation,
> > > > > >> > > > >>>>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>> you'd
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> rather we focus on the API for now,
> but
> > I
> > > > > think
> > > > > >> > > > >> it's
> > > > > >> > > > >>>>>> important
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>> consider
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> how to implement the desired API, in
> > case
> > > we
> > > > > >> come
> > > > > >> > > > >> up
> > > > > >> > > > >>>> with
> > > > > >> > > > >>>>>> an
> > > > > >> > > > >>>>>>>> API
> > > > > >> > > > >>>>>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> cannot be implemented efficiently, or
> > > even at
> > > > > >> all!
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> Thoughts?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> --
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno
> > > Cadonna <
> > > > > >> > > > >>>>>>>> cadonna@apache.org
> > > > > >> > > > >>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> 6.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> Of course, you are right! My bad!
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> Wiping out the state in the
> downgrading
> > > case
> > > > > is
> > > > > >> > > > >> fine.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> 3a.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> Focus on the public facing changes
> for
> > > the
> > > > > >> KIP. We
> > > > > >> > > > >>>> will
> > > > > >> > > > >>>>>> manage
> > > > > >> > > > >>>>>>>>>> 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.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> Best,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> Bruno
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford
> > wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Thanks for getting back to me!
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> 2.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> The fact that implementations can
> > always
> > > > > track
> > > > > >> > > > >>>> estimated
> > > > > >> > > > >>>>>>>> memory
> > > > > >> > > > >>>>>>>>>>>>>>>> usage
> > > > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> the wrapper is a good point. I can
> > > remove -1
> > > > > >> as
> > > > > >> > > > >> an
> > > > > >> > > > >>>>>> option,
> > > > > >> > > > >>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>> I'll
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> clarify
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> the JavaDoc that 0 is not just for
> > > > > >> > > > >> non-transactional
> > > > > >> > > > >>>>>> stores,
> > > > > >> > > > >>>>>>>>>>>>>>>> which is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> currently misleading.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> 6.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> The problem with catching the
> > exception
> > > in
> > > > > the
> > > > > >> > > > >>>> downgrade
> > > > > >> > > > >>>>>>>>>> process
> > > > > >> > > > >>>>>>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> would require new code in the Kafka
> > > version
> > > > > >> being
> > > > > >> > > > >>>>>> downgraded
> > > > > >> > > > >>>>>>>>>> to.
> > > > > >> > > > >>>>>>>>>>>>>>>> Since
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> users could conceivably downgrade to
> > > almost
> > > > > >> *any*
> > > > > >> > > > >>>> older
> > > > > >> > > > >>>>>>>> version
> > > > > >> > > > >>>>>>>>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>>> Kafka
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Streams, I'm not sure how we could
> add
> > > that
> > > > > >> code?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> The only way I can think of doing it
> > > would
> > > > > be
> > > > > >> to
> > > > > >> > > > >>>> provide
> > > > > >> > > > >>>>>> a
> > > > > >> > > > >>>>>>>>>>>>>>>> dedicated
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> downgrade tool, that goes through
> > every
> > > > > local
> > > > > >> > > > >> store
> > > > > >> > > > >>>> and
> > > > > >> > > > >>>>>>>> removes
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> offsets column families. But that
> > seems
> > > like
> > > > > >> an
> > > > > >> > > > >>>>>> unnecessary
> > > > > >> > > > >>>>>>>>>>>>>>>> amount of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> extra
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> code to maintain just to handle a
> > > somewhat
> > > > > >> niche
> > > > > >> > > > >>>>>> situation,
> > > > > >> > > > >>>>>>>>>> when
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> alternative (automatically wipe and
> > > restore
> > > > > >> > > > >> stores)
> > > > > >> > > > >>>>>> should be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> acceptable.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the
> changes
> > > > > you've
> > > > > >> > > > >>>> requested.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> 3a.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> I agree that IsolationLevel makes
> more
> > > sense
> > > > > >> at
> > > > > >> > > > >>>>>> query-time,
> > > > > >> > > > >>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>> I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> actually
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> initially attempted to place the
> > > > > >> IsolationLevel
> > > > > >> > > > >> at
> > > > > >> > > > >>>>>>>> query-time,
> > > > > >> > > > >>>>>>>>>>>>>>>> but I
> > > > > >> > > > >>>>>>>>>>>>>>>>>> ran
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> into some problems:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> - The key issue is that, under ALOS
> > > we're
> > > > > not
> > > > > >> > > > >> staging
> > > > > >> > > > >>>>>> writes
> > > > > >> > > > >>>>>>>> in
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> transactions, so can't perform
> writes
> > > at the
> > > > > >> > > > >>>>>> READ_COMMITTED
> > > > > >> > > > >>>>>>>>>>>>>>>> isolation
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> level. However, this may be
> addressed
> > > if we
> > > > > >> > > > >> decide to
> > > > > >> > > > >>>>>>>> *always*
> > > > > >> > > > >>>>>>>>>>>>>>>> use
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> transactions as discussed under 3b.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> > > > > >> > > > >>>> implementations. I
> > > > > >> > > > >>>>>>>>>> remember
> > > > > >> > > > >>>>>>>>>>>>>>>>>> having
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> some difficulty understanding the
> IQv1
> > > > > >> internals,
> > > > > >> > > > >>>> which
> > > > > >> > > > >>>>>> made
> > > > > >> > > > >>>>>>>> it
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> difficult
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> to determine what needed to be
> > changed.
> > > > > >> However,
> > > > > >> > > > >> I
> > > > > >> > > > >>>>>> *think*
> > > > > >> > > > >>>>>>>> this
> > > > > >> > > > >>>>>>>>>>>>>>>> can be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> addressed for both implementations
> by
> > > > > wrapping
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>>>> RocksDBStore
> > > > > >> > > > >>>>>>>>>>>>>>>> in an
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> IsolationLevel-dependent wrapper,
> that
> > > > > >> overrides
> > > > > >> > > > >> read
> > > > > >> > > > >>>>>> methods
> > > > > >> > > > >>>>>>>>>>>>>>>> (get,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> etc.)
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> to either read directly from the
> > > database or
> > > > > >> > > > >> from the
> > > > > >> > > > >>>>>> ongoing
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> But IQv1 might still be difficult.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> - If IsolationLevel becomes a query
> > > > > >> constraint,
> > > > > >> > > > >> then
> > > > > >> > > > >>>> all
> > > > > >> > > > >>>>>>>> other
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> StateStores
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> will need to respect it, including
> the
> > > > > >> in-memory
> > > > > >> > > > >>>> stores.
> > > > > >> > > > >>>>>> This
> > > > > >> > > > >>>>>>>>>>>>>>>> would
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> require
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> us to adapt in-memory stores to
> stage
> > > their
> > > > > >> > > > >> writes so
> > > > > >> > > > >>>>>> they
> > > > > >> > > > >>>>>>>> can
> > > > > >> > > > >>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> isolated
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> from READ_COMMITTTED queries. It
> would
> > > also
> > > > > >> > > > >> become an
> > > > > >> > > > >>>>>>>> important
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> consideration for third-party stores
> > on
> > > > > >> upgrade,
> > > > > >> > > > >> as
> > > > > >> > > > >>>>>> without
> > > > > >> > > > >>>>>>>>>>>>>>>> changes,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> they
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> would not support READ_COMMITTED
> > queries
> > > > > >> > > > >> correctly.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Ultimately, I may need some help
> > making
> > > the
> > > > > >> > > > >> necessary
> > > > > >> > > > >>>>>> change
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> IQv1
> > > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> support this, but I don't think it's
> > > > > >> > > > >> fundamentally
> > > > > >> > > > >>>>>>>> impossible,
> > > > > >> > > > >>>>>>>>>>>>>>>> if we
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> want
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> to pursue this route.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> 3b.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> The main reason I chose to keep ALOS
> > > > > >> > > > >> un-transactional
> > > > > >> > > > >>>>>> was to
> > > > > >> > > > >>>>>>>>>>>>>>>> minimize
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> behavioural change for most users (I
> > > believe
> > > > > >> most
> > > > > >> > > > >>>> Streams
> > > > > >> > > > >>>>>>>> users
> > > > > >> > > > >>>>>>>>>>>>>>>> use
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> default configuration, which is
> ALOS).
> > > That
> > > > > >> said,
> > > > > >> > > > >>>> it's
> > > > > >> > > > >>>>>> clear
> > > > > >> > > > >>>>>>>>>>>>>>>> that if
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> ALOS
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> also used transactional stores, the
> > only
> > > > > >> change
> > > > > >> > > > >> in
> > > > > >> > > > >>>>>> behaviour
> > > > > >> > > > >>>>>>>>>>>>>>>> would be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> it would become *more correct*,
> which
> > > could
> > > > > be
> > > > > >> > > > >>>>>> considered a
> > > > > >> > > > >>>>>>>>>> "bug
> > > > > >> > > > >>>>>>>>>>>>>>>> fix"
> > > > > >> > > > >>>>>>>>>>>>>>>>>> by
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> users, rather than a change they
> need
> > to
> > > > > >> handle.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> I believe that performance using
> > > > > transactions
> > > > > >> > > > >> (aka.
> > > > > >> > > > >>>>>> RocksDB
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> WriteBatches)
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> should actually be *better* than the
> > > > > >> un-batched
> > > > > >> > > > >>>>>> write-path
> > > > > >> > > > >>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> currently used[1]. The only
> > > "performance"
> > > > > >> > > > >>>> consideration
> > > > > >> > > > >>>>>> will
> > > > > >> > > > >>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> increased memory usage that
> > transactions
> > > > > >> require.
> > > > > >> > > > >>>> Given
> > > > > >> > > > >>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>> mitigations
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> for
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> this memory that we have in place, I
> > > would
> > > > > >> expect
> > > > > >> > > > >>>> that
> > > > > >> > > > >>>>>> this
> > > > > >> > > > >>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>> not a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> problem for most users.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> If we're happy to do so, we can make
> > > ALOS
> > > > > also
> > > > > >> > > > >> use
> > > > > >> > > > >>>>>>>>>> transactions.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Link 1:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>
> > > > > >> > > > >>
> > > > > >> > >
> > > > > >>
> > >
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno
> > > > > Cadonna <
> > > > > >> > > > >>>>>>>>>>>> cadonna@apache.org
> > > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Thanks for the updates and sorry
> for
> > > the
> > > > > >> delay
> > > > > >> > > > >> on my
> > > > > >> > > > >>>>>> side!
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Making the default implementation
> for
> > > > > >> flush() a
> > > > > >> > > > >>>> no-op
> > > > > >> > > > >>>>>> sounds
> > > > > >> > > > >>>>>>>>>>>>>>>> good to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> me.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I think what was bugging me here is
> > > that a
> > > > > >> > > > >>>> third-party
> > > > > >> > > > >>>>>> state
> > > > > >> > > > >>>>>>>>>>>>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> needs
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> to implement the state store
> > interface.
> > > > > That
> > > > > >> > > > >> means
> > > > > >> > > > >>>> they
> > > > > >> > > > >>>>>> need
> > > > > >> > > > >>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> implement a wrapper around the
> actual
> > > state
> > > > > >> > > > >> store
> > > > > >> > > > >>>> as we
> > > > > >> > > > >>>>>> do
> > > > > >> > > > >>>>>>>> for
> > > > > >> > > > >>>>>>>>>>>>>>>>>> RocksDB
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> with RocksDBStore. So, a
> third-party
> > > state
> > > > > >> > > > >> store can
> > > > > >> > > > >>>>>> always
> > > > > >> > > > >>>>>>>>>>>>>>>> estimate
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> uncommitted bytes, if it wants,
> > > because the
> > > > > >> > > > >> wrapper
> > > > > >> > > > >>>> can
> > > > > >> > > > >>>>>>>> record
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> added
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> bytes.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> One case I can think of where
> > > returning -1
> > > > > >> makes
> > > > > >> > > > >>>> sense
> > > > > >> > > > >>>>>> is
> > > > > >> > > > >>>>>>>> when
> > > > > >> > > > >>>>>>>>>>>>>>>>>> Streams
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> does not need to estimate the size
> of
> > > the
> > > > > >> write
> > > > > >> > > > >>>> batch
> > > > > >> > > > >>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>> trigger
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> extraordinary commits, because the
> > > > > >> third-party
> > > > > >> > > > >> state
> > > > > >> > > > >>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>> takes care
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memory. But in that case the method
> > > could
> > > > > >> also
> > > > > >> > > > >> just
> > > > > >> > > > >>>>>> return
> > > > > >> > > > >>>>>>>> 0.
> > > > > >> > > > >>>>>>>>>>>>>>>> Even
> > > > > >> > > > >>>>>>>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> case would be better solved with a
> > > method
> > > > > >> that
> > > > > >> > > > >>>> returns
> > > > > >> > > > >>>>>>>> whether
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> store manages itself the memory
> used
> > > for
> > > > > >> > > > >> uncommitted
> > > > > >> > > > >>>>>> bytes
> > > > > >> > > > >>>>>>>> or
> > > > > >> > > > >>>>>>>>>>>>>>>> not.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Said that, I am fine with keeping
> the
> > > -1
> > > > > >> return
> > > > > >> > > > >>>> value,
> > > > > >> > > > >>>>>> I was
> > > > > >> > > > >>>>>>>>>>>>>>>> just
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> wondering when and if it will be
> > used.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Regarding returning 0 for
> > transactional
> > > > > state
> > > > > >> > > > >> stores
> > > > > >> > > > >>>>>> when
> > > > > >> > > > >>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> batch
> > > > > >> > > > >>>>>>>>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> empty, I was just wondering because
> > you
> > > > > >> > > > >> explicitly
> > > > > >> > > > >>>>>> stated
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> "or {@code 0} if this StateStore
> does
> > > not
> > > > > >> > > > >> support
> > > > > >> > > > >>>>>>>>>>>> transactions."
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> So it seemed to me returning 0
> could
> > > only
> > > > > >> > > > >> happen for
> > > > > >> > > > >>>>>>>>>>>>>>>>>> non-transactional
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> state stores.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a) What do you think if we move the
> > > > > isolation
> > > > > >> > > > >> level
> > > > > >> > > > >>>> to
> > > > > >> > > > >>>>>> IQ
> > > > > >> > > > >>>>>>>> (v1
> > > > > >> > > > >>>>>>>>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>> v2)?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> In the end this is the only
> component
> > > that
> > > > > >> > > > >> really
> > > > > >> > > > >>>> needs
> > > > > >> > > > >>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> specify
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> isolation level. It is similar to
> the
> > > Kafka
> > > > > >> > > > >> consumer
> > > > > >> > > > >>>>>> that
> > > > > >> > > > >>>>>>>> can
> > > > > >> > > > >>>>>>>>>>>>>>>> choose
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> with what isolation level to read
> the
> > > input
> > > > > >> > > > >> topic.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> For IQv1 the isolation level should
> > go
> > > into
> > > > > >> > > > >>>>>>>>>>>>>>>> StoreQueryParameters. For
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> IQv2, I would add it to the Query
> > > > > interface.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> b) Point a) raises the question
> what
> > > should
> > > > > >> > > > >> happen
> > > > > >> > > > >>>>>> during
> > > > > >> > > > >>>>>>>>>>>>>>>>>> at-least-once
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing when the state store
> does
> > > not
> > > > > use
> > > > > >> > > > >>>>>> transactions?
> > > > > >> > > > >>>>>>>>>> John
> > > > > >> > > > >>>>>>>>>>>>>>>> in
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> past proposed to also use
> > transactions
> > > on
> > > > > >> state
> > > > > >> > > > >>>> stores
> > > > > >> > > > >>>>>> for
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> at-least-once. I like that idea,
> > > because it
> > > > > >> > > > >> avoids
> > > > > >> > > > >>>>>>>> aggregating
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>> same
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> records over and over again in the
> > > case of
> > > > > a
> > > > > >> > > > >>>> failure. We
> > > > > >> > > > >>>>>>>> had a
> > > > > >> > > > >>>>>>>>>>>>>>>> case
> > > > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the past where a Streams
> applications
> > > in
> > > > > >> > > > >>>> at-least-once
> > > > > >> > > > >>>>>> mode
> > > > > >> > > > >>>>>>>>>> was
> > > > > >> > > > >>>>>>>>>>>>>>>>>> failing
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> continuously for some reasons I do
> > not
> > > > > >> remember
> > > > > >> > > > >>>> before
> > > > > >> > > > >>>>>>>>>>>>>>>> committing the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> offsets. After each failover, the
> app
> > > > > >> aggregated
> > > > > >> > > > >>>> again
> > > > > >> > > > >>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>> again the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> same records. Of course the
> aggregate
> > > > > >> increased
> > > > > >> > > > >> to
> > > > > >> > > > >>>> very
> > > > > >> > > > >>>>>>>> wrong
> > > > > >> > > > >>>>>>>>>>>>>>>> values
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> just because of the failover. With
> > > > > >> transactions
> > > > > >> > > > >> on
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>> state
> > > > > >> > > > >>>>>>>>>>>>>>>> stores
> > > > > >> > > > >>>>>>>>>>>>>>>>>> we
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> could have avoided this. The app
> > would
> > > have
> > > > > >> > > > >> output
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>> same
> > > > > >> > > > >>>>>>>>>>>>>>>> aggregate
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> multiple times (i.e., after each
> > > failover)
> > > > > >> but
> > > > > >> > > > >> at
> > > > > >> > > > >>>> least
> > > > > >> > > > >>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> value of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> aggregate would not depend on the
> > > number of
> > > > > >> > > > >>>> failovers.
> > > > > >> > > > >>>>>>>>>>>>>>>> Outputting the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> same aggregate multiple times would
> > be
> > > > > >> incorrect
> > > > > >> > > > >>>> under
> > > > > >> > > > >>>>>>>>>>>>>>>> exactly-once
> > > > > >> > > > >>>>>>>>>>>>>>>>>> but
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> it is OK for at-least-once.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> If it makes sense to add a config
> to
> > > turn
> > > > > on
> > > > > >> > > > >> and off
> > > > > >> > > > >>>>>>>>>>>>>>>> transactions on
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> state stores under at-least-once or
> > > just
> > > > > use
> > > > > >> > > > >>>>>> transactions in
> > > > > >> > > > >>>>>>>>>>>>>>>> any case
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a question we should also discuss
> in
> > > this
> > > > > >> KIP.
> > > > > >> > > > >> It
> > > > > >> > > > >>>>>> depends a
> > > > > >> > > > >>>>>>>>>> bit
> > > > > >> > > > >>>>>>>>>>>>>>>> on
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> performance trade-off. Maybe to be
> > > safe, I
> > > > > >> would
> > > > > >> > > > >>>> add a
> > > > > >> > > > >>>>>>>> config.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 4.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Your points are all valid. I tend
> to
> > > say to
> > > > > >> > > > >> keep the
> > > > > >> > > > >>>>>> metrics
> > > > > >> > > > >>>>>>>>>>>>>>>> around
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() until we remove flush()
> > > completely
> > > > > >> from
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>>>> interface.
> > > > > >> > > > >>>>>>>>>>>>>>>> Calls
> > > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() might still exist since
> > > existing
> > > > > >> > > > >> processors
> > > > > >> > > > >>>>>> might
> > > > > >> > > > >>>>>>>>>> still
> > > > > >> > > > >>>>>>>>>>>>>>>> call
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() explicitly as you mentioned
> > in
> > > 1).
> > > > > >> For
> > > > > >> > > > >>>> sure, we
> > > > > >> > > > >>>>>> need
> > > > > >> > > > >>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> document
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> how the metrics change due to the
> > > > > >> transactions
> > > > > >> > > > >> in
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>>> upgrade
> > > > > >> > > > >>>>>>>>>>>>>>>> notes.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 5.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I see. Then you should describe how
> > the
> > > > > >> > > > >> .position
> > > > > >> > > > >>>> files
> > > > > >> > > > >>>>>> are
> > > > > >> > > > >>>>>>>>>>>>>>>> handled
> > > > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a dedicated section of the KIP or
> > > > > incorporate
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>>>> description
> > > > > >> > > > >>>>>>>>>>>>>>>> in the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> "Atomic Checkpointing" section
> > instead
> > > of
> > > > > >> only
> > > > > >> > > > >>>>>> mentioning it
> > > > > >> > > > >>>>>>>>>> in
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> "Compatibility, 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:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>
> > > > > >> > > > >>
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > >
> >
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> It is a bit brittle, but it works.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford
> > > wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for taking the time to
> review
> > > the
> > > > > >> KIP.
> > > > > >> > > > >> I'm
> > > > > >> > > > >>>> back
> > > > > >> > > > >>>>>> from
> > > > > >> > > > >>>>>>>>>>>>>>>> leave
> > > > > >> > > > >>>>>>>>>>>>>>>>>> now
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> intend to move this forwards as
> > > quickly
> > > > > as I
> > > > > >> > > > >> can.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Addressing your points:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Because flush() is part of the
> > > StateStore
> > > > > >> API,
> > > > > >> > > > >> it's
> > > > > >> > > > >>>>>> exposed
> > > > > >> > > > >>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>> custom
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors, which might be making
> > > calls to
> > > > > >> > > > >> flush().
> > > > > >> > > > >>>>>> This
> > > > > >> > > > >>>>>>>> was
> > > > > >> > > > >>>>>>>>>>>>>>>>>> actually
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> case in a few integration tests.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> To maintain as much compatibility
> as
> > > > > >> possible,
> > > > > >> > > > >> I'd
> > > > > >> > > > >>>>>> prefer
> > > > > >> > > > >>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>> make
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> this
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> an UnsupportedOperationException,
> as
> > > it
> > > > > will
> > > > > >> > > > >> cause
> > > > > >> > > > >>>>>>>> previously
> > > > > >> > > > >>>>>>>>>>>>>>>>>> working
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors to start throwing
> > > exceptions at
> > > > > >> > > > >> runtime.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that it doesn't make sense
> > > for it
> > > > > to
> > > > > >> > > > >> proxy
> > > > > >> > > > >>>>>>>> commit(),
> > > > > >> > > > >>>>>>>>>>>>>>>> though,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> as
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> that would cause it to violate the
> > > > > >> "StateStores
> > > > > >> > > > >>>> commit
> > > > > >> > > > >>>>>> only
> > > > > >> > > > >>>>>>>>>>>>>>>> when the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> Task
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commits" rule.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Instead, I think we should make
> > this a
> > > > > >> no-op.
> > > > > >> > > > >> That
> > > > > >> > > > >>>> way,
> > > > > >> > > > >>>>>>>>>>>>>>>> existing
> > > > > >> > > > >>>>>>>>>>>>>>>>>> user
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors will continue to work
> > > > > as-before,
> > > > > >> > > > >> without
> > > > > >> > > > >>>>>>>> violation
> > > > > >> > > > >>>>>>>>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> consistency that would be caused
> by
> > > > > >> premature
> > > > > >> > > > >>>>>> flush/commit
> > > > > >> > > > >>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> StateStore
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> data to disk.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> What do you think?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> As stated in the JavaDoc, when a
> > > > > StateStore
> > > > > >> > > > >>>>>> implementation
> > > > > >> > > > >>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, but is unable to
> > > estimate
> > > > > the
> > > > > >> > > > >>>>>> uncommitted
> > > > > >> > > > >>>>>>>>>>>> memory
> > > > > >> > > > >>>>>>>>>>>>>>>>>> usage,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> method will return -1.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> The intention here is to permit
> > > > > third-party
> > > > > >> > > > >>>>>> implementations
> > > > > >> > > > >>>>>>>>>>>>>>>> that may
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> able to estimate memory usage.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has
> > > been
> > > > > >> > > > >> written to
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>> yet. I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> thought
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> that was implied by "This method
> > will
> > > > > >> return an
> > > > > >> > > > >>>>>>>> approximation
> > > > > >> > > > >>>>>>>>>>>>>>>> of the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memory
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> would be freed by the next call to
> > > {@link
> > > > > >> > > > >>>>>> #commit(Map)}"
> > > > > >> > > > >>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>> "@return
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> The
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> approximate size of all records
> > > awaiting
> > > > > >> {@link
> > > > > >> > > > >>>>>>>>>>>> #commit(Map)}",
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> however,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> can add it explicitly to the
> JavaDoc
> > > if
> > > > > you
> > > > > >> > > > >> think
> > > > > >> > > > >>>> this
> > > > > >> > > > >>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>> unclear?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I realise this is probably the
> most
> > > > > >> contentious
> > > > > >> > > > >>>> point
> > > > > >> > > > >>>>>> in my
> > > > > >> > > > >>>>>>>>>>>>>>>> design,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I'm
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> open to changing it if I'm unable
> to
> > > > > >> convince
> > > > > >> > > > >> you
> > > > > >> > > > >>>> of
> > > > > >> > > > >>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> benefits.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s)
> > are
> > > > > >> directly
> > > > > >> > > > >>>> provided
> > > > > >> > > > >>>>>>>>>>>>>>>> StateStores
> > > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> query, and it may be important for
> > > users
> > > > > to
> > > > > >> > > > >>>>>>>> programmatically
> > > > > >> > > > >>>>>>>>>>>>>>>> know
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> which
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> mode the StateStore is operating
> > > under. If
> > > > > >> we
> > > > > >> > > > >>>> simply
> > > > > >> > > > >>>>>>>> provide
> > > > > >> > > > >>>>>>>>>>>> an
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "eosEnabled" boolean (as used
> > > throughout
> > > > > the
> > > > > >> > > > >>>> internal
> > > > > >> > > > >>>>>>>> streams
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> engine), or
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> similar, then users will need to
> > > > > understand
> > > > > >> the
> > > > > >> > > > >>>>>> operation
> > > > > >> > > > >>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> consequences
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> of each available processing mode
> > and
> > > how
> > > > > it
> > > > > >> > > > >>>> pertains
> > > > > >> > > > >>>>>> to
> > > > > >> > > > >>>>>>>>>> their
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Interactive Query users aren't the
> > > only
> > > > > >> people
> > > > > >> > > > >> that
> > > > > >> > > > >>>>>> care
> > > > > >> > > > >>>>>>>>>> about
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> processing.mode/IsolationLevel of
> a
> > > > > >> StateStore:
> > > > > >> > > > >>>>>>>> implementers
> > > > > >> > > > >>>>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>>> custom
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStores also need to
> understand
> > > the
> > > > > >> > > > >> behaviour
> > > > > >> > > > >>>>>> expected
> > > > > >> > > > >>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>> their
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> implementation. KIP-892 introduces
> > > some
> > > > > >> > > > >> assumptions
> > > > > >> > > > >>>>>> into
> > > > > >> > > > >>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> Streams
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Engine
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> about how StateStores operate
> under
> > > each
> > > > > >> > > > >> processing
> > > > > >> > > > >>>>>> mode,
> > > > > >> > > > >>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>> it's
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> important that custom
> > implementations
> > > > > >> adhere to
> > > > > >> > > > >>>> those
> > > > > >> > > > >>>>>>>>>>>>>>>> assumptions in
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> order
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> to maintain the consistency
> > > guarantees.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> IsolationLevels provide a
> high-level
> > > > > >> contract
> > > > > >> > > > >> on
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>>>>> behaviour
> > > > > >> > > > >>>>>>>>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStore: a user knows that
> under
> > > > > >> > > > >> READ_COMMITTED,
> > > > > >> > > > >>>>>> they
> > > > > >> > > > >>>>>>>> will
> > > > > >> > > > >>>>>>>>>>>>>>>> see
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> writes
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> only after the Task has committed,
> > and
> > > > > under
> > > > > >> > > > >>>>>>>> READ_UNCOMMITTED
> > > > > >> > > > >>>>>>>>>>>>>>>> they
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> see
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> writes immediately. No
> understanding
> > > of
> > > > > the
> > > > > >> > > > >>>> details of
> > > > > >> > > > >>>>>> each
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing.mode
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> is required, either for IQ users
> or
> > > > > >> StateStore
> > > > > >> > > > >>>>>>>> implementers.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> An argument can be made that these
> > > > > >> contractual
> > > > > >> > > > >>>>>> guarantees
> > > > > >> > > > >>>>>>>> can
> > > > > >> > > > >>>>>>>>>>>>>>>> simply
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> documented for the processing.mode
> > > (i.e.
> > > > > >> that
> > > > > >> > > > >>>>>> exactly-once
> > > > > >> > > > >>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> exactly-once-v2 behave like
> > > READ_COMMITTED
> > > > > >> and
> > > > > >> > > > >>>>>>>> at-least-once
> > > > > >> > > > >>>>>>>>>>>>>>>> behaves
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> like
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are
> > > several
> > > > > >> small
> > > > > >> > > > >>>> issues
> > > > > >> > > > >>>>>> with
> > > > > >> > > > >>>>>>>>>>>>>>>> this I'd
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> prefer
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> to avoid:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - Where would we document
> > > these
> > > > > >> > > > >> contracts,
> > > > > >> > > > >>>> in
> > > > > >> > > > >>>>>> a way
> > > > > >> > > > >>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> difficult
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          for users/implementers to
> > > > > >> miss/ignore?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It's not clear to users
> > > that
> > > > > the
> > > > > >> > > > >>>> processing
> > > > > >> > > > >>>>>> mode
> > > > > >> > > > >>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> communicating
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          an expectation of read
> > > isolation,
> > > > > >> > > > >> unless
> > > > > >> > > > >>>> they
> > > > > >> > > > >>>>>> read
> > > > > >> > > > >>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> documentation. Users
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          rarely consult
> > documentation
> > > > > unless
> > > > > >> > > > >> they
> > > > > >> > > > >>>> feel
> > > > > >> > > > >>>>>> they
> > > > > >> > > > >>>>>>>>>> need
> > > > > >> > > > >>>>>>>>>>>>>>>> to, so
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> it's
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> likely
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          this detail would get
> > missed
> > > by
> > > > > >> many
> > > > > >> > > > >> users.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It tightly couples
> > > processing
> > > > > >> modes
> > > > > >> > > > >> to
> > > > > >> > > > >>>> read
> > > > > >> > > > >>>>>>>>>> isolation.
> > > > > >> > > > >>>>>>>>>>>>>>>> Adding
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> new
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          processing modes, or
> > > changing the
> > > > > >> read
> > > > > >> > > > >>>>>> isolation of
> > > > > >> > > > >>>>>>>>>>>>>>>> existing
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          modes would be
> > > > > >> difficult/impossible.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Ultimately, the cost of
> introducing
> > > > > >> > > > >>>> IsolationLevels is
> > > > > >> > > > >>>>>>>> just a
> > > > > >> > > > >>>>>>>>>>>>>>>> single
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> method, since we re-use the
> existing
> > > > > >> > > > >> IsolationLevel
> > > > > >> > > > >>>>>> enum
> > > > > >> > > > >>>>>>>> from
> > > > > >> > > > >>>>>>>>>>>>>>>> Kafka.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> This
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> gives us a clear place to document
> > the
> > > > > >> > > > >> contractual
> > > > > >> > > > >>>>>>>> guarantees
> > > > > >> > > > >>>>>>>>>>>>>>>>>> expected
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> of/provided by StateStores, that
> is
> > > > > >> accessible
> > > > > >> > > > >>>> both by
> > > > > >> > > > >>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>> StateStore
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> itself, and by IQ users.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> (Writing this I've just realised
> > that
> > > the
> > > > > >> > > > >>>> StateStore
> > > > > >> > > > >>>>>> and IQ
> > > > > >> > > > >>>>>>>>>>>>>>>> APIs
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> actually
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> don't provide access to
> > > StateStoreContext
> > > > > >> that
> > > > > >> > > > >> IQ
> > > > > >> > > > >>>> users
> > > > > >> > > > >>>>>>>> would
> > > > > >> > > > >>>>>>>>>>>>>>>> have
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> direct
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> access to... Perhaps StateStore
> > should
> > > > > >> expose
> > > > > >> > > > >>>>>>>>>> isolationLevel()
> > > > > >> > > > >>>>>>>>>>>>>>>>>> itself
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> too?)
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 4.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming
> > the
> > > > > >> metrics
> > > > > >> > > > >>>> in-place
> > > > > >> > > > >>>>>>>>>>>>>>>> either, as
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> it's a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> backwards incompatible change. My
> > > concern
> > > > > is
> > > > > >> > > > >> that,
> > > > > >> > > > >>>> if
> > > > > >> > > > >>>>>> we
> > > > > >> > > > >>>>>>>>>> leave
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> existing
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics in place, they
> will
> > be
> > > > > >> > > > >> confusing to
> > > > > >> > > > >>>>>> users.
> > > > > >> > > > >>>>>>>>>>>>>>>> Right
> > > > > >> > > > >>>>>>>>>>>>>>>>>> now,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics record explicit
> > > flushes to
> > > > > >> > > > >> disk,
> > > > > >> > > > >>>> but
> > > > > >> > > > >>>>>> under
> > > > > >> > > > >>>>>>>>>>>>>>>> KIP-892,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> even
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() will not explicitly flush
> > > data to
> > > > > >> > > > >> disk -
> > > > > >> > > > >>>>>> RocksDB
> > > > > >> > > > >>>>>>>>>> will
> > > > > >> > > > >>>>>>>>>>>>>>>>>> decide
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> on
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> when to flush memtables to disk
> > > itself.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> If we keep the existing "flush"
> > > metrics,
> > > > > >> we'd
> > > > > >> > > > >> have
> > > > > >> > > > >>>> two
> > > > > >> > > > >>>>>>>>>>>> options,
> > > > > >> > > > >>>>>>>>>>>>>>>>>> which
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> both
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> seem pretty bad to me:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          1. Have them record calls
> > to
> > > > > >> commit(),
> > > > > >> > > > >>>> which
> > > > > >> > > > >>>>>> would
> > > > > >> > > > >>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> misleading, as
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          data is no longer
> > explicitly
> > > > > >> "flushed"
> > > > > >> > > > >> to
> > > > > >> > > > >>>> disk
> > > > > >> > > > >>>>>> by
> > > > > >> > > > >>>>>>>> this
> > > > > >> > > > >>>>>>>>>>>>>>>> call.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          2. Have them record
> nothing
> > > at
> > > > > all,
> > > > > >> > > > >> which
> > > > > >> > > > >>>> is
> > > > > >> > > > >>>>>>>>>> equivalent
> > > > > >> > > > >>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> removing
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          metrics, except that
> users
> > > will
> > > > > see
> > > > > >> > the
> > > > > >> > > > >>>> metric
> > > > > >> > > > >>>>>>>> still
> > > > > >> > > > >>>>>>>>>>>>>>>> exists and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> so
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> assume
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          that the metric is
> correct,
> > > and
> > > > > >> that
> > > > > >> > > > >>>> there's a
> > > > > >> > > > >>>>>>>> problem
> > > > > >> > > > >>>>>>>>>>>>>>>> with
> > > > > >> > > > >>>>>>>>>>>>>>>>>> their
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> system
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          when there isn't.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that removing them is
> also a
> > > bad
> > > > > >> > > > >> solution,
> > > > > >> > > > >>>> and
> > > > > >> > > > >>>>>> I'd
> > > > > >> > > > >>>>>>>>>>>>>>>> like some
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> guidance on the best path forward
> > > here.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 5.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Position files are updated on
> every
> > > write
> > > > > >> to a
> > > > > >> > > > >>>>>> StateStore.
> > > > > >> > > > >>>>>>>>>>>>>>>> Since our
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> writes
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> are now buffered until commit(),
> we
> > > can't
> > > > > >> > > > >> update
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>>> Position
> > > > > >> > > > >>>>>>>>>>>>>>>> file
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> until
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() has been called,
> otherwise
> > it
> > > > > >> would be
> > > > > >> > > > >>>>>>>> inconsistent
> > > > > >> > > > >>>>>>>>>>>>>>>> with
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> data
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> in the event of a rollback.
> > > Consequently,
> > > > > we
> > > > > >> > > > >> need
> > > > > >> > > > >>>> to
> > > > > >> > > > >>>>>> manage
> > > > > >> > > > >>>>>>>>>>>>>>>> these
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> offsets
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> the same way we manage the
> > checkpoint
> > > > > >> offsets,
> > > > > >> > > > >> and
> > > > > >> > > > >>>>>> ensure
> > > > > >> > > > >>>>>>>>>>>>>>>> they're
> > > > > >> > > > >>>>>>>>>>>>>>>>>> only
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> written on commit().
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 6.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Agreed, although I'm not exactly
> > sure
> > > yet
> > > > > >> what
> > > > > >> > > > >>>> tests to
> > > > > >> > > > >>>>>>>>>> write.
> > > > > >> > > > >>>>>>>>>>>>>>>> How
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> explicit
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> do we need to be here in the KIP?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade
> is
> > > > > >> designed
> > > > > >> > > > >> to be
> > > > > >> > > > >>>>>>>> seamless,
> > > > > >> > > > >>>>>>>>>>>>>>>> and we
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> should
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> definitely add some tests around
> > that.
> > > > > >> > > > >> Downgrade,
> > > > > >> > > > >>>> it
> > > > > >> > > > >>>>>>>>>>>>>>>> transpires,
> > > > > >> > > > >>>>>>>>>>>>>>>>>> isn't
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> currently possible, as the extra
> > > column
> > > > > >> family
> > > > > >> > > > >> for
> > > > > >> > > > >>>>>> offset
> > > > > >> > > > >>>>>>>>>>>>>>>> storage is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> incompatible with the pre-KIP-892
> > > > > >> > > > >> implementation:
> > > > > >> > > > >>>> when
> > > > > >> > > > >>>>>> you
> > > > > >> > > > >>>>>>>>>>>>>>>> open a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> database, you must open all
> > available
> > > > > column
> > > > > >> > > > >>>> families
> > > > > >> > > > >>>>>> or
> > > > > >> > > > >>>>>>>>>>>>>>>> receive an
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> error.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> What currently happens on
> downgrade
> > is
> > > > > that
> > > > > >> it
> > > > > >> > > > >>>>>> attempts to
> > > > > >> > > > >>>>>>>>>>>>>>>> open the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> store,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> throws an error about the offsets
> > > column
> > > > > >> > > > >> family not
> > > > > >> > > > >>>>>> being
> > > > > >> > > > >>>>>>>>>>>>>>>> opened,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> which
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> triggers a wipe and rebuild of the
> > > Task.
> > > > > >> Given
> > > > > >> > > > >> that
> > > > > >> > > > >>>>>>>>>> downgrades
> > > > > >> > > > >>>>>>>>>>>>>>>>>> should
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> uncommon, I think this is
> acceptable
> > > > > >> > > > >> behaviour, as
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> end-state is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> consistent, even if it results in
> an
> > > > > >> > > > >> undesirable
> > > > > >> > > > >>>> state
> > > > > >> > > > >>>>>>>>>>>> restore.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Should I document the
> > > upgrade/downgrade
> > > > > >> > > > >> behaviour
> > > > > >> > > > >>>>>>>> explicitly
> > > > > >> > > > >>>>>>>>>>>>>>>> in the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> KIP?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> --
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31,
> Bruno
> > > > > Cadonna
> > > > > >> <
> > > > > >> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Nick!
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates!
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Why does StateStore#flush()
> default
> > > to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > StateStore#commit(Collections.emptyMap())?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Since calls to flush() will not
> > exist
> > > > > >> anymore
> > > > > >> > > > >>>> after
> > > > > >> > > > >>>>>> this
> > > > > >> > > > >>>>>>>> KIP
> > > > > >> > > > >>>>>>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> released, I would rather throw an
> > > > > >> unsupported
> > > > > >> > > > >>>>>> operation
> > > > > >> > > > >>>>>>>>>>>>>>>> exception
> > > > > >> > > > >>>>>>>>>>>>>>>>>> by
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> default.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> When would a state store return
> -1
> > > from
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > StateStore#approximateNumUncommittedBytes()
> > > > > >> > > > >> while
> > > > > >> > > > >>>>>> being
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> transactional?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Wouldn't
> > > > > >> > > > >>>> StateStore#approximateNumUncommittedBytes()
> > > > > >> > > > >>>>>> also
> > > > > >> > > > >>>>>>>>>>>>>>>> return 0
> > > > > >> > > > >>>>>>>>>>>>>>>>>> if
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the state store is transactional
> > but
> > > > > >> nothing
> > > > > >> > > > >> has
> > > > > >> > > > >>>> been
> > > > > >> > > > >>>>>>>>>> written
> > > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> state store yet?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Sorry for bringing this up again.
> > > Does
> > > > > this
> > > > > >> > > > >> KIP
> > > > > >> > > > >>>> really
> > > > > >> > > > >>>>>>>> need
> > > > > >> > > > >>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> introduce
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> StateStoreContext#isolationLevel()?
> > > > > >> > > > >>>> StateStoreContext
> > > > > >> > > > >>>>>> has
> > > > > >> > > > >>>>>>>>>>>>>>>> already
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> appConfigs() which basically
> > exposes
> > > the
> > > > > >> same
> > > > > >> > > > >>>>>> information,
> > > > > >> > > > >>>>>>>>>>>>>>>> i.e., if
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> EOS
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> is enabled or not.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> In one of your previous e-mails
> you
> > > > > wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> "My idea was to try to keep the
> > > > > StateStore
> > > > > >> > > > >>>> interface
> > > > > >> > > > >>>>>> as
> > > > > >> > > > >>>>>>>>>>>>>>>> loosely
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> coupled
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as
> > possible,
> > > to
> > > > > >> give
> > > > > >> > > > >>>>>> implementers
> > > > > >> > > > >>>>>>>>>>>> more
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> freedom,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and reduce the amount of internal
> > > > > knowledge
> > > > > >> > > > >>>> required."
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> While I understand the intent, I
> > > doubt
> > > > > >> that it
> > > > > >> > > > >>>>>> decreases
> > > > > >> > > > >>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> coupling of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> a StateStore interface and the
> > > Streams
> > > > > >> engine.
> > > > > >> > > > >>>>>>>>>> READ_COMMITTED
> > > > > >> > > > >>>>>>>>>>>>>>>> only
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> applies to IQ but not to reads by
> > > > > >> processors.
> > > > > >> > > > >>>> Thus,
> > > > > >> > > > >>>>>>>>>>>>>>>> implementers
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> need to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand how Streams accesses
> the
> > > state
> > > > > >> > > > >> stores.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> I would like to hear what others
> > > think
> > > > > >> about
> > > > > >> > > > >> this.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 4.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Great exposing new metrics for
> > > > > >> transactional
> > > > > >> > > > >> state
> > > > > >> > > > >>>>>> stores!
> > > > > >> > > > >>>>>>>>>>>>>>>>>> However, I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> would prefer to add new metrics
> and
> > > > > >> deprecate
> > > > > >> > > > >> (in
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>>>> docs)
> > > > > >> > > > >>>>>>>>>>>>>>>> the 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 also need to
> > > adapt/add
> > > > > >> > > > >> integration
> > > > > >> > > > >>>>>> tests
> > > > > >> > > > >>>>>>>>>>>>>>>> besides
> > > > > >> > > > >>>>>>>>>>>>>>>>>> unit
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> tests. Additionally, we probably
> > need
> > > > > >> > > > >> integration
> > > > > >> > > > >>>> or
> > > > > >> > > > >>>>>>>> system
> > > > > >> > > > >>>>>>>>>>>>>>>> tests
> > > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> verify that upgrades and
> downgrades
> > > > > between
> > > > > >> > > > >>>>>> transactional
> > > > > >> > > > >>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> non-transactional state stores
> work
> > > as
> > > > > >> > > > >> expected.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford
> > > wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> One more thing: I noted John's
> > > > > suggestion
> > > > > >> in
> > > > > >> > > > >> the
> > > > > >> > > > >>>> KIP,
> > > > > >> > > > >>>>>>>> under
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> "Rejected
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> Alternatives". I still think
> it's
> > an
> > > > > idea
> > > > > >> > > > >> worth
> > > > > >> > > > >>>>>> pursuing,
> > > > > >> > > > >>>>>>>>>>>>>>>> but I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> believe
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> that it's out of the scope of
> this
> > > KIP,
> > > > > >> > > > >> because
> > > > > >> > > > >>>> it
> > > > > >> > > > >>>>>>>> solves a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> different
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> set
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> of problems to this KIP, and the
> > > scope
> > > > > of
> > > > > >> > > > >> this
> > > > > >> > > > >>>> one
> > > > > >> > > > >>>>>> has
> > > > > >> > > > >>>>>>>>>>>>>>>> already
> > > > > >> > > > >>>>>>>>>>>>>>>>>> grown
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> quite
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> large!
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33,
> Nick
> > > > > >> Telford <
> > > > > >> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP (
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>
> > > > > >> > > > >>
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> )
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with the latest changes; mostly
> > > > > bringing
> > > > > >> > > > >> back
> > > > > >> > > > >>>>>> "Atomic
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> Checkpointing"
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> (for
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> what feels like the 10th
> time!).
> > I
> > > > > think
> > > > > >> > > > >> the one
> > > > > >> > > > >>>>>> thing
> > > > > >> > > > >>>>>>>>>>>>>>>> missing is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> some
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> changes to metrics (notably the
> > > store
> > > > > >> > > > >> "flush"
> > > > > >> > > > >>>>>> metrics
> > > > > >> > > > >>>>>>>> will
> > > > > >> > > > >>>>>>>>>>>>>>>> need
> > > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> renamed to "commit").
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> The reason I brought back
> Atomic
> > > > > >> > > > >> Checkpointing
> > > > > >> > > > >>>> was
> > > > > >> > > > >>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> decouple
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flush
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from store commit. This is
> > > important,
> > > > > >> > > > >> because
> > > > > >> > > > >>>> with
> > > > > >> > > > >>>>>>>>>>>>>>>> Transactional
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> StateStores, we now need to
> call
> > > > > "flush"
> > > > > >> on
> > > > > >> > > > >>>> *every*
> > > > > >> > > > >>>>>> Task
> > > > > >> > > > >>>>>>>>>>>>>>>> commit,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> just when the StateStore is
> > > closing,
> > > > > >> > > > >> otherwise
> > > > > >> > > > >>>> our
> > > > > >> > > > >>>>>>>>>>>>>>>> transaction
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> buffer
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> never be written and persisted,
> > > instead
> > > > > >> > > > >> growing
> > > > > >> > > > >>>>>>>> unbounded!
> > > > > >> > > > >>>>>>>>>>>> I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> experimented
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with some simple solutions,
> like
> > > > > forcing
> > > > > >> a
> > > > > >> > > > >> store
> > > > > >> > > > >>>>>> flush
> > > > > >> > > > >>>>>>>>>>>>>>>> whenever
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer was likely
> to
> > > exceed
> > > > > >> its
> > > > > >> > > > >>>>>> configured
> > > > > >> > > > >>>>>>>>>>>>>>>> size, but
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> this
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> brittle: it prevented the
> > > transaction
> > > > > >> buffer
> > > > > >> > > > >>>> from
> > > > > >> > > > >>>>>> being
> > > > > >> > > > >>>>>>>>>>>>>>>>>> configured
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> unbounded, and it still would
> > have
> > > > > >> required
> > > > > >> > > > >>>> explicit
> > > > > >> > > > >>>>>>>>>>>>>>>> flushes of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> RocksDB,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yielding sub-optimal
> performance
> > > and
> > > > > >> memory
> > > > > >> > > > >>>>>> utilization.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing
> to
> > > be the
> > > > > >> > > > >> "right"
> > > > > >> > > > >>>> way
> > > > > >> > > > >>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> resolve
> > > > > >> > > > >>>>>>>>>>>>>>>>>> this
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> problem. By ensuring that the
> > > changelog
> > > > > >> > > > >> offsets
> > > > > >> > > > >>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>> correspond
> > > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> most
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> recently written records are
> > always
> > > > > >> > > > >> atomically
> > > > > >> > > > >>>>>> written
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (by writing them to the same
> > > > > transaction
> > > > > >> > > > >>>> buffer),
> > > > > >> > > > >>>>>> we can
> > > > > >> > > > >>>>>>>>>>>>>>>> avoid
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> forcibly
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables
> to
> > > disk,
> > > > > >> > > > >> letting
> > > > > >> > > > >>>>>> RocksDB
> > > > > >> > > > >>>>>>>>>>>>>>>> flush
> > > > > >> > > > >>>>>>>>>>>>>>>>>> them
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> only
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when necessary, without losing
> > any
> > > of
> > > > > our
> > > > > >> > > > >>>>>> consistency
> > > > > >> > > > >>>>>>>>>>>>>>>> guarantees.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> See
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> updated KIP for more info.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I have fully implemented these
> > > changes,
> > > > > >> > > > >>>> although I'm
> > > > > >> > > > >>>>>>>> still
> > > > > >> > > > >>>>>>>>>>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> entirely
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> happy with the implementation
> for
> > > > > >> segmented
> > > > > >> > > > >>>>>> StateStores,
> > > > > >> > > > >>>>>>>>>> so
> > > > > >> > > > >>>>>>>>>>>>>>>> I
> > > > > >> > > > >>>>>>>>>>>>>>>>>> plan
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> refactor that. Despite that,
> all
> > > tests
> > > > > >> > > > >> pass. If
> > > > > >> > > > >>>>>> you'd
> > > > > >> > > > >>>>>>>> like
> > > > > >> > > > >>>>>>>>>>>>>>>> to try
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> out
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> or
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> review this highly experimental
> > and
> > > > > >> > > > >> incomplete
> > > > > >> > > > >>>>>> branch,
> > > > > >> > > > >>>>>>>>>> it's
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> available
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> here:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>
> > > https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > > > > >> > > > >>>>>>>> .
> > > > > >> > > > >>>>>>>>>>>>>>>> Note:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> it's
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> built
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I
> > had a
> > > > > >> stable
> > > > > >> > > > >> base
> > > > > >> > > > >>>> to
> > > > > >> > > > >>>>>> build
> > > > > >> > > > >>>>>>>>>>>>>>>> and test
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> it
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> on,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and to enable easy
> > apples-to-apples
> > > > > >> > > > >> comparisons
> > > > > >> > > > >>>> in a
> > > > > >> > > > >>>>>>>> live
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> environment. I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> plan to rebase it against trunk
> > > once
> > > > > it's
> > > > > >> > > > >> nearer
> > > > > >> > > > >>>>>>>>>> completion
> > > > > >> > > > >>>>>>>>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>> has
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> been
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> proven on our main application.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I would really appreciate help
> in
> > > > > >> reviewing
> > > > > >> > > > >> and
> > > > > >> > > > >>>>>> testing:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session
> > and
> > > > > >> Window)
> > > > > >> > > > >>>> stores
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Global stores
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> As I do not currently use
> either
> > of
> > > > > >> these,
> > > > > >> > > > >> so my
> > > > > >> > > > >>>>>> primary
> > > > > >> > > > >>>>>>>>>>>>>>>> test
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> environment
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> doesn't test these areas.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I'm going on Parental Leave
> > > starting
> > > > > next
> > > > > >> > > > >> week
> > > > > >> > > > >>>> for
> > > > > >> > > > >>>>>> a few
> > > > > >> > > > >>>>>>>>>>>>>>>> weeks,
> > > > > >> > > > >>>>>>>>>>>>>>>>>> so
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> will
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> not have time to move this
> > forward
> > > > > until
> > > > > >> > > > >> late
> > > > > >> > > > >>>>>> August.
> > > > > >> > > > >>>>>>>> That
> > > > > >> > > > >>>>>>>>>>>>>>>> said,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> your
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> feedback is welcome and
> > > appreciated, I
> > > > > >> just
> > > > > >> > > > >>>> won't be
> > > > > >> > > > >>>>>>>> able
> > > > > >> > > > >>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> respond
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> as
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> quickly as usual.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23,
> Nick
> > > > > >> Telford <
> > > > > >> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, that's correct, although
> > the
> > > > > impact
> > > > > >> > > > >> on IQ
> > > > > >> > > > >>>> is
> > > > > >> > > > >>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>> something
> > > > > >> > > > >>>>>>>>>>>>>>>>>> I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> had
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> considered.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating
> > the
> > > > > state
> > > > > >> > > > >> store
> > > > > >> > > > >>>>>> from the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval
> > and
> > > > > >> writing
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>>>> checkpoint
> > > > > >> > > > >>>>>>>>>>>>>>>> (thus,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every
> configured
> > > amount
> > > > > >> of
> > > > > >> > > > >> data
> > > > > >> > > > >>>>>> and/or
> > > > > >> > > > >>>>>>>>>>>>>>>> number of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow.
> Are
> > > you
> > > > > >> > > > >> suggesting
> > > > > >> > > > >>>>>> that we
> > > > > >> > > > >>>>>>>>>>>>>>>> add an
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> additional
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> config for the max number of
> > > commit
> > > > > >> > > > >> intervals
> > > > > >> > > > >>>>>> between
> > > > > >> > > > >>>>>>>>>>>>>>>>>> checkpoints?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> That
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> way, we would checkpoint
> > *either*
> > > when
> > > > > >> the
> > > > > >> > > > >>>>>> transaction
> > > > > >> > > > >>>>>>>>>>>>>>>> buffers
> > > > > >> > > > >>>>>>>>>>>>>>>>>> are
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> nearly
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain
> > > number
> > > > > of
> > > > > >> > > > >> commit
> > > > > >> > > > >>>>>>>> intervals
> > > > > >> > > > >>>>>>>>>>>>>>>> have
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> elapsed,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> whichever comes first?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> That certainly seems
> reasonable,
> > > > > >> although
> > > > > >> > > > >> this
> > > > > >> > > > >>>>>>>> re-ignites
> > > > > >> > > > >>>>>>>>>>>>>>>> an
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> earlier
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> debate about whether a config
> > > should
> > > > > be
> > > > > >> > > > >>>> measured in
> > > > > >> > > > >>>>>>>>>>>>>>>> "number of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> commit
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> intervals", instead of just an
> > > > > absolute
> > > > > >> > > > >> time.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> FWIW, I realised that this
> issue
> > > is
> > > > > the
> > > > > >> > > > >> reason
> > > > > >> > > > >>>> I
> > > > > >> > > > >>>>>> was
> > > > > >> > > > >>>>>>>>>>>>>>>> pursuing
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Atomic
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples
> > > memtable
> > > > > >> > > > >> flush
> > > > > >> > > > >>>> from
> > > > > >> > > > >>>>>>>>>>>>>>>> checkpointing,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> which
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> enables us to just checkpoint
> on
> > > every
> > > > > >> > > > >> commit
> > > > > >> > > > >>>>>> without
> > > > > >> > > > >>>>>>>> any
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> performance
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing
> is
> > > > > >> definitely
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>> "best"
> > > > > >> > > > >>>>>>>>>>>>>>>> solution,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> but
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm not
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sure if this is enough to
> bring
> > it
> > > > > back
> > > > > >> > > > >> into
> > > > > >> > > > >>>> this
> > > > > >> > > > >>>>>> KIP.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm currently working on
> moving
> > > all
> > > > > the
> > > > > >> > > > >>>>>> transactional
> > > > > >> > > > >>>>>>>>>>>> logic
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> directly
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> into
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which
> does
> > > away
> > > > > >> with
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore#newTransaction
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> method, and reduces the number
> > of
> > > new
> > > > > >> > > > >> classes
> > > > > >> > > > >>>>>>>> introduced,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> significantly
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reducing the complexity. If it
> > > works,
> > > > > >> and
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>>>> complexity
> > > > > >> > > > >>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> drastically
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reduced, I may try bringing
> back
> > > > > Atomic
> > > > > >> > > > >>>> Checkpoints
> > > > > >> > > > >>>>>>>> into
> > > > > >> > > > >>>>>>>>>>>>>>>> this
> > > > > >> > > > >>>>>>>>>>>>>>>>>> KIP.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27,
> > Bruno
> > > > > >> Cadonna
> > > > > >> > > > >> <
> > > > > >> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very
> > > > > >> interesting!
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> As far as I understand, you
> > want
> > > to
> > > > > >> > > > >> atomically
> > > > > >> > > > >>>>>> update
> > > > > >> > > > >>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> state
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> from the transaction buffer,
> > > flush
> > > > > the
> > > > > >> > > > >>>> memtable
> > > > > >> > > > >>>>>> of a
> > > > > >> > > > >>>>>>>>>>>> state
> > > > > >> > > > >>>>>>>>>>>>>>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write the checkpoint not
> after
> > > the
> > > > > >> commit
> > > > > >> > > > >> time
> > > > > >> > > > >>>>>> elapsed
> > > > > >> > > > >>>>>>>>>>>> but
> > > > > >> > > > >>>>>>>>>>>>>>>>>> after
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reached a
> > size
> > > > > that
> > > > > >> > > > >> would
> > > > > >> > > > >>>> lead
> > > > > >> > > > >>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> exceeding
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > statestore.transaction.buffer.max.bytes
> > > > > >> > > > >>>> before the
> > > > > >> > > > >>>>>>>> next
> > > > > >> > > > >>>>>>>>>>>>>>>> commit
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> interval
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> ends.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> That means, the Kafka
> > transaction
> > > > > would
> > > > > >> > > > >> commit
> > > > > >> > > > >>>>>> every
> > > > > >> > > > >>>>>>>>>>>>>>>> commit
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> interval
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store will only be
> > > > > atomically
> > > > > >> > > > >>>> updated
> > > > > >> > > > >>>>>>>> roughly
> > > > > >> > > > >>>>>>>>>>>>>>>> every
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> statestore.transaction.buffer.max.bytes of
> > > > > >> > > > >>>> data.
> > > > > >> > > > >>>>>> Also
> > > > > >> > > > >>>>>>>> IQ
> > > > > >> > > > >>>>>>>>>>>>>>>> would
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> then
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> only
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see new data roughly every
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > statestore.transaction.buffer.max.bytes.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> After a failure the state
> store
> > > needs
> > > > > >> to
> > > > > >> > > > >>>> restore
> > > > > >> > > > >>>>>> up to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> statestore.transaction.buffer.max.bytes.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Is this correct?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically
> updating
> > > the
> > > > > >> state
> > > > > >> > > > >> store
> > > > > >> > > > >>>>>> from
> > > > > >> > > > >>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval
> > and
> > > > > >> writing
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>>>> checkpoint
> > > > > >> > > > >>>>>>>>>>>>>>>> (thus,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every
> configured
> > > amount
> > > > > >> of
> > > > > >> > > > >> data
> > > > > >> > > > >>>>>> and/or
> > > > > >> > > > >>>>>>>>>>>>>>>> number of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we
> > > would
> > > > > have
> > > > > >> > > > >> the
> > > > > >> > > > >>>> same
> > > > > >> > > > >>>>>> delay
> > > > > >> > > > >>>>>>>>>>>> for
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> records
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> appearing in output topics
> and
> > IQ
> > > > > >> because
> > > > > >> > > > >> both
> > > > > >> > > > >>>>>> would
> > > > > >> > > > >>>>>>>>>>>>>>>> appear
> > > > > >> > > > >>>>>>>>>>>>>>>>>> when
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka transaction is
> committed.
> > > > > >> However,
> > > > > >> > > > >>>> after a
> > > > > >> > > > >>>>>>>> failure
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> statestore.transaction.buffer.max.bytes
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it might restore data that is
> > > already
> > > > > >> in
> > > > > >> > > > >> the
> > > > > >> > > > >>>> state
> > > > > >> > > > >>>>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>>>> because
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the
> last
> > > > > stable
> > > > > >> > > > >> offset
> > > > > >> > > > >>>>>> (i.e.
> > > > > >> > > > >>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> last
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> committed
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset) of the changelog
> > topics.
> > > > > >> Restoring
> > > > > >> > > > >>>> data
> > > > > >> > > > >>>>>> that
> > > > > >> > > > >>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>> already
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> in
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> state store is idempotent, so
> > eos
> > > > > >> should
> > > > > >> > > > >> not
> > > > > >> > > > >>>>>> violated.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> This solution needs at least
> > one
> > > new
> > > > > >> > > > >> config to
> > > > > >> > > > >>>>>> specify
> > > > > >> > > > >>>>>>>>>>>>>>>> when a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> should be written.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> A small correction to your
> > > previous
> > > > > >> e-mail
> > > > > >> > > > >>>> that
> > > > > >> > > > >>>>>> does
> > > > > >> > > > >>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>> change
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> anything
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> you said: Under alos the
> > default
> > > > > commit
> > > > > >> > > > >>>> interval
> > > > > >> > > > >>>>>> is 30
> > > > > >> > > > >>>>>>>>>>>>>>>> seconds,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> five
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> seconds.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick
> Telford
> > > > > wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've begun performance
> testing
> > > my
> > > > > >> branch
> > > > > >> > > > >> on
> > > > > >> > > > >>>> our
> > > > > >> > > > >>>>>>>> staging
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> environment,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting it through its paces
> > in
> > > our
> > > > > >> > > > >>>> non-trivial
> > > > > >> > > > >>>>>>>>>>>>>>>> application.
> > > > > >> > > > >>>>>>>>>>>>>>>>>> I'm
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing the same increased
> > > flush
> > > > > >> rate
> > > > > >> > > > >> that
> > > > > >> > > > >>>> we
> > > > > >> > > > >>>>>> saw
> > > > > >> > > > >>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> last
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> time
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> attempted to use a version
> of
> > > this
> > > > > >> KIP,
> > > > > >> > > > >> but
> > > > > >> > > > >>>> this
> > > > > >> > > > >>>>>>>> time,
> > > > > >> > > > >>>>>>>>>> I
> > > > > >> > > > >>>>>>>>>>>>>>>>>> think I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> know
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> why.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892,
> > > StreamTask#postCommit,
> > > > > >> > > > >> which is
> > > > > >> > > > >>>>>> called
> > > > > >> > > > >>>>>>>> at
> > > > > >> > > > >>>>>>>>>>>>>>>> the end
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Task
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit process, has the
> > > following
> > > > > >> > > > >> behaviour:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under ALOS:
> > > checkpoint
> > > > > >> the
> > > > > >> > > > >> state
> > > > > >> > > > >>>>>> stores.
> > > > > >> > > > >>>>>>>>>> This
> > > > > >> > > > >>>>>>>>>>>>>>>>>> includes
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            flushing
> memtables
> > in
> > > > > >> RocksDB.
> > > > > >> > > > >>>> This is
> > > > > >> > > > >>>>>>>>>>>> acceptable
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> because the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> default
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> commit.interval.ms
> > > is 5
> > > > > >> > > > >> seconds,
> > > > > >> > > > >>>> so
> > > > > >> > > > >>>>>>>> forcibly
> > > > > >> > > > >>>>>>>>>>>>>>>> flushing
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> every 5
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            seconds is
> > > acceptable for
> > > > > >> most
> > > > > >> > > > >>>>>>>> applications.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under EOS:
> > > > > checkpointing
> > > > > >> is
> > > > > >> > > > >> not
> > > > > >> > > > >>>> done,
> > > > > >> > > > >>>>>>>>>> *unless*
> > > > > >> > > > >>>>>>>>>>>>>>>> it's
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> being
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> forced, due
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to e.g. the Task
> > > closing
> > > > > or
> > > > > >> > > > >> being
> > > > > >> > > > >>>>>> revoked.
> > > > > >> > > > >>>>>>>>>> This
> > > > > >> > > > >>>>>>>>>>>>>>>> means
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> under
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> normal
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing
> > > conditions,
> > > > > the
> > > > > >> > > > >> state
> > > > > >> > > > >>>> stores
> > > > > >> > > > >>>>>>>> will
> > > > > >> > > > >>>>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpointed,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            not have
> memtables
> > > > > flushed
> > > > > >> at
> > > > > >> > > > >> all ,
> > > > > >> > > > >>>>>> unless
> > > > > >> > > > >>>>>>>>>>>> RocksDB
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> decides to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush them on
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            its own.
> > > Checkpointing
> > > > > >> stores
> > > > > >> > > > >> and
> > > > > >> > > > >>>>>>>>>> force-flushing
> > > > > >> > > > >>>>>>>>>>>>>>>> their
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            done when a Task
> is
> > > being
> > > > > >> > > > >> closed.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to
> > > > > checkpoint
> > > > > >> > > > >>>> stores on
> > > > > >> > > > >>>>>> at
> > > > > >> > > > >>>>>>>>>>>> least
> > > > > >> > > > >>>>>>>>>>>>>>>>>> *some*
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> normal
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task commits, in order to
> > write
> > > the
> > > > > >> > > > >> RocksDB
> > > > > >> > > > >>>>>>>> transaction
> > > > > >> > > > >>>>>>>>>>>>>>>>>> buffers
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure
> > the
> > > > > >> offsets
> > > > > >> > > > >> are
> > > > > >> > > > >>>>>> synced to
> > > > > >> > > > >>>>>>>>>>>>>>>> disk to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> prevent
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> restores from getting out of
> > > hand.
> > > > > >> > > > >>>> Consequently,
> > > > > >> > > > >>>>>> my
> > > > > >> > > > >>>>>>>>>>>>>>>> current
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on
> > *every*
> > > > > Task
> > > > > >> > > > >> commit,
> > > > > >> > > > >>>>>> which
> > > > > >> > > > >>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>> far too
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> frequent.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This causes checkpoints
> every
> > > 10,000
> > > > > >> > > > >> records,
> > > > > >> > > > >>>>>> which
> > > > > >> > > > >>>>>>>> is
> > > > > >> > > > >>>>>>>>>> a
> > > > > >> > > > >>>>>>>>>>>>>>>>>> change
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> in
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, potentially
> causing
> > > > > >> > > > >> performance
> > > > > >> > > > >>>>>> problems
> > > > > >> > > > >>>>>>>> for
> > > > > >> > > > >>>>>>>>>>>>>>>> some
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> applications.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm looking into possible
> > > solutions,
> > > > > >> and
> > > > > >> > > > >> I'm
> > > > > >> > > > >>>>>>>> currently
> > > > > >> > > > >>>>>>>>>>>>>>>> leaning
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> towards
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> using the
> > > > > >> > > > >>>> statestore.transaction.buffer.max.bytes
> > > > > >> > > > >>>>>>>>>>>>>>>>>> configuration
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are
> > > likely
> > > > > to
> > > > > >> > > > >>>> exceed it.
> > > > > >> > > > >>>>>>>> This
> > > > > >> > > > >>>>>>>>>>>>>>>> would
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> complement the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
> > > > > >> > > > >> functionality
> > > > > >> > > > >>>> that
> > > > > >> > > > >>>>>> this
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> configuration
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provides, in the following
> > way:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Currently, we
> use
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> statestore.transaction.buffer.max.bytes
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> force an
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            early Task commit
> > if
> > > > > >> > processing
> > > > > >> > > > >>>> more
> > > > > >> > > > >>>>>>>> records
> > > > > >> > > > >>>>>>>>>>>> would
> > > > > >> > > > >>>>>>>>>>>>>>>>>> cause
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> our
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> state
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            transactions to
> > > exceed
> > > > > the
> > > > > >> > > > >> memory
> > > > > >> > > > >>>>>> assigned
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> them.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - New
> > functionality:
> > > > > when a
> > > > > >> > > > >> Task
> > > > > >> > > > >>>> *does*
> > > > > >> > > > >>>>>>>>>> commit,
> > > > > >> > > > >>>>>>>>>>>>>>>> we will
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the stores (and
> > hence
> > > > > flush
> > > > > >> > the
> > > > > >> > > > >>>>>> transaction
> > > > > >> > > > >>>>>>>>>>>>>>>> buffers)
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> unless
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> expect to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            cross the
> > > > > >> > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> threshold
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the next
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm also open to
> suggestions.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at
> 14:06,
> > > Nick
> > > > > >> > > > >> Telford <
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> By "less predictable for
> > > users", I
> > > > > >> > > > >> meant in
> > > > > >> > > > >>>>>> terms of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> understanding
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance profile under
> > > various
> > > > > >> > > > >>>>>> circumstances. The
> > > > > >> > > > >>>>>>>>>>>>>>>> more
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> complex
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, the more
> difficult
> > it
> > > > > would
> > > > > >> > > > >> be for
> > > > > >> > > > >>>>>> users
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> understand
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance they see. For
> > > example,
> > > > > >> > > > >> spilling
> > > > > >> > > > >>>>>> records
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> disk
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches
> a
> > > > > >> threshold
> > > > > >> > > > >>>> would, I
> > > > > >> > > > >>>>>>>>>> expect,
> > > > > >> > > > >>>>>>>>>>>>>>>>>> reduce
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> write
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction
> in
> > > write
> > > > > >> > > > >>>> throughput
> > > > > >> > > > >>>>>> could
> > > > > >> > > > >>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> unexpected,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> > > > > >> > > > >>>> diagnose/understand for
> > > > > >> > > > >>>>>>>>>> users.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the
> > > "early
> > > > > >> > > > >> commit"
> > > > > >> > > > >>>>>> concept is
> > > > > >> > > > >>>>>>>>>>>>>>>>>> relatively
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy
> to
> > > > > >> document,
> > > > > >> > > > >> and
> > > > > >> > > > >>>>>>>>>>>> conceptually
> > > > > >> > > > >>>>>>>>>>>>>>>>>> fairly
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> obvious to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users. We could probably
> add
> > a
> > > > > >> metric to
> > > > > >> > > > >>>> make it
> > > > > >> > > > >>>>>>>>>> easier
> > > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it happens though.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is
> > > *essentially*
> > > > > >> an
> > > > > >> > > > >>>> indirect
> > > > > >> > > > >>>>>> way
> > > > > >> > > > >>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>>> telling
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether they should be
> > > > > transactional.
> > > > > >> > > > >>>>>> READ_COMMITTED
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> essentially
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> requires
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, because it
> > > dictates
> > > > > >> that
> > > > > >> > > > >> two
> > > > > >> > > > >>>>>> threads
> > > > > >> > > > >>>>>>>>>>>>>>>> calling
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should
> not
> > > see
> > > > > >> writes
> > > > > >> > > > >>>> from
> > > > > >> > > > >>>>>> the
> > > > > >> > > > >>>>>>>>>> other
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> transaction
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> until
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they have been committed.
> > With
> > > > > >> > > > >>>>>> READ_UNCOMMITTED, all
> > > > > >> > > > >>>>>>>>>>>>>>>> bets are
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> off,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to
> > > observe
> > > > > >> > > > >> written
> > > > > >> > > > >>>>>> records
> > > > > >> > > > >>>>>>>> at
> > > > > >> > > > >>>>>>>>>>>>>>>> any
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> time,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> which is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially "no
> > transactions".
> > > That
> > > > > >> > > > >> said,
> > > > > >> > > > >>>>>>>> StateStores
> > > > > >> > > > >>>>>>>>>>>>>>>> are
> > > > > >> > > > >>>>>>>>>>>>>>>>>> free
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implement
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these guarantees however
> they
> > > can,
> > > > > >> > > > >> which is
> > > > > >> > > > >>>> a
> > > > > >> > > > >>>>>> bit
> > > > > >> > > > >>>>>>>> more
> > > > > >> > > > >>>>>>>>>>>>>>>>>> relaxed
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> than
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dictating "you must use
> > > > > >> transactions".
> > > > > >> > > > >> For
> > > > > >> > > > >>>>>> example,
> > > > > >> > > > >>>>>>>>>>>> with
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement these as
> > > READ_COMMITTED
> > > > > ==
> > > > > >> > > > >>>> WBWI-based
> > > > > >> > > > >>>>>>>>>>>>>>>>>> "transactions",
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct
> > > writes
> > > > > to
> > > > > >> the
> > > > > >> > > > >>>>>> database.
> > > > > >> > > > >>>>>>>> But
> > > > > >> > > > >>>>>>>>>>>>>>>> with
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> other
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> storage
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> engines, it might be
> > > preferable to
> > > > > >> > > > >> *always*
> > > > > >> > > > >>>> use
> > > > > >> > > > >>>>>>>>>>>>>>>> transactions,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> even
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may
> be
> > > > > storage
> > > > > >> > > > >> engines
> > > > > >> > > > >>>>>> that
> > > > > >> > > > >>>>>>>>>> don't
> > > > > >> > > > >>>>>>>>>>>>>>>>>> provide
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, but the
> > isolation
> > > > > >> > > > >> guarantees
> > > > > >> > > > >>>> can
> > > > > >> > > > >>>>>> be
> > > > > >> > > > >>>>>>>> met
> > > > > >> > > > >>>>>>>>>>>>>>>> using a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> different
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> technique.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep
> > the
> > > > > >> > > > >> StateStore
> > > > > >> > > > >>>>>> interface
> > > > > >> > > > >>>>>>>> as
> > > > > >> > > > >>>>>>>>>>>>>>>>>> loosely
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> coupled
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as
> > > > > possible,
> > > > > >> to
> > > > > >> > > > >> give
> > > > > >> > > > >>>>>>>>>>>>>>>> implementers
> > > > > >> > > > >>>>>>>>>>>>>>>>>> more
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> freedom, and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reduce the amount of
> internal
> > > > > >> knowledge
> > > > > >> > > > >>>>>> required.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That said, I understand
> that
> > > > > >> > > > >>>> "IsolationLevel"
> > > > > >> > > > >>>>>> might
> > > > > >> > > > >>>>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> right
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can
> > always
> > > make
> > > > > >> it
> > > > > >> > > > >> much
> > > > > >> > > > >>>> more
> > > > > >> > > > >>>>>>>>>>>>>>>> explicit if
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> required, e.g.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7-8.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I can make these changes
> > either
> > > > > later
> > > > > >> > > > >> today
> > > > > >> > > > >>>> or
> > > > > >> > > > >>>>>>>>>>>> tomorrow.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Small update:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on
> > > trunk and
> > > > > >> > > > >> fixed a
> > > > > >> > > > >>>>>> bunch of
> > > > > >> > > > >>>>>>>>>>>>>>>> issues
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all
> > the
> > > > > tests
> > > > > >> > > > >> pass,
> > > > > >> > > > >>>>>> which is
> > > > > >> > > > >>>>>>>>>>>>>>>>>> promising,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> but
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> it
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to undergo some
> > > performance
> > > > > >> > > > >> testing. I
> > > > > >> > > > >>>>>> haven't
> > > > > >> > > > >>>>>>>>>>>>>>>> (yet)
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> worked
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> on
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> removing the
> > `newTransaction()`
> > > > > >> stuff,
> > > > > >> > > > >> but I
> > > > > >> > > > >>>>>> would
> > > > > >> > > > >>>>>>>>>>>>>>>> expect
> > > > > >> > > > >>>>>>>>>>>>>>>>>> that,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should
> make
> > > no
> > > > > >> > > > >>>> difference. The
> > > > > >> > > > >>>>>>>>>> branch
> > > > > >> > > > >>>>>>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> available
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> at
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>
> https://github.com/nicktelford/kafka/tree/KIP-892-c
> > > > > >> > > > >>>>>>>>>> if
> > > > > >> > > > >>>>>>>>>>>>>>>>>> anyone
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interested in taking an
> early
> > > look.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at
> 11:59,
> > > Bruno
> > > > > >> > > > >> Cadonna
> > > > > >> > > > >>>> <
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> cadonna@apache.org>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you.
> That
> > > was
> > > > > >> > > > >> actually
> > > > > >> > > > >>>> also
> > > > > >> > > > >>>>>> my
> > > > > >> > > > >>>>>>>>>>>>>>>> point. I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understood
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that John was proposing
> the
> > > > > >> ingestion
> > > > > >> > > > >> path
> > > > > >> > > > >>>> as
> > > > > >> > > > >>>>>> a way
> > > > > >> > > > >>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> avoid
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> early
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I
> > > > > misinterpreted
> > > > > >> the
> > > > > >> > > > >>>> intent.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree with John here,
> that
> > > > > >> actually
> > > > > >> > > > >> it is
> > > > > >> > > > >>>>>> public
> > > > > >> > > > >>>>>>>>>>>>>>>> API. My
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> question
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern
> > affects
> > > > > >> normal
> > > > > >> > > > >>>>>> processing.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that
> checking
> > > for
> > > > > the
> > > > > >> > > > >> size
> > > > > >> > > > >>>> of the
> > > > > >> > > > >>>>>>>>>>>>>>>> transaction
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> buffer
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early
> > > commit
> > > > > >> > > > >> affects
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>> whole
> > > > > >> > > > >>>>>>>>>>>>>>>>>> processing
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams. The
> > transactionality
> > > of a
> > > > > >> > > > >> state
> > > > > >> > > > >>>> store
> > > > > >> > > > >>>>>> is
> > > > > >> > > > >>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> confined to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store itself, but
> > spills
> > > > > over
> > > > > >> and
> > > > > >> > > > >>>>>> changes the
> > > > > >> > > > >>>>>>>>>>>>>>>> behavior
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I
> agree
> > > with
> > > > > >> you
> > > > > >> > > > >> that
> > > > > >> > > > >>>> it
> > > > > >> > > > >>>>>> is a
> > > > > >> > > > >>>>>>>>>>>>>>>> decent
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compromise. I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the
> > > > > downsides
> > > > > >> > > > >> and
> > > > > >> > > > >>>> list
> > > > > >> > > > >>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> options to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> overcome
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with
> you
> > > that
> > > > > all
> > > > > >> > > > >>>> options
> > > > > >> > > > >>>>>> seem
> > > > > >> > > > >>>>>>>>>>>> quite
> > > > > >> > > > >>>>>>>>>>>>>>>>>> heavy
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compared
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not
> > > understand
> > > > > >> > > > >> what you
> > > > > >> > > > >>>>>> mean
> > > > > >> > > > >>>>>>>> with
> > > > > >> > > > >>>>>>>>>>>>>>>> "less
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> predictable
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I found the discussions
> > about
> > > the
> > > > > >> > > > >>>> alternatives
> > > > > >> > > > >>>>>>>> really
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> interesting.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> But I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also think that your plan
> > > sounds
> > > > > >> good
> > > > > >> > > > >> and
> > > > > >> > > > >>>> we
> > > > > >> > > > >>>>>> should
> > > > > >> > > > >>>>>>>>>>>>>>>> continue
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> with
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> it!
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Some comments on your
> reply
> > > to my
> > > > > >> > > > >> e-mail on
> > > > > >> > > > >>>>>> June
> > > > > >> > > > >>>>>>>>>> 20th:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the
> > > > > reasoning
> > > > > >> > > > >> behind
> > > > > >> > > > >>>>>> putting
> > > > > >> > > > >>>>>>>>>>>>>>>> isolation
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> level
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store context.
> > > Thanks!
> > > > > >> Should
> > > > > >> > > > >>>> that
> > > > > >> > > > >>>>>> also
> > > > > >> > > > >>>>>>>> be
> > > > > >> > > > >>>>>>>>>> a
> > > > > >> > > > >>>>>>>>>>>>>>>> way
> > > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> give
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store the
> > > opportunity to
> > > > > >> > > > >> decide
> > > > > >> > > > >>>>>> whether
> > > > > >> > > > >>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> turn on
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was
> more
> > > > > >> concerned
> > > > > >> > > > >> about
> > > > > >> > > > >>>>>> how do
> > > > > >> > > > >>>>>>>>>> you
> > > > > >> > > > >>>>>>>>>>>>>>>> know
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> if a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to
> be
> > > > > written
> > > > > >> > > > >> under
> > > > > >> > > > >>>> EOS,
> > > > > >> > > > >>>>>> if
> > > > > >> > > > >>>>>>>> you
> > > > > >> > > > >>>>>>>>>>>>>>>> do not
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> have a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to know if the state store
> > is
> > > > > >> > > > >>>> transactional or
> > > > > >> > > > >>>>>> not.
> > > > > >> > > > >>>>>>>>>> If
> > > > > >> > > > >>>>>>>>>>>>>>>> a
> > > > > >> > > > >>>>>>>>>>>>>>>>>> state
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional, the
> > checkpoint
> > > file
> > > > > >> can
> > > > > >> > > > >> be
> > > > > >> > > > >>>>>> written
> > > > > >> > > > >>>>>>>>>>>>>>>> during
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> normal
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If
> the
> > > state
> > > > > >> > > > >> store
> > > > > >> > > > >>>> is not
> > > > > >> > > > >>>>>>>>>>>>>>>>>> transactional,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not
> be
> > > > > written
> > > > > >> > > > >> under
> > > > > >> > > > >>>> EOS.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My point was about not
> only
> > > > > >> > > > >> considering the
> > > > > >> > > > >>>>>> bytes
> > > > > >> > > > >>>>>>>> in
> > > > > >> > > > >>>>>>>>>>>>>>>> memory
> > > > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> config
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > statestore.uncommitted.max.bytes,
> > > > > >> but
> > > > > >> > > > >> also
> > > > > >> > > > >>>>>> bytes
> > > > > >> > > > >>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>> might
> > > > > >> > > > >>>>>>>>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> spilled
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was
> > > > > wondering
> > > > > >> > > > >>>> whether you
> > > > > >> > > > >>>>>>>>>> should
> > > > > >> > > > >>>>>>>>>>>>>>>>>> remove
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum
> number
> > of
> > > > > >> memory
> > > > > >> > > > >>>> bytes to
> > > > > >> > > > >>>>>> be
> > > > > >> > > > >>>>>>>>>> used
> > > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted
> > state-store
> > > > > >> > > > >> records." My
> > > > > >> > > > >>>>>>>> thinking
> > > > > >> > > > >>>>>>>>>>>>>>>> was
> > > > > >> > > > >>>>>>>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> even
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store spills
> > uncommitted
> > > > > >> bytes to
> > > > > >> > > > >>>> disk,
> > > > > >> > > > >>>>>>>>>> limiting
> > > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> overall
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> bytes
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking
> > > about
> > > > > it
> > > > > >> > > > >> again
> > > > > >> > > > >>>> and
> > > > > >> > > > >>>>>>>>>>>>>>>> considering
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> recent
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not
> > make
> > > too
> > > > > >> much
> > > > > >> > > > >>>> sense
> > > > > >> > > > >>>>>>>> anymore.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I like the name
> > > > > >> > > > >>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> you
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 8.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A high-level description
> > > (without
> > > > > >> > > > >>>>>> implementation
> > > > > >> > > > >>>>>>>>>>>>>>>> details) of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> how
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the
> > > commit of
> > > > > >> > > > >> changelog
> > > > > >> > > > >>>>>>>>>>>>>>>> transactions,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions and
> > checkpointing
> > > > > >> would be
> > > > > >> > > > >>>> great.
> > > > > >> > > > >>>>>>>> Would
> > > > > >> > > > >>>>>>>>>>>> be
> > > > > >> > > > >>>>>>>>>>>>>>>>>> great
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> if
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> you
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could also add some
> > sentences
> > > > > about
> > > > > >> the
> > > > > >> > > > >>>>>> behavior in
> > > > > >> > > > >>>>>>>>>>>>>>>> case of
> > > > > >> > > > >>>>>>>>>>>>>>>>>> a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> failure.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For instance how does a
> > > > > >> transactional
> > > > > >> > > > >> state
> > > > > >> > > > >>>>>> store
> > > > > >> > > > >>>>>>>>>>>>>>>> recover
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> after a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure or what happens
> with
> > > the
> > > > > >> > > > >>>> transaction
> > > > > >> > > > >>>>>>>> buffer,
> > > > > >> > > > >>>>>>>>>>>>>>>> etc.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> (that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> what
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in
> > > point
> > > > > 9.)
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick
> > > Telford
> > > > > >> wrote:
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the
> same
> > > issue
> > > > > >> that
> > > > > >> > > > >>>>>>>>>>>>>>>> WriteBatchWithIndex
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding
> (or
> > > > > likely
> > > > > >> to
> > > > > >> > > > >>>> exceed)
> > > > > >> > > > >>>>>>>>>>>>>>>> configured
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> memory
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needs to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big
> > > concerns.
> > > > > >> > > > >>>> Ultimately,
> > > > > >> > > > >>>>>> any
> > > > > >> > > > >>>>>>>>>>>>>>>> approach
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> based
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> on
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cracking
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals
> and
> > > using
> > > > > >> it in
> > > > > >> > > > >>>> ways
> > > > > >> > > > >>>>>> it's
> > > > > >> > > > >>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>> really
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> designed
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> likely to have some
> > unforseen
> > > > > >> > > > >> performance
> > > > > >> > > > >>>> or
> > > > > >> > > > >>>>>>>>>>>>>>>> consistency
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> issues.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation
> for
> > > > > removing
> > > > > >> > > > >> these
> > > > > >> > > > >>>>>> early
> > > > > >> > > > >>>>>>>>>>>>>>>> commits?
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> While
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> not
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent
> > > compromise
> > > > > >> to
> > > > > >> > > > >>>> ensure
> > > > > >> > > > >>>>>>>>>>>>>>>> consistency
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> whilst
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maintaining
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good and predictable
> > > performance.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested
> > ideas
> > > > > seem
> > > > > >> > > > >> *very*
> > > > > >> > > > >>>>>>>>>>>>>>>> complicated, and
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> might
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less
> > > predictable
> > > > > for
> > > > > >> > > > >> users
> > > > > >> > > > >>>> as a
> > > > > >> > > > >>>>>>>>>>>>>>>> consequence.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that
> > the
> > > > > scope
> > > > > >> of
> > > > > >> > > > >> this
> > > > > >> > > > >>>>>> KIP is
> > > > > >> > > > >>>>>>>>>>>>>>>> growing a
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> bit
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> out
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good
> to
> > > > > discuss
> > > > > >> > > > >> ideas
> > > > > >> > > > >>>> for
> > > > > >> > > > >>>>>>>> future
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> improvements, I
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > > >> > > > >>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow
> > the
> > > > > scope
> > > > > >> > > > >> down
> > > > > >> > > > >>>> to a
> > > > > >> > > > >>>>>>>> design
> > > > > >> > > > >>>>>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> achieves
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> most
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives
> > (constant
> > > > > sized
> > > > > >> > > > >>>>>> restorations
> > > > > >> > > > >>>>>>>>>>>>>>>> during
> > > > > >> > > > >>>>>>>>>>>>>>>>>> dirty
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors).
> > Any
> > > > > >> design
> > > > > >> > > > >> that
> > > > > >> > > > >>>>>> this KIP
> > > > > >> > > > >>>>>>>>>>>>>>>> produces
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> can
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ultimately
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future,
> > > > > >> especially
> > > > > >> > > > >> if
> > > > > >> > > > >>>> the
> > > > > >> > > > >>>>>> bulk
> > > > > >> > > > >>>>>>>> of
> > > > > >> > > > >>>>>>>>>>>>>>>> it is
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> internal
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some
> > time
> > > next
> > > > > >> week
> > > > > >> > > > >>>> trying
> > > > > >> > > > >>>>>> to
> > > > > >> > > > >>>>>>>>>>>>>>>> re-work
> > > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex
> design
> > to
> > > > > >> remove
> > > > > >> > > > >> the
> > > > > >> > > > >>>>>>>>>>>>>>>> newTransaction()
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> method,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> such
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's just an
> implementation
> > > > > detail
> > > > > >> of
> > > > > >> > > > >>>>>>>> RocksDBStore.
> > > > > >> > > > >>>>>>>>>>>>>>>> That
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> way, if
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with
> something
> > > in
> > > > > the
> > > > > >> > > > >> future,
> > > > > >> > > > >>>>>> like
> > > > > >> > > > >>>>>>>> the
> > > > > >> > > > >>>>>>>>>>>>>>>> SST
> > > > > >> > > > >>>>>>>>>>>>>>>>>> file
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> management
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we
> > > can do
> > > > > so
> > > > > >> > > > >> with
> > > > > >> > > > >>>>>> little/no
> > > > > >> > > > >>>>>>>>>>>> API
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>> changes.
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>>
> > > > > >> > > > >>>>>>>>>>
> > > > > >> > > > >>>>>>>>>
> > > > > >> > > > >>>>>>>>
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>>>
> > > > > >> > > > >>>>
> > > > > >> > > > >>
> > > > > >> > > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > >
> > >
> >
>

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

Posted by Nick Telford <ni...@gmail.com>.
OK, I've adjusted the KIP to use "enable.transactional.statestores" that
now unconditionally defaults to "false".

This required a fairly significant rewrite of the KIP to remove all the
references to isolation level, so you might need to re-read the entire KIP.

I did not update the references to isolation level in the "Rejected
Alternatives" section, because it still seemed relevant there.

If there are no other concerns, I'll bring the KIP to a vote.

Thanks for all your input!

Nick

On Tue, 16 Apr 2024 at 17:33, Sophie Blee-Goldman <so...@responsive.dev>
wrote:

> Yeah I had missed the internal Streams configs when I first looked
> and was going to suggest the "enable.x.y" format based on the client
> configs until I noticed we had internal configs of the form x.y.enabled
>
> But I fully agree that it's silly to align ourselves with internal config
> names
> and we should just follow the established pattern in public configs.
>
> So yeah, I'm on board with enable.transactional.stores
>
> ...I think that's the last open question then?
>
> On Mon, Apr 15, 2024 at 5:32 AM Lucas Brutschy
> <lb...@confluent.io.invalid> wrote:
>
> > Hi Nick,
> >
> > the java clients do have some properties that are of the form
> > `enable....` (`enable.auto.comit` and `enable.idempotance`), so I
> > would go with this form. State updater and processing thread configs
> > use the `....enabled` form, but they are internal and could be changed
> > to the `enable...` form for consistency. But, not intending to start a
> > big discussion here. Both options are fine with me.
> >
> > Thanks for the updates!
> >
> >
> > On Sun, Apr 14, 2024 at 10:58 PM Sophie Blee-Goldman
> > <so...@responsive.dev> wrote:
> > >
> > > Makes sense to me! Regarding the new config name -- doesn't seem like
> > > we have any *public *configs in StreamsConfig that set a standard yet,
> so
> > > imo we're free to pick whatever we think sounds good.
> > >
> > > However we do have a few internal configs that follow the pattern
> > > *feature.name.enabled* so if we want to play it safe and adhere to the
> > > established pattern, we should call it
> > "transactional.statestores.enabled"
> > >
> > > I also personally think x.y.enabled sounds better than enable.x.y, but
> > > honestly I don't feel too strongly either way. I don't think it's worth
> > > prolonging
> > > the discussion any further over
> > >
> > > I'll take a look at KIP-892 as soon as I can since it's a prereq for
> this
> > > one.
> > > Thanks for the updates!
> > >
> > >
> > > On Thu, Apr 11, 2024 at 12:02 PM Nick Telford <ni...@gmail.com>
> > > wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > After various off-list discussions, it's become clear there are still
> > some
> > > > contentious points to this KIP.
> > > >
> > > > 1.
> > > > We agreed to change the "default.state.isolation.level" config
> > property to
> > > > be a transactional state store feature toggle. What should we name
> > this?
> > > > "enable.transactional.statestores"? What's the convention for this
> > kind of
> > > > flag?
> > > >
> > > > 2.
> > > > The "atomic checkpointing" section has been broken out into KIP-1035
> > (with
> > > > one significant change, and otherwise just more explanation). This is
> > now a
> > > > hard dependency of KIP-892.
> > > >
> > > > Regards,
> > > > Nick
> > > >
> > > > On Mon, 6 Nov 2023 at 17:00, Nick Telford <ni...@gmail.com>
> > wrote:
> > > >
> > > > > Hi everyone,
> > > > >
> > > > > Sorry for the delay.
> > > > >
> > > > > I've updated the KIP based on our discussion. I've also added some
> > > > details
> > > > > on "Transactional support under READ_UNCOMMITTED" and  "Query-time
> > > > > Isolation Levels" in the "Rejected Alternatives" section, to
> > highlight
> > > > the
> > > > > potential to improve things in a future KIP.
> > > > >
> > > > > If there are no more requested changes or objections, I'll kick off
> > the
> > > > > vote thread tomorrow, approximately 24 hours from now.
> > > > >
> > > > > While I'd love for this KIP to make it into 3.7.0, I can't commit
> to
> > it
> > > > > being code-complete by the December 20th deadline. While the
> branch I
> > > > have
> > > > > is broadly working, there are areas that will likely require
> > improvement,
> > > > > and others that are untested:
> > > > >
> > > > > - Untested outside of unit tests: Windowed stores, Versioned
> stores,
> > > > > Global stores, IQv2. They should all work as expected, but the
> > Position
> > > > > file logic might be a bit janky.
> > > > > - Requires improvement: Position data handling. Very clunky right
> > now.
> > > > > - Requires implementation: The "feature flag", where we disable
> > > > > transactionality under EOS+READ_UNCOMMITTED. I've started this, but
> > it's
> > > > > far from simple. Plus this will require fleshing out the test suite
> > > > quite a
> > > > > bit.
> > > > >
> > > > > For the curious, my active development branch has been rebased
> > against
> > > > > 3.6.0: https://github.com/nicktelford/kafka/tree/KIP-892-3.6.0
> > > > >
> > > > > Regards,
> > > > > Nick
> > > > >
> > > > > On Mon, 30 Oct 2023 at 23:32, Sophie Blee-Goldman <
> > sophie@responsive.dev
> > > > >
> > > > > wrote:
> > > > >
> > > > >> Hey Nick, sounds like things are moving along here. I think you're
> > > > already
> > > > >> aware, but just as a reminder to us all, the KIP freeze deadline
> > for 3.7
> > > > >> is
> > > > >> approaching and is currently set for Nov 18. I think we can all
> > agree
> > > > that
> > > > >> it would be great to have this KIP accepted by then, and it sounds
> > like
> > > > >> much of the implementation is already completed, in which case
> this
> > > > might
> > > > >> be able to make it into 3.7.
> > > > >>
> > > > >> It sounds like this discussion is wrapping up so I just want to
> > > > summarize
> > > > >> my understanding of the current plan regarding configs since it
> > doesn't
> > > > >> seem like the KIP has been updated to include this yet.
> > > > >>
> > > > >> Basically we're all in agreement to go with option #1 that you
> > presented
> > > > >> earlier, right? Assuming so, I just want to clear up some details
> > around
> > > > >> the default behavior. What happens if:
> > > > >> 1. User sets EOS and sets READ_UNCOMMITTED: txn state stores will
> be
> > > > >> disabled/feature-flagged
> > > > >> 2. User sets EOS and does not set anything for the isolation
> level:
> > > > >> although the default is READ_UNCOMMITTED, enabling EOS will change
> > the
> > > > >> default to READ_COMMITTED and txn state stores will be used
> > > > >>
> > > > >> As for the future, it sounds like when READ_UNCOMMITTED mode is
> > > > >> implemented, we will basically just remove this "feature flag" and
> > txn
> > > > >> state stores will always be used for all EOS cases. EOS users will
> > be
> > > > able
> > > > >> to configure the isolation level independently, although it will
> > still
> > > > >> default to READ_COMMITTED when EOS is enabled and it wasn't
> > explicitly
> > > > >> configured.
> > > > >>
> > > > >> Is this understanding correct? I think this was the right choice
> > > > >> regardless, as it will give people a way to disable the txn stores
> > in an
> > > > >> emergency -- as a project we went a long time with little pressure
> > to
> > > > >> feature flag things, and our users paid the cost for that. Even if
> > we
> > > > >> managed to ship something without bugs, it was often only after an
> > > > intense
> > > > >> period of successive blocker bugs that delayed the entire released
> > for
> > > > >> weeks. Other times, major bugs slipped through and some versions
> > became
> > > > >> unusable for certain use cases. So having some way to disable the
> > txn
> > > > >> state
> > > > >> stores when EOS is used feels like a good strategy, since you just
> > never
> > > > >> know what might get through all the testing we do.
> > > > >>
> > > > >> If this ends up slipping to 4.0 and you manage to implement the
> > > > >> READ_UNCOMMITTED mode within the same release, I think it's worth
> > > > >> considering to add in an additional feature flag, even if it's
> just
> > a
> > > > >> backdoor internal config (eg as we did in KIP-441 with the
> internal
> > task
> > > > >> assignor config). But for now let's assume the first release this
> > KIP
> > > > >> appears in will have the behavior as described above, with
> > > > >> READ_UNCOMMITTED
> > > > >> mode acting as a feature flag
> > > > >>
> > > > >> Nick -- please let us know when you've updated the KIP to clarify
> > the
> > > > >> config behavior, and are ready for a vote!
> > > > >>
> > > > >>
> > > > >> On Sun, Oct 29, 2023 at 12:02 PM Colt McNealy <
> colt@littlehorse.io>
> > > > >> wrote:
> > > > >>
> > > > >> > Guozhang—I agree, I am in favor of moving forward with the KIP
> now
> > > > that
> > > > >> the
> > > > >> > Transactional State Stores will be behind a feature flag.
> > > > >> >
> > > > >> > Nick—I just did a bit more light testing of your branch
> > > > `KIP-892-3.5.0`
> > > > >> > with your most recent changes. I couldn't detect a performance
> > > > >> difference
> > > > >> > versus trunk (in the past there was a slight degradation of
> > > > performance
> > > > >> on
> > > > >> > the restoration path, but that has been fixed). I don't believe
> > that
> > > > >> your
> > > > >> > branch has the state updater thread enabled, so I didn't test
> that
> > > > path
> > > > >> too
> > > > >> > heavily.
> > > > >> >
> > > > >> > As expected, however, our internal correctness tests failed due
> > to the
> > > > >> IQ
> > > > >> > read-your-own-writes issue we discussed previously. The
> community
> > as a
> > > > >> > whole would vastly benefit from this KIP getting over the finish
> > line
> > > > in
> > > > >> > 3.7.0, and so long as it is behind a feature flag so that we at
> > > > >> LittleHorse
> > > > >> > can still guarantee RYOW for our users, I think it's purely a
> win
> > 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 <
> > > > >> guozhang.wang.us@gmail.com
> > > > >> > >
> > > > >> > wrote:
> > > > >> >
> > > > >> > > I'd agree with you guys that as long as we are in agreement
> > about
> > > > the
> > > > >> > > configuration semantics, that would be a big win to move
> > forward for
> > > > >> > > this KIP. As for the TaskCorruptedException handling like
> wiping
> > > > state
> > > > >> > > stores, we can discuss that in the PR rather than in the KIP.
> > > > >> > >
> > > > >> > > Just to clarify, I'm onboard with the latest proposal, and
> > probably
> > > > we
> > > > >> > > can move on for voting on this KIP now?
> > > > >> > >
> > > > >> > > Guozhang
> > > > >> > >
> > > > >> > > On Thu, Oct 19, 2023 at 5:33 AM Bruno Cadonna <
> > cadonna@apache.org>
> > > > >> > wrote:
> > > > >> > > >
> > > > >> > > > Hi Nick,
> > > > >> > > >
> > > > >> > > > What you and Lucas wrote about the different configurations
> of
> > > > >> ALOS/EOS
> > > > >> > > > and READ_COMMITTED/READ_UNCOMMITTED make sense to me. My
> > earlier
> > > > >> > > > concerns about changelogs diverging from the content of the
> > local
> > > > >> state
> > > > >> > > > stores turned out to not apply. So I think, we can move on
> > with
> > > > >> those
> > > > >> > > > configurations.
> > > > >> > > >
> > > > >> > > > Regarding the TaskCorruptedException and wiping out the
> state
> > > > stores
> > > > >> > > > under EOS, couldn't we abort the transaction on the state
> > store
> > > > and
> > > > >> > > > close the task dirty? If the Kafka transaction was indeed
> > > > committed,
> > > > >> > the
> > > > >> > > > store would restore the missing part from the changelog
> > topic. If
> > > > >> the
> > > > >> > > > Kafka transaction was not committed, changelog topic and
> state
> > > > store
> > > > >> > are
> > > > >> > > > in-sync.
> > > > >> > > >
> > > > >> > > > In any case, IMO those are implementation details that we do
> > not
> > > > >> need
> > > > >> > to
> > > > >> > > > discuss and solve in the KIP discussion. We can solve them
> on
> > the
> > > > >> PR.
> > > > >> > > > The important thing is that the processing guarantees hold.
> > > > >> > > >
> > > > >> > > > Best,
> > > > >> > > > Bruno
> > > > >> > > >
> > > > >> > > > On 10/18/23 3:56 PM, Nick Telford wrote:
> > > > >> > > > > Hi Lucas,
> > > > >> > > > >
> > > > >> > > > > TaskCorruptedException is how Streams signals that the
> Task
> > > > state
> > > > >> > > needs to
> > > > >> > > > > be wiped, so we can't retain that exception without also
> > wiping
> > > > >> state
> > > > >> > > on
> > > > >> > > > > timeouts.
> > > > >> > > > >
> > > > >> > > > > Regards,
> > > > >> > > > > Nick
> > > > >> > > > >
> > > > >> > > > > On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <
> > > > >> lbrutschy@confluent.io
> > > > >> > > .invalid>
> > > > >> > > > > wrote:
> > > > >> > > > >
> > > > >> > > > >> Hi Nick,
> > > > >> > > > >>
> > > > >> > > > >> I think indeed the better behavior would be to retry
> > > > >> > commitTransaction
> > > > >> > > > >> until we risk running out of time to meet `
> > > > max.poll.interval.ms
> > > > >> `.
> > > > >> > > > >>
> > > > >> > > > >> However, if it's handled as a `TaskCorruptedException` at
> > the
> > > > >> > moment,
> > > > >> > > > >> I would do the same in this KIP, and leave exception
> > handling
> > > > >> > > > >> improvements to future work. 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 <
> > > > >> > nick.telford@gmail.com>
> > > > >> > > > >> wrote:
> > > > >> > > > >>>
> > > > >> > > > >>> Hi Lucas,
> > > > >> > > > >>>
> > > > >> > > > >>> Yeah, this is pretty much the direction I'm thinking of
> > going
> > > > in
> > > > >> > > now. You
> > > > >> > > > >>> make an interesting point about committing on-error
> under
> > > > >> > > > >>> ALOS/READ_COMMITTED, although I haven't had a chance to
> > think
> > > > >> > > through the
> > > > >> > > > >>> implications yet.
> > > > >> > > > >>>
> > > > >> > > > >>> Something that I ran into earlier this week is an issue
> > with
> > > > the
> > > > >> > new
> > > > >> > > > >>> handling of TimeoutException. Without TX stores,
> > > > >> TimeoutException
> > > > >> > > under
> > > > >> > > > >> EOS
> > > > >> > > > >>> throws a TaskCorruptedException, which wipes the stores.
> > > > >> However,
> > > > >> > > with TX
> > > > >> > > > >>> stores, TimeoutException is now just bubbled up and
> dealt
> > with
> > > > >> as
> > > > >> > it
> > > > >> > > is
> > > > >> > > > >>> under ALOS. The problem arises when the
> > > > >> Producer#commitTransaction
> > > > >> > > call
> > > > >> > > > >>> times out: Streams attempts to ignore the error and
> > continue
> > > > >> > > producing,
> > > > >> > > > >>> which causes the next call to Producer#send to throw
> > > > >> > > > >>> "IllegalStateException: Cannot attempt operation `send`
> > > > because
> > > > >> the
> > > > >> > > > >>> previous call to `commitTransaction` timed out and must
> be
> > > > >> > retried".
> > > > >> > > > >>>
> > > > >> > > > >>> I'm not sure what we should do here: retrying the
> > > > >> commitTransaction
> > > > >> > > seems
> > > > >> > > > >>> logical, but what if it times out again? Where do we
> draw
> > the
> > > > >> line
> > > > >> > > and
> > > > >> > > > >>> shutdown the instance?
> > > > >> > > > >>>
> > > > >> > > > >>> Regards,
> > > > >> > > > >>> Nick
> > > > >> > > > >>>
> > > > >> > > > >>> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <
> > > > >> > lbrutschy@confluent.io
> > > > >> > > > >> .invalid>
> > > > >> > > > >>> wrote:
> > > > >> > > > >>>
> > > > >> > > > >>>> Hi all,
> > > > >> > > > >>>>
> > > > >> > > > >>>> I think I liked your suggestion of allowing EOS with
> > > > >> > > READ_UNCOMMITTED,
> > > > >> > > > >>>> but keep wiping the state on error, and I'd vote for
> this
> > > > >> solution
> > > > >> > > > >>>> when introducing `default.state.isolation.level`. This
> > way,
> > > > >> we'd
> > > > >> > > have
> > > > >> > > > >>>> the most low-risk roll-out of this feature (no behavior
> > > > change
> > > > >> > > without
> > > > >> > > > >>>> reconfiguration), with the possibility of switching to
> > the
> > > > most
> > > > >> > > sane /
> > > > >> > > > >>>> battle-tested default settings in 4.0. Essentially,
> we'd
> > > > have a
> > > > >> > > > >>>> feature flag but call it
> `default.state.isolation.level`
> > and
> > > > >> don't
> > > > >> > > > >>>> have to deprecate it later.
> > > > >> > > > >>>>
> > > > >> > > > >>>> So the possible configurations would then be this:
> > > > >> > > > >>>>
> > > > >> > > > >>>> 1. ALOS/READ_UNCOMMITTED (default) = processing uses
> > > > >> direct-to-DB,
> > > > >> > > IQ
> > > > >> > > > >>>> reads from DB.
> > > > >> > > > >>>> 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ
> > reads
> > > > >> from
> > > > >> > > > >>>> WriteBatch/DB. Flush on error (see note below).
> > > > >> > > > >>>> 3. EOS/READ_UNCOMMITTED (default) = processing uses
> > > > >> direct-to-DB,
> > > > >> > IQ
> > > > >> > > > >>>> reads from DB. Wipe state on error.
> > > > >> > > > >>>> 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ
> > reads
> > > > >> from
> > > > >> > > > >>>> WriteBatch/DB.
> > > > >> > > > >>>>
> > > > >> > > > >>>> I believe the feature is important enough that we will
> > see
> > > > good
> > > > >> > > > >>>> adoption even without changing the default. In 4.0,
> when
> > we
> > > > >> have
> > > > >> > > seen
> > > > >> > > > >>>> this being adopted and is battle-tested, we make
> > > > READ_COMMITTED
> > > > >> > the
> > > > >> > > > >>>> default for EOS, or even READ_COMITTED always the
> > default,
> > > > >> > depending
> > > > >> > > > >>>> on our experiences. And we could add a clever
> > implementation
> > > > of
> > > > >> > > > >>>> READ_UNCOMITTED with WriteBatches later.
> > > > >> > > > >>>>
> > > > >> > > > >>>> The only smell here is that
> > `default.state.isolation.level`
> > > > >> > wouldn't
> > > > >> > > > >>>> be purely an IQ setting, but it would also (slightly)
> > change
> > > > >> the
> > > > >> > > > >>>> behavior of the processing, but that seems unavoidable
> as
> > > > long
> > > > >> as
> > > > >> > we
> > > > >> > > > >>>> haven't solve READ_UNCOMITTED IQ with WriteBatches.
> > > > >> > > > >>>>
> > > > >> > > > >>>> Minor: As for Bruno's point 4, I think if we are
> > concerned
> > > > >> about
> > > > >> > > this
> > > > >> > > > >>>> 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).
> > > > >> > > > >>>>
> > > > >> > > > >>>> Cheers,
> > > > >> > > > >>>> Lucas
> > > > >> > > > >>>>
> > > > >> > > > >>>> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <
> > > > >> > > nick.telford@gmail.com>
> > > > >> > > > >>>> wrote:
> > > > >> > > > >>>>>
> > > > >> > > > >>>>> Hi Guozhang,
> > > > >> > > > >>>>>
> > > > >> > > > >>>>> The KIP as it stands introduces a new configuration,
> > > > >> > > > >>>>> default.state.isolation.level, which is independent of
> > > > >> > > > >> processing.mode.
> > > > >> > > > >>>>> It's intended that this new configuration be used to
> > > > >> configure a
> > > > >> > > > >> global
> > > > >> > > > >>>> IQ
> > > > >> > > > >>>>> isolation level in the short term, with a future KIP
> > > > >> introducing
> > > > >> > > the
> > > > >> > > > >>>>> capability to change the isolation level on a
> per-query
> > > > basis,
> > > > >> > > > >> falling
> > > > >> > > > >>>> back
> > > > >> > > > >>>>> to the "default" defined by this config. That's why I
> > called
> > > > >> it
> > > > >> > > > >>>> "default",
> > > > >> > > > >>>>> for future-proofing.
> > > > >> > > > >>>>>
> > > > >> > > > >>>>> However, it currently includes the caveat that
> > > > >> READ_UNCOMMITTED
> > > > >> > is
> > > > >> > > > >> not
> > > > >> > > > >>>>> available under EOS. I think this is the coupling you
> > are
> > > > >> > alluding
> > > > >> > > > >> to?
> > > > >> > > > >>>>>
> > > > >> > > > >>>>> This isn't intended to be a restriction of the API,
> but
> > is
> > > > >> > > currently
> > > > >> > > > >> a
> > > > >> > > > >>>>> technical limitation. However, after discussing with
> > some
> > > > >> users
> > > > >> > > about
> > > > >> > > > >>>>> use-cases that would require READ_UNCOMMITTED under
> > EOS, I'm
> > > > >> > > > >> inclined to
> > > > >> > > > >>>>> remove that clause and put in the necessary work to
> make
> > > > that
> > > > >> > > > >> combination
> > > > >> > > > >>>>> possible now.
> > > > >> > > > >>>>>
> > > > >> > > > >>>>> I currently see two possible approaches:
> > > > >> > > > >>>>>
> > > > >> > > > >>>>>     1. Disable TX StateStores internally when the
> > > > >> IsolationLevel
> > > > >> > is
> > > > >> > > > >>>>>     READ_UNCOMMITTED and the processing.mode is EOS.
> > This is
> > > > >> more
> > > > >> > > > >>>> difficult
> > > > >> > > > >>>>>     than it sounds, as there are many assumptions
> being
> > made
> > > > >> > > > >> throughout
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>     internals about the guarantees StateStores
> provide.
> > It
> > > > >> would
> > > > >> > > > >>>> definitely add
> > > > >> > > > >>>>>     a lot of extra "if (read_uncommitted && eos)"
> > branches,
> > > > >> > > > >> complicating
> > > > >> > > > >>>>>     maintenance and testing.
> > > > >> > > > >>>>>     2. Invest the time *now* to make READ_UNCOMMITTED
> > of EOS
> > > > >> > > > >> StateStores
> > > > >> > > > >>>>>     possible. I have some ideas on how this could be
> > > > achieved,
> > > > >> > but
> > > > >> > > > >> they
> > > > >> > > > >>>> would
> > > > >> > > > >>>>>     need testing and could introduce some additional
> > issues.
> > > > >> The
> > > > >> > > > >> benefit
> > > > >> > > > >>>> of
> > > > >> > > > >>>>>     this approach is that it would make query-time
> > > > >> > IsolationLevels
> > > > >> > > > >> much
> > > > >> > > > >>>> simpler
> > > > >> > > > >>>>>     to implement in the future.
> > > > >> > > > >>>>>
> > > > >> > > > >>>>> Unfortunately, both will require considerable work
> that
> > will
> > > > >> > > further
> > > > >> > > > >>>> delay
> > > > >> > > > >>>>> this KIP, which was the reason I placed the
> restriction
> > in
> > > > the
> > > > >> > KIP
> > > > >> > > > >> in the
> > > > >> > > > >>>>> first place.
> > > > >> > > > >>>>>
> > > > >> > > > >>>>> Regards,
> > > > >> > > > >>>>> Nick
> > > > >> > > > >>>>>
> > > > >> > > > >>>>> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
> > > > >> > > > >> guozhang.wang.us@gmail.com>
> > > > >> > > > >>>>> wrote:
> > > > >> > > > >>>>>
> > > > >> > > > >>>>>> Hello Nick,
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>>> First of all, thanks a lot for the great effort
> you've
> > put
> > > > in
> > > > >> > > > >> driving
> > > > >> > > > >>>>>> this KIP! I really like it coming through finally, as
> > many
> > > > >> > people
> > > > >> > > > >> in
> > > > >> > > > >>>>>> the community have raised this. At the same time I
> > honestly
> > > > >> > feel a
> > > > >> > > > >> bit
> > > > >> > > > >>>>>> ashamed for not putting enough of my time supporting
> > it and
> > > > >> > > > >> pushing it
> > > > >> > > > >>>>>> through the finish line (you raised this KIP almost a
> > year
> > > > >> ago).
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>>> I briefly passed through the DISCUSS thread so far,
> not
> > > > sure
> > > > >> > I've
> > > > >> > > > >> 100
> > > > >> > > > >>>>>> percent digested all the bullet points. But with the
> > goal
> > > > of
> > > > >> > > > >> trying to
> > > > >> > > > >>>>>> help take it through the finish line in mind, I'd
> want
> > to
> > > > >> throw
> > > > >> > > > >>>>>> thoughts on top of my head only on the point #4 above
> > > > which I
> > > > >> > felt
> > > > >> > > > >> may
> > > > >> > > > >>>>>> be the main hurdle for the current KIP to drive to a
> > > > >> consensus
> > > > >> > > now.
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>>> The general question I asked myself is, whether we
> > want to
> > > > >> > couple
> > > > >> > > > >> "IQ
> > > > >> > > > >>>>>> reading mode" with "processing mode". While
> > technically I
> > > > >> tend
> > > > >> > to
> > > > >> > > > >>>>>> agree with you that, it's feels like a bug if some
> > single
> > > > >> user
> > > > >> > > > >> chose
> > > > >> > > > >>>>>> "EOS" for processing mode while choosing "read
> > uncommitted"
> > > > >> for
> > > > >> > IQ
> > > > >> > > > >>>>>> reading mode, at the same time, I'm thinking if it's
> > > > possible
> > > > >> > that
> > > > >> > > > >>>>>> there could be two different persons (or even two
> > teams)
> > > > that
> > > > >> > > > >> would be
> > > > >> > > > >>>>>> using the stream API to build the app, and the IQ API
> > to
> > > > >> query
> > > > >> > the
> > > > >> > > > >>>>>> running state of the app. I know this is less of a
> > > > technical
> > > > >> > thing
> > > > >> > > > >> but
> > > > >> > > > >>>>>> rather a more design stuff, but if it could be ever
> the
> > > > case,
> > > > >> > I'm
> > > > >> > > > >>>>>> wondering if the personale using the IQ API knows
> > about the
> > > > >> > risks
> > > > >> > > > >> of
> > > > >> > > > >>>>>> using read uncommitted but still chose so for the
> > favor of
> > > > >> > > > >>>>>> performance, no matter if the underlying stream
> > processing
> > > > >> mode
> > > > >> > > > >>>>>> configured by another personale is EOS or not. In
> that
> > > > >> regard,
> > > > >> > I'm
> > > > >> > > > >>>>>> leaning towards a "leaving the door open, and close
> it
> > > > later
> > > > >> if
> > > > >> > we
> > > > >> > > > >>>>>> found it's a bad idea" aspect with a configuration
> > 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 can
> still
> > > > >> enforce
> > > > >> > > > >>>>>> coupling them.
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>>> Again, just my 2c here. Thanks again for the great
> > patience
> > > > >> and
> > > > >> > > > >>>>>> diligence on this KIP.
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>>> Guozhang
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>>> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
> > > > >> > > > >> nick.telford@gmail.com>
> > > > >> > > > >>>>>> wrote:
> > > > >> > > > >>>>>>>
> > > > >> > > > >>>>>>> Hi Bruno,
> > > > >> > > > >>>>>>>
> > > > >> > > > >>>>>>> 4.
> > > > >> > > > >>>>>>> I'll hold off on making that change until we have a
> > > > >> consensus
> > > > >> > as
> > > > >> > > > >> to
> > > > >> > > > >>>> what
> > > > >> > > > >>>>>>> configuration to use to control all of this, as
> it'll
> > be
> > > > >> > > > >> affected by
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>> decision on EOS isolation levels.
> > > > >> > > > >>>>>>>
> > > > >> > > > >>>>>>> 5.
> > > > >> > > > >>>>>>> Done. I've chosen "committedOffsets".
> > > > >> > > > >>>>>>>
> > > > >> > > > >>>>>>> Regards,
> > > > >> > > > >>>>>>> Nick
> > > > >> > > > >>>>>>>
> > > > >> > > > >>>>>>> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <
> > > > >> > cadonna@apache.org>
> > > > >> > > > >>>> wrote:
> > > > >> > > > >>>>>>>
> > > > >> > > > >>>>>>>> Hi Nick,
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>>> 1.
> > > > >> > > > >>>>>>>> Yeah, you are probably right that it does not make
> > too
> > > > much
> > > > >> > > > >> sense.
> > > > >> > > > >>>>>>>> Thanks for the clarification!
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>>> 4.
> > > > >> > > > >>>>>>>> Yes, sorry for the back and forth, but I think for
> > the
> > > > >> sake of
> > > > >> > > > >> the
> > > > >> > > > >>>> KIP
> > > > >> > > > >>>>>>>> it is better to let the ALOS behavior as it is for
> > now
> > > > due
> > > > >> to
> > > > >> > > > >> the
> > > > >> > > > >>>>>>>> possible issues you would run into. Maybe we can
> > find a
> > > > >> > > > >> solution
> > > > >> > > > >>>> in the
> > > > >> > > > >>>>>>>> future. Now the question returns to whether we
> really
> > > > need
> > > > >> > > > >>>>>>>> default.state.isolation.level. Maybe the config
> > could be
> > > > >> the
> > > > >> > > > >>>> feature
> > > > >> > > > >>>>>>>> flag Sophie requested.
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>>> 5.
> > > > >> > > > >>>>>>>> There 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
> > > > >> committedOffsetFor().
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>>> Best,
> > > > >> > > > >>>>>>>> Bruno
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>>> On 10/13/23 3:21 PM, Nick Telford wrote:
> > > > >> > > > >>>>>>>>> Hi Bruno,
> > > > >> > > > >>>>>>>>>
> > > > >> > > > >>>>>>>>> Thanks for getting back to me.
> > > > >> > > > >>>>>>>>>
> > > > >> > > > >>>>>>>>> 1.
> > > > >> > > > >>>>>>>>> I think this should be possible. Are you thinking
> > of the
> > > > >> > > > >>>> situation
> > > > >> > > > >>>>>> where
> > > > >> > > > >>>>>>>> a
> > > > >> > > > >>>>>>>>> user may downgrade to a previous version of Kafka
> > > > >> Streams? In
> > > > >> > > > >>>> that
> > > > >> > > > >>>>>> case,
> > > > >> > > > >>>>>>>>> sadly, the RocksDBStore would get wiped by the
> older
> > > > >> version
> > > > >> > > > >> of
> > > > >> > > > >>>> Kafka
> > > > >> > > > >>>>>>>>> Streams anyway, because that version wouldn't
> > understand
> > > > >> the
> > > > >> > > > >>>> extra
> > > > >> > > > >>>>>> column
> > > > >> > > > >>>>>>>>> family (that holds offsets), so the missing
> Position
> > > > file
> > > > >> > > > >> would
> > > > >> > > > >>>>>>>>> automatically get rebuilt when the store is
> rebuilt
> > from
> > > > >> the
> > > > >> > > > >>>>>> changelog.
> > > > >> > > > >>>>>>>>> Are there other situations than downgrade where a
> > > > >> > > > >> transactional
> > > > >> > > > >>>> store
> > > > >> > > > >>>>>>>> could
> > > > >> > > > >>>>>>>>> be replaced by a non-transactional one? I can't
> > think of
> > > > >> any.
> > > > >> > > > >>>>>>>>>
> > > > >> > > > >>>>>>>>> 2.
> > > > >> > > > >>>>>>>>> Ahh yes, the Test Plan - my Kryptonite! This
> section
> > > > >> > > > >> definitely
> > > > >> > > > >>>>>> needs to
> > > > >> > > > >>>>>>>> be
> > > > >> > > > >>>>>>>>> fleshed out. I'll work on that. How much detail do
> > you
> > > > >> need?
> > > > >> > > > >>>>>>>>>
> > > > >> > > > >>>>>>>>> 3.
> > > > >> > > > >>>>>>>>> See my previous email discussing this.
> > > > >> > > > >>>>>>>>>
> > > > >> > > > >>>>>>>>> 4.
> > > > >> > > > >>>>>>>>> Hmm, this is an interesting point. Are you
> > suggesting
> > > > that
> > > > >> > > > >> under
> > > > >> > > > >>>> ALOS
> > > > >> > > > >>>>>>>>> READ_COMMITTED should not be supported?
> > > > >> > > > >>>>>>>>>
> > > > >> > > > >>>>>>>>> Regards,
> > > > >> > > > >>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>
> > > > >> > > > >>>>>>>>> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
> > > > >> > > > >> cadonna@apache.org>
> > > > >> > > > >>>>>> wrote:
> > > > >> > > > >>>>>>>>>
> > > > >> > > > >>>>>>>>>> Hi Nick,
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>> I think the KIP is converging!
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>> 1.
> > > > >> > > > >>>>>>>>>> I am wondering whether it makes sense to write
> the
> > > > >> position
> > > > >> > > > >> file
> > > > >> > > > >>>>>> during
> > > > >> > > > >>>>>>>>>> close as we do for the checkpoint file, so that
> in
> > case
> > > > >> the
> > > > >> > > > >>>> state
> > > > >> > > > >>>>>> store
> > > > >> > > > >>>>>>>>>> is replaced with a non-transactional state store
> > the
> > > > >> > > > >>>>>> non-transactional
> > > > >> > > > >>>>>>>>>> state store finds the position file. I think,
> this
> > is
> > > > not
> > > > >> > > > >>>> strictly
> > > > >> > > > >>>>>>>>>> needed, but would be a nice behavior instead of
> > just
> > > > >> > > > >> deleting
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>>>>> position file.
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>> 2.
> > > > >> > > > >>>>>>>>>> The test plan does not mention integration tests.
> > Do
> > > > you
> > > > >> not
> > > > >> > > > >>>> need to
> > > > >> > > > >>>>>>>>>> extend existing ones and add new ones. Also for
> > > > upgrading
> > > > >> > > > >> and
> > > > >> > > > >>>>>>>>>> downgrading you might need integration and/or
> > system
> > > > >> tests.
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>> 3.
> > > > >> > > > >>>>>>>>>> I think Sophie made a point. Although, IQ reading
> > from
> > > > >> > > > >>>> uncommitted
> > > > >> > > > >>>>>> data
> > > > >> > > > >>>>>>>>>> under EOS might be considered a bug by some
> people.
> > > > Thus,
> > > > >> > > > >> your
> > > > >> > > > >>>> KIP
> > > > >> > > > >>>>>> would
> > > > >> > > > >>>>>>>>>> fix a bug rather than changing the intended
> > behavior.
> > > > >> > > > >> However, I
> > > > >> > > > >>>>>> also
> > > > >> > > > >>>>>>>>>> see that a feature flag would help users that
> rely
> > on
> > > > >> this
> > > > >> > > > >> buggy
> > > > >> > > > >>>>>>>>>> behavior (at least until AK 4.0).
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>> 4.
> > > > >> > > > >>>>>>>>>> This is related to the previous point. I assume
> > that
> > > > the
> > > > >> > > > >>>> difference
> > > > >> > > > >>>>>>>>>> between READ_COMMITTED and READ_UNCOMMITTED for
> > ALOS is
> > > > >> > > > >> that in
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>>>>> former you enable transactions on the state store
> > and
> > > > in
> > > > >> the
> > > > >> > > > >>>> latter
> > > > >> > > > >>>>>> you
> > > > >> > > > >>>>>>>>>> disable them. If my assumption is correct, I
> think
> > that
> > > > >> is
> > > > >> > > > >> an
> > > > >> > > > >>>> issue.
> > > > >> > > > >>>>>>>>>> Let's assume under ALOS Streams fails over a
> > couple of
> > > > >> times
> > > > >> > > > >>>> more or
> > > > >> > > > >>>>>>>>>> less at the same step in processing after value 3
> > is
> > > > >> added
> > > > >> > > > >> to an
> > > > >> > > > >>>>>>>>>> aggregation but the offset of the corresponding
> > input
> > > > >> record
> > > > >> > > > >>>> was not
> > > > >> > > > >>>>>>>>>> committed. Without transactions disabled, the
> > > > aggregation
> > > > >> > > > >> value
> > > > >> > > > >>>>>> would
> > > > >> > > > >>>>>>>>>> increase by 3 for each failover. With
> transactions
> > > > >> enabled,
> > > > >> > > > >>>> value 3
> > > > >> > > > >>>>>>>>>> would only be added to the aggregation once when
> > the
> > > > >> offset
> > > > >> > > > >> of
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>> input
> > > > >> > > > >>>>>>>>>> record is committed and the transaction finally
> > > > >> completes.
> > > > >> > > > >> So
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>>>>> content of the state store would change depending
> > on
> > > > the
> > > > >> > > > >>>>>> configuration
> > > > >> > > > >>>>>>>>>> for IQ. IMO, the content of the state store
> should
> > be
> > > > >> > > > >>>> independent
> > > > >> > > > >>>>>> from
> > > > >> > > > >>>>>>>>>> IQ. Given this issue, I propose to not use
> > transactions
> > > > >> with
> > > > >> > > > >>>> ALOS at
> > > > >> > > > >>>>>>>>>> all. I was a big proponent of using transactions
> > with
> > > > >> ALOS,
> > > > >> > > > >> but
> > > > >> > > > >>>> I
> > > > >> > > > >>>>>>>>>> realized that transactions with ALOS is not as
> > easy as
> > > > >> > > > >> enabling
> > > > >> > > > >>>>>>>>>> transactions on state stores. Another aspect that
> > is
> > > > >> > > > >>>> problematic is
> > > > >> > > > >>>>>> that
> > > > >> > > > >>>>>>>>>> the changelog topic which actually replicates the
> > state
> > > > >> > > > >> store
> > > > >> > > > >>>> is not
> > > > >> > > > >>>>>>>>>> transactional under ALOS. Thus, it might happen
> > that
> > > > the
> > > > >> > > > >> state
> > > > >> > > > >>>>>> store and
> > > > >> > > > >>>>>>>>>> the changelog differ in their content. All of
> this
> > is
> > > > >> maybe
> > > > >> > > > >>>> solvable
> > > > >> > > > >>>>>>>>>> somehow, but for the sake of this KIP, I would
> > leave it
> > > > >> for
> > > > >> > > > >> the
> > > > >> > > > >>>>>> future.
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>> Best,
> > > > >> > > > >>>>>>>>>> Bruno
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > > > >> > > > >>>>>>>>>>> Hey Nick! First of all thanks for taking up this
> > > > awesome
> > > > >> > > > >>>> feature,
> > > > >> > > > >>>>>> I'm
> > > > >> > > > >>>>>>>>>> sure
> > > > >> > > > >>>>>>>>>>> every single
> > > > >> > > > >>>>>>>>>>> Kafka Streams user and dev would agree that it
> is
> > > > sorely
> > > > >> > > > >>>> needed.
> > > > >> > > > >>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>> I've just been catching up on the KIP and
> > surrounding
> > > > >> > > > >>>> discussion,
> > > > >> > > > >>>>>> so
> > > > >> > > > >>>>>>>>>> please
> > > > >> > > > >>>>>>>>>>> forgive me
> > > > >> > > > >>>>>>>>>>> for any misunderstandings or misinterpretations
> > of the
> > > > >> > > > >> current
> > > > >> > > > >>>>>> plan and
> > > > >> > > > >>>>>>>>>>> don't hesitate to
> > > > >> > > > >>>>>>>>>>> correct me.
> > > > >> > > > >>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>> Before I jump in, I just want to say that having
> > seen
> > > > >> this
> > > > >> > > > >>>> drag on
> > > > >> > > > >>>>>> for
> > > > >> > > > >>>>>>>> so
> > > > >> > > > >>>>>>>>>>> long, my singular
> > > > >> > > > >>>>>>>>>>> goal in responding is to help this KIP past a
> > > > perceived
> > > > >> > > > >>>> impasse so
> > > > >> > > > >>>>>> we
> > > > >> > > > >>>>>>>> can
> > > > >> > > > >>>>>>>>>>> finally move on
> > > > >> > > > >>>>>>>>>>> to voting and implementing it. Long discussions
> > are to
> > > > >> be
> > > > >> > > > >>>> expected
> > > > >> > > > >>>>>> for
> > > > >> > > > >>>>>>>>>>> major features like
> > > > >> > > > >>>>>>>>>>> this but it's completely on us as the Streams
> > devs to
> > > > >> make
> > > > >> > > > >> sure
> > > > >> > > > >>>>>> there
> > > > >> > > > >>>>>>>> is
> > > > >> > > > >>>>>>>>>> an
> > > > >> > > > >>>>>>>>>>> end in sight
> > > > >> > > > >>>>>>>>>>> for any ongoing discussion.
> > > > >> > > > >>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>> With that said, it's my understanding that the
> > KIP as
> > > > >> > > > >> currently
> > > > >> > > > >>>>>>>> proposed
> > > > >> > > > >>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>> just not tenable
> > > > >> > > > >>>>>>>>>>> for Kafka Streams, and would prevent some EOS
> > users
> > > > from
> > > > >> > > > >>>> upgrading
> > > > >> > > > >>>>>> to
> > > > >> > > > >>>>>>>> the
> > > > >> > > > >>>>>>>>>>> version it
> > > > >> > > > >>>>>>>>>>> first appears in. Given that we can't predict or
> > > > >> guarantee
> > > > >> > > > >>>> whether
> > > > >> > > > >>>>>> any
> > > > >> > > > >>>>>>>> of
> > > > >> > > > >>>>>>>>>>> the followup KIPs
> > > > >> > > > >>>>>>>>>>> would be completed in the same release cycle as
> > this
> > > > >> one,
> > > > >> > > > >> we
> > > > >> > > > >>>> need
> > > > >> > > > >>>>>> to
> > > > >> > > > >>>>>>>> make
> > > > >> > > > >>>>>>>>>>> sure that the
> > > > >> > > > >>>>>>>>>>> feature is either compatible with all current
> > users or
> > > > >> else
> > > > >> > > > >>>>>>>>>> feature-flagged
> > > > >> > > > >>>>>>>>>>> so that they may
> > > > >> > > > >>>>>>>>>>> opt in/out.
> > > > >> > > > >>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>> Therefore, IIUC we need to have either (or both)
> > of
> > > > >> these
> > > > >> > > > >> as
> > > > >> > > > >>>>>>>>>>> fully-implemented config options:
> > > > >> > > > >>>>>>>>>>> 1. default.state.isolation.level
> > > > >> > > > >>>>>>>>>>> 2. enable.transactional.state.stores
> > > > >> > > > >>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>> This way EOS users for whom read_committed
> > semantics
> > > > are
> > > > >> > > > >> not
> > > > >> > > > >>>>>> viable can
> > > > >> > > > >>>>>>>>>>> still upgrade,
> > > > >> > > > >>>>>>>>>>> and either use the isolation.level config to
> > leverage
> > > > >> the
> > > > >> > > > >> new
> > > > >> > > > >>>> txn
> > > > >> > > > >>>>>> state
> > > > >> > > > >>>>>>>>>>> stores without sacrificing
> > > > >> > > > >>>>>>>>>>> their application semantics, or else simply keep
> > the
> > > > >> > > > >>>> transactional
> > > > >> > > > >>>>>>>> state
> > > > >> > > > >>>>>>>>>>> stores disabled until we
> > > > >> > > > >>>>>>>>>>> are able to fully implement the isolation level
> > > > >> > > > >> configuration
> > > > >> > > > >>>> at
> > > > >> > > > >>>>>> either
> > > > >> > > > >>>>>>>>>> an
> > > > >> > > > >>>>>>>>>>> application or query level.
> > > > >> > > > >>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>> Frankly you are the expert here and know much
> more
> > > > about
> > > > >> > > > >> the
> > > > >> > > > >>>>>> tradeoffs
> > > > >> > > > >>>>>>>> in
> > > > >> > > > >>>>>>>>>>> both semantics and
> > > > >> > > > >>>>>>>>>>> effort level of implementing one of these
> configs
> > vs
> > > > the
> > > > >> > > > >>>> other. In
> > > > >> > > > >>>>>> my
> > > > >> > > > >>>>>>>>>>> opinion, either option would
> > > > >> > > > >>>>>>>>>>> be fine and I would leave the decision of which
> > one to
> > > > >> > > > >> include
> > > > >> > > > >>>> in
> > > > >> > > > >>>>>> this
> > > > >> > > > >>>>>>>>>> KIP
> > > > >> > > > >>>>>>>>>>> completely up to you.
> > > > >> > > > >>>>>>>>>>> I just don't see a way for the KIP to proceed
> > without
> > > > >> some
> > > > >> > > > >>>>>> variation of
> > > > >> > > > >>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>> above that would allow
> > > > >> > > > >>>>>>>>>>> EOS users to opt-out of read_committed.
> > > > >> > > > >>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>> (If it's all the same to you, I would recommend
> > 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 that gets released.
> > > > >> > > > >>>>>>>>>>> So personally, my choice would be to add the
> > feature
> > > > >> flag
> > > > >> > > > >> and
> > > > >> > > > >>>>>> leave it
> > > > >> > > > >>>>>>>>>> off
> > > > >> > > > >>>>>>>>>>> by default. If all 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
> > > > >> > > > >>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > > > >> > > > >>>>>> nick.telford@gmail.com>
> > > > >> > > > >>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>> Hi Bruno,
> > > > >> > > > >>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>> Agreed, I can live with that for now.
> > > > >> > > > >>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>> In an effort to keep the scope of this KIP from
> > > > >> > > > >> expanding, I'm
> > > > >> > > > >>>>>> leaning
> > > > >> > > > >>>>>>>>>>>> towards just providing a configurable
> > > > >> > > > >>>>>> default.state.isolation.level
> > > > >> > > > >>>>>>>> and
> > > > >> > > > >>>>>>>>>>>> removing IsolationLevel from the
> > StateStoreContext.
> > > > >> This
> > > > >> > > > >>>> would be
> > > > >> > > > >>>>>>>>>>>> compatible with adding support for query-time
> > > > >> > > > >> IsolationLevels
> > > > >> > > > >>>> in
> > > > >> > > > >>>>>> the
> > > > >> > > > >>>>>>>>>>>> future, whilst providing a way for users to
> > select an
> > > > >> > > > >>>> isolation
> > > > >> > > > >>>>>> level
> > > > >> > > > >>>>>>>>>> now.
> > > > >> > > > >>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>> The big problem with this, however, is that if
> a
> > user
> > > > >> > > > >> selects
> > > > >> > > > >>>>>>>>>>>> processing.mode
> > > > >> > > > >>>>>>>>>>>> = "exactly-once(-v2|-beta)", and
> > > > >> > > > >>>> default.state.isolation.level =
> > > > >> > > > >>>>>>>>>>>> "READ_UNCOMMITTED", we need to guarantee that
> the
> > > > data
> > > > >> > > > >> isn't
> > > > >> > > > >>>>>> written
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>> disk until commit() is called, but we also need
> > to
> > > > >> permit
> > > > >> > > > >> IQ
> > > > >> > > > >>>>>> threads
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>> read from the ongoing transaction.
> > > > >> > > > >>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>> A simple solution would be to (temporarily)
> > forbid
> > > > this
> > > > >> > > > >>>>>> combination of
> > > > >> > > > >>>>>>>>>>>> configuration, and have
> > default.state.isolation.level
> > > > >> > > > >>>>>> automatically
> > > > >> > > > >>>>>>>>>> switch
> > > > >> > > > >>>>>>>>>>>> to READ_COMMITTED when processing.mode is
> > anything
> > > > >> other
> > > > >> > > > >> than
> > > > >> > > > >>>>>>>>>>>> at-least-once. Do you think this would be
> > acceptable?
> > > > >> > > > >>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>> In a later KIP, we can add support for
> query-time
> > > > >> > > > >> isolation
> > > > >> > > > >>>>>> levels and
> > > > >> > > > >>>>>>>>>>>> solve this particular problem there, which
> would
> > > > relax
> > > > >> > > > >> this
> > > > >> > > > >>>>>>>> restriction.
> > > > >> > > > >>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>> Regards,
> > > > >> > > > >>>>>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> > > > >> > > > >>>> cadonna@apache.org>
> > > > >> > > > >>>>>>>>>> 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,
> > > > >> > > > >>>>>>>>>>>>> Bruno
> > > > >> > > > >>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > > > >> > > > >>>>>>>>>>>>>> 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 <
> > > > >> > > > >>>>>> nick.telford@gmail.com
> > > > >> > > > >>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> Hi everyone,
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> I agree that having IsolationLevel be
> > determined
> > > > at
> > > > >> > > > >>>> query-time
> > > > >> > > > >>>>>> is
> > > > >> > > > >>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>> ideal design, but there are a few sticking
> > points:
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> 1.
> > > > >> > > > >>>>>>>>>>>>>>> There needs to be some way to communicate
> the
> > > > >> > > > >>>> IsolationLevel
> > > > >> > > > >>>>>> down
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>> RocksDBStore itself, so that the query can
> > respect
> > > > >> it.
> > > > >> > > > >>>> Since
> > > > >> > > > >>>>>> stores
> > > > >> > > > >>>>>>>>>>>> are
> > > > >> > > > >>>>>>>>>>>>>>> "layered" in functionality (i.e.
> > > > ChangeLoggingStore,
> > > > >> > > > >>>>>> MeteredStore,
> > > > >> > > > >>>>>>>>>>>>> etc.),
> > > > >> > > > >>>>>>>>>>>>>>> we need some way to deliver that information
> > to
> > > > the
> > > > >> > > > >> bottom
> > > > >> > > > >>>>>> layer.
> > > > >> > > > >>>>>>>> For
> > > > >> > > > >>>>>>>>>>>>> IQv2,
> > > > >> > > > >>>>>>>>>>>>>>> we can use the existing State#query()
> method,
> > but
> > > > >> IQv1
> > > > >> > > > >> has
> > > > >> > > > >>>> no
> > > > >> > > > >>>>>> way
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>> do
> > > > >> > > > >>>>>>>>>>>>>>> this.
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> A simple approach, which would potentially
> > open up
> > > > >> > > > >> other
> > > > >> > > > >>>>>> options,
> > > > >> > > > >>>>>>>>>>>> would
> > > > >> > > > >>>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>> to add something like:
> > ReadOnlyKeyValueStore<K, V>
> > > > >> > > > >>>>>>>>>>>>>>> readOnlyView(IsolationLevel isolationLevel)
> to
> > > > >> > > > >>>>>>>> ReadOnlyKeyValueStore
> > > > >> > > > >>>>>>>>>>>>> (and
> > > > >> > > > >>>>>>>>>>>>>>> similar to ReadOnlyWindowStore,
> > > > >> ReadOnlySessionStore,
> > > > >> > > > >>>> etc.).
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> 2.
> > > > >> > > > >>>>>>>>>>>>>>> As mentioned above, RocksDB WriteBatches are
> > not
> > > > >> > > > >>>> thread-safe,
> > > > >> > > > >>>>>> which
> > > > >> > > > >>>>>>>>>>>>> causes
> > > > >> > > > >>>>>>>>>>>>>>> a problem if we want to provide
> > READ_UNCOMMITTED
> > > > >> > > > >>>> Iterators. I
> > > > >> > > > >>>>>> also
> > > > >> > > > >>>>>>>>>>>> had a
> > > > >> > > > >>>>>>>>>>>>>>> look at RocksDB Transactions[1], but they
> > solve a
> > > > >> very
> > > > >> > > > >>>>>> different
> > > > >> > > > >>>>>>>>>>>>> problem,
> > > > >> > > > >>>>>>>>>>>>>>> and have the same thread-safety issue.
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> One possible approach that I mentioned is
> > chaining
> > > > >> > > > >>>>>> WriteBatches:
> > > > >> > > > >>>>>>>>>> every
> > > > >> > > > >>>>>>>>>>>>>>> time a new Interactive Query is received
> (i.e.
> > > > >> > > > >>>> readOnlyView,
> > > > >> > > > >>>>>> see
> > > > >> > > > >>>>>>>>>>>> above,
> > > > >> > > > >>>>>>>>>>>>>>> is called) we "freeze" the existing
> > WriteBatch,
> > > > and
> > > > >> > > > >> start a
> > > > >> > > > >>>>>> new one
> > > > >> > > > >>>>>>>>>>>> for
> > > > >> > > > >>>>>>>>>>>>> new
> > > > >> > > > >>>>>>>>>>>>>>> writes. The Interactive Query queries the
> > "chain"
> > > > of
> > > > >> > > > >>>> previous
> > > > >> > > > >>>>>>>>>>>>> WriteBatches
> > > > >> > > > >>>>>>>>>>>>>>> + the underlying database; while the
> > StreamThread
> > > > >> > > > >> starts
> > > > >> > > > >>>>>> writing to
> > > > >> > > > >>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>> *new* WriteBatch. On-commit, the
> StreamThread
> > > > would
> > > > >> > > > >> write
> > > > >> > > > >>>> *all*
> > > > >> > > > >>>>>>>>>>>>>>> WriteBatches in the chain to the database
> > (that
> > > > have
> > > > >> > > > >> not
> > > > >> > > > >>>> yet
> > > > >> > > > >>>>>> been
> > > > >> > > > >>>>>>>>>>>>> written).
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> WriteBatches would be closed/freed only when
> > they
> > > > >> have
> > > > >> > > > >> been
> > > > >> > > > >>>>>> both
> > > > >> > > > >>>>>>>>>>>>>>> committed, and all open Interactive Queries
> on
> > > > them
> > > > >> > > > >> have
> > > > >> > > > >>>> been
> > > > >> > > > >>>>>>>> closed.
> > > > >> > > > >>>>>>>>>>>>> This
> > > > >> > > > >>>>>>>>>>>>>>> would require some reference counting.
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> Obviously a drawback of this approach is the
> > > > >> potential
> > > > >> > > > >> for
> > > > >> > > > >>>>>>>> increased
> > > > >> > > > >>>>>>>>>>>>>>> memory usage: if an Interactive Query is
> > > > long-lived,
> > > > >> > > > >> for
> > > > >> > > > >>>>>> example by
> > > > >> > > > >>>>>>>>>>>>> doing a
> > > > >> > > > >>>>>>>>>>>>>>> full scan over a large database, or even
> just
> > > > >> pausing
> > > > >> > > > >> in
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>> middle
> > > > >> > > > >>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>> an
> > > > >> > > > >>>>>>>>>>>>>>> iteration, then the existing chain of
> > WriteBatches
> > > > >> > > > >> could be
> > > > >> > > > >>>>>> kept
> > > > >> > > > >>>>>>>>>>>> around
> > > > >> > > > >>>>>>>>>>>>> for
> > > > >> > > > >>>>>>>>>>>>>>> a long time, potentially forever.
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> --
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> A.
> > > > >> > > > >>>>>>>>>>>>>>> Going off on a tangent, it looks like in
> > addition
> > > > to
> > > > >> > > > >>>> supporting
> > > > >> > > > >>>>>>>>>>>>>>> READ_COMMITTED queries, we could go further
> > and
> > > > >> support
> > > > >> > > > >>>>>>>>>>>> REPEATABLE_READ
> > > > >> > > > >>>>>>>>>>>>>>> queries (i.e. where subsequent reads to the
> > same
> > > > key
> > > > >> > > > >> in the
> > > > >> > > > >>>>>> same
> > > > >> > > > >>>>>>>>>>>>>>> Interactive Query are guaranteed to yield
> the
> > same
> > > > >> > > > >> value)
> > > > >> > > > >>>> by
> > > > >> > > > >>>>>> making
> > > > >> > > > >>>>>>>>>>>> use
> > > > >> > > > >>>>>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>> RocksDB Snapshots[2]. These are fairly
> > > > lightweight,
> > > > >> so
> > > > >> > > > >> the
> > > > >> > > > >>>>>>>>>> performance
> > > > >> > > > >>>>>>>>>>>>>>> impact is likely to be negligible, but they
> do
> > > > >> require
> > > > >> > > > >>>> that the
> > > > >> > > > >>>>>>>>>>>>> Interactive
> > > > >> > > > >>>>>>>>>>>>>>> Query session can be explicitly closed.
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> This could be achieved if we made the above
> > > > >> > > > >> readOnlyView
> > > > >> > > > >>>>>> interface
> > > > >> > > > >>>>>>>>>>>> look
> > > > >> > > > >>>>>>>>>>>>>>> more like:
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueView<K, V>
> > implements
> > > > >> > > > >>>>>>>>>>>> ReadOnlyKeyValueStore<K,
> > > > >> > > > >>>>>>>>>>>>>>> V>, AutoCloseable {}
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > > > >> > > > >>>>>>>>>>>>>>>         ...
> > > > >> > > > >>>>>>>>>>>>>>>         ReadOnlyKeyValueView<K, V>
> > > > >> > > > >>>> readOnlyView(IsolationLevel
> > > > >> > > > >>>>>>>>>>>>> isolationLevel);
> > > > >> > > > >>>>>>>>>>>>>>> }
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> But this would be a breaking change, as
> > existing
> > > > >> IQv1
> > > > >> > > > >>>> queries
> > > > >> > > > >>>>>> are
> > > > >> > > > >>>>>>>>>>>>>>> guaranteed to never call store.close(), and
> > > > >> therefore
> > > > >> > > > >> these
> > > > >> > > > >>>>>> would
> > > > >> > > > >>>>>>>>>> leak
> > > > >> > > > >>>>>>>>>>>>>>> memory under REPEATABLE_READ.
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> B.
> > > > >> > > > >>>>>>>>>>>>>>> One thing that's notable: MyRocks states
> that
> > they
> > > > >> > > > >> support
> > > > >> > > > >>>>>>>>>>>>> READ_COMMITTED
> > > > >> > > > >>>>>>>>>>>>>>> and REPEATABLE_READ, but they make no
> mention
> > of
> > > > >> > > > >>>>>>>>>>>> READ_UNCOMMITTED[3][4].
> > > > >> > > > >>>>>>>>>>>>>>> This could be because doing so is
> technically
> > > > >> > > > >>>>>> difficult/impossible
> > > > >> > > > >>>>>>>>>>>> using
> > > > >> > > > >>>>>>>>>>>>>>> the primitives available in RocksDB.
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> --
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> Lucas, to address your points:
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> U1.
> > > > >> > > > >>>>>>>>>>>>>>> It's only "SHOULD" to permit alternative
> (i.e.
> > > > >> > > > >> non-RocksDB)
> > > > >> > > > >>>>>>>>>>>>>>> implementations of StateStore that do not
> > support
> > > > >> > > > >> atomic
> > > > >> > > > >>>>>> writes.
> > > > >> > > > >>>>>>>>>>>>> Obviously
> > > > >> > > > >>>>>>>>>>>>>>> in those cases, the guarantees Kafka Streams
> > > > >> > > > >>>> provides/expects
> > > > >> > > > >>>>>> would
> > > > >> > > > >>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>> relaxed. Do you think we should require all
> > > > >> > > > >>>> implementations to
> > > > >> > > > >>>>>>>>>> support
> > > > >> > > > >>>>>>>>>>>>>>> atomic writes?
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> U2.
> > > > >> > > > >>>>>>>>>>>>>>> Stores can support multiple IsolationLevels.
> > As
> > > > >> we've
> > > > >> > > > >>>> discussed
> > > > >> > > > >>>>>>>>>> above,
> > > > >> > > > >>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>> ideal scenario would be to specify the
> > > > >> IsolationLevel
> > > > >> > > > >> at
> > > > >> > > > >>>>>>>> query-time.
> > > > >> > > > >>>>>>>>>>>>>>> Failing that, I think the second-best
> > approach is
> > > > to
> > > > >> > > > >>>> define the
> > > > >> > > > >>>>>>>>>>>>>>> IsolationLevel for *all* queries based on
> the
> > > > >> > > > >>>> processing.mode,
> > > > >> > > > >>>>>>>> which
> > > > >> > > > >>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>> what the default
> > > > StateStoreContext#isolationLevel()
> > > > >> > > > >>>> achieves.
> > > > >> > > > >>>>>> Would
> > > > >> > > > >>>>>>>>>>>> you
> > > > >> > > > >>>>>>>>>>>>>>> prefer an alternative?
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> While the existing implementation is
> > equivalent to
> > > > >> > > > >>>>>>>> READ_UNCOMMITTED,
> > > > >> > > > >>>>>>>>>>>>> this
> > > > >> > > > >>>>>>>>>>>>>>> can yield unexpected results/errors under
> > EOS, if
> > > > a
> > > > >> > > > >>>>>> transaction is
> > > > >> > > > >>>>>>>>>>>>> rolled
> > > > >> > > > >>>>>>>>>>>>>>> back. While this would be a change in
> > behaviour
> > > > for
> > > > >> > > > >> users,
> > > > >> > > > >>>> it
> > > > >> > > > >>>>>> would
> > > > >> > > > >>>>>>>>>>>> look
> > > > >> > > > >>>>>>>>>>>>>>> more like a bug fix than a breaking change.
> > That
> > > > >> said,
> > > > >> > > > >> we
> > > > >> > > > >>>>>> *could*
> > > > >> > > > >>>>>>>>>> make
> > > > >> > > > >>>>>>>>>>>>> it
> > > > >> > > > >>>>>>>>>>>>>>> configurable, and default to the existing
> > > > behaviour
> > > > >> > > > >>>>>>>>>> (READ_UNCOMMITTED)
> > > > >> > > > >>>>>>>>>>>>>>> instead of inferring it from the
> > processing.mode?
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> N1, N2.
> > > > >> > > > >>>>>>>>>>>>>>> These were only primitives to avoid boxing
> > costs,
> > > > >> but
> > > > >> > > > >> since
> > > > >> > > > >>>>>> this is
> > > > >> > > > >>>>>>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>> a
> > > > >> > > > >>>>>>>>>>>>>>> performance sensitive area, it should be
> fine
> > to
> > > > >> > > > >> change if
> > > > >> > > > >>>>>> that's
> > > > >> > > > >>>>>>>>>>>>> desirable.
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> N3.
> > > > >> > > > >>>>>>>>>>>>>>> It's because the store "manages its own
> > offsets",
> > > > >> which
> > > > >> > > > >>>>>> includes
> > > > >> > > > >>>>>>>> both
> > > > >> > > > >>>>>>>>>>>>>>> committing the offset, *and providing it*
> via
> > > > >> > > > >>>>>> getCommittedOffset().
> > > > >> > > > >>>>>>>>>>>>>>> Personally, I think "managesOffsets" conveys
> > this
> > > > >> best,
> > > > >> > > > >>>> but I
> > > > >> > > > >>>>>> don't
> > > > >> > > > >>>>>>>>>>>> mind
> > > > >> > > > >>>>>>>>>>>>>>> changing it if the nomenclature is unclear.
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> Sorry for the massive emails/essays!
> > > > >> > > > >>>>>>>>>>>>>>> --
> > > > >> > > > >>>>>>>>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> 1:
> > > > >> > > > >> https://github.com/facebook/rocksdb/wiki/Transactions
> > > > >> > > > >>>>>>>>>>>>>>> 2:
> > > > >> https://github.com/facebook/rocksdb/wiki/Snapshot
> > > > >> > > > >>>>>>>>>>>>>>> 3:
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>
> > > > https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > > >> > > > >>>>>>>>>>>>>>> 4:
> > > > >> > > > >>>>
> > https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas
> Brutschy
> > > > >> > > > >>>>>>>>>>>>>>> <lb...@confluent.io.invalid> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>> Hi Nick,
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>> since I last read it in April, the KIP has
> > become
> > > > >> much
> > > > >> > > > >>>>>> cleaner and
> > > > >> > > > >>>>>>>>>>>>>>>> easier to read. Great work!
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>> It feels to me the last big open point is
> > whether
> > > > >> we
> > > > >> > > > >> can
> > > > >> > > > >>>>>> implement
> > > > >> > > > >>>>>>>>>>>>>>>> isolation level as a query parameter. I
> > > > understand
> > > > >> > > > >> that
> > > > >> > > > >>>> there
> > > > >> > > > >>>>>> are
> > > > >> > > > >>>>>>>>>>>>>>>> implementation concerns, but as Colt says,
> it
> > > > would
> > > > >> > > > >> be a
> > > > >> > > > >>>> great
> > > > >> > > > >>>>>>>>>>>>>>>> addition, and would also simplify the
> > migration
> > > > >> path
> > > > >> > > > >> for
> > > > >> > > > >>>> this
> > > > >> > > > >>>>>>>>>> change.
> > > > >> > > > >>>>>>>>>>>>>>>> Is the implementation problem you mentioned
> > > > caused
> > > > >> by
> > > > >> > > > >> the
> > > > >> > > > >>>>>>>> WriteBatch
> > > > >> > > > >>>>>>>>>>>>>>>> not having a notion of a snapshot, as the
> > > > >> underlying
> > > > >> > > > >> DB
> > > > >> > > > >>>>>> iterator
> > > > >> > > > >>>>>>>>>>>> does?
> > > > >> > > > >>>>>>>>>>>>>>>> In that case, I am not sure a chain of
> > > > WriteBatches
> > > > >> > > > >> as you
> > > > >> > > > >>>>>> propose
> > > > >> > > > >>>>>>>>>>>>>>>> would fully solve the problem, but maybe I
> > didn't
> > > > >> dig
> > > > >> > > > >>>> enough
> > > > >> > > > >>>>>> into
> > > > >> > > > >>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> details to fully understand it.
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>> If it's not possible to implement it now,
> > would
> > > > it
> > > > >> be
> > > > >> > > > >> an
> > > > >> > > > >>>>>> option to
> > > > >> > > > >>>>>>>>>>>>>>>> make sure in this KIP that we do not fully
> > close
> > > > >> the
> > > > >> > > > >> door
> > > > >> > > > >>>> on
> > > > >> > > > >>>>>>>>>>>> per-query
> > > > >> > > > >>>>>>>>>>>>>>>> isolation levels in the interface, as it
> may
> > be
> > > > >> > > > >> possible
> > > > >> > > > >>>> to
> > > > >> > > > >>>>>>>>>> implement
> > > > >> > > > >>>>>>>>>>>>>>>> the missing primitives in RocksDB or Speedb
> > in
> > > > the
> > > > >> > > > >> future.
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>> Understanding:
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>> * U1) Why is it only "SHOULD" for
> > > > changelogOffsets
> > > > >> to
> > > > >> > > > >> be
> > > > >> > > > >>>>>> persisted
> > > > >> > > > >>>>>>>>>>>>>>>> atomically with the records?
> > > > >> > > > >>>>>>>>>>>>>>>> * U2) Don't understand the default
> > implementation
> > > > >> of
> > > > >> > > > >>>>>>>>>>>> `isolationLevel`.
> > > > >> > > > >>>>>>>>>>>>>>>> The isolation level should be a property of
> > the
> > > > >> > > > >> underlying
> > > > >> > > > >>>>>> store,
> > > > >> > > > >>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>> not be defined by the default config?
> > Existing
> > > > >> stores
> > > > >> > > > >>>> probably
> > > > >> > > > >>>>>>>> don't
> > > > >> > > > >>>>>>>>>>>>>>>> guarantee READ_COMMITTED, so the default
> > should
> > > > be
> > > > >> to
> > > > >> > > > >>>> return
> > > > >> > > > >>>>>>>>>>>>>>>> READ_UNCOMMITTED.
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>> Nits:
> > > > >> > > > >>>>>>>>>>>>>>>> * N1) Could `getComittedOffset` use an
> > > > >> `OptionalLong`
> > > > >> > > > >>>> return
> > > > >> > > > >>>>>> type,
> > > > >> > > > >>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> avoid the `null`?
> > > > >> > > > >>>>>>>>>>>>>>>> * N2) Could `apporixmateNumUncomittedBytes`
> > use
> > > > an
> > > > >> > > > >>>>>> `OptionalLong`
> > > > >> > > > >>>>>>>>>>>>>>>> return type, to avoid the `-1`?
> > > > >> > > > >>>>>>>>>>>>>>>> * N3) I don't understand why
> `managesOffsets`
> > > > uses
> > > > >> the
> > > > >> > > > >>>>>> 'manage'
> > > > >> > > > >>>>>>>>>> verb,
> > > > >> > > > >>>>>>>>>>>>>>>> whereas all other methods use the "commits"
> > verb.
> > > > >> I'd
> > > > >> > > > >>>> suggest
> > > > >> > > > >>>>>>>>>>>>>>>> `commitsOffsets`.
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>> Either way, it feels this KIP is very close
> > to
> > > > the
> > > > >> > > > >> finish
> > > > >> > > > >>>>>> line,
> > > > >> > > > >>>>>>>> I'm
> > > > >> > > > >>>>>>>>>>>>>>>> looking forward to seeing this in
> production!
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>> Cheers,
> > > > >> > > > >>>>>>>>>>>>>>>> Lucas
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt
> McNealy
> > <
> > > > >> > > > >>>>>> colt@littlehorse.io
> > > > >> > > > >>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>> Making IsolationLevel a query-time
> > constraint,
> > > > >> > > > >> rather
> > > > >> > > > >>>> than
> > > > >> > > > >>>>>>>> linking
> > > > >> > > > >>>>>>>>>>>> it
> > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>> the processing.guarantee.
> > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>> As I understand it, would this allow even
> a
> > user
> > > > >> of
> > > > >> > > > >> EOS
> > > > >> > > > >>>> to
> > > > >> > > > >>>>>>>> control
> > > > >> > > > >>>>>>>>>>>>>>>> whether
> > > > >> > > > >>>>>>>>>>>>>>>>> reading committed or uncommitted records?
> > If so,
> > > > >> I am
> > > > >> > > > >>>> highly
> > > > >> > > > >>>>>> in
> > > > >> > > > >>>>>>>>>>>> favor
> > > > >> > > > >>>>>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>> this.
> > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>> I know that I was one of the early people
> to
> > > > point
> > > > >> > > > >> out
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>>> current
> > > > >> > > > >>>>>>>>>>>>>>>>> shortcoming that IQ reads uncommitted
> > records,
> > > > but
> > > > >> > > > >> just
> > > > >> > > > >>>> this
> > > > >> > > > >>>>>>>>>>>> morning I
> > > > >> > > > >>>>>>>>>>>>>>>>> realized a pattern we use which means that
> > (for
> > > > >> > > > >> certain
> > > > >> > > > >>>>>> queries)
> > > > >> > > > >>>>>>>>>> our
> > > > >> > > > >>>>>>>>>>>>>>>> system
> > > > >> > > > >>>>>>>>>>>>>>>>> needs to be able to read uncommitted
> > records,
> > > > >> which
> > > > >> > > > >> is
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>>> current
> > > > >> > > > >>>>>>>>>>>>>>>> behavior
> > > > >> > > > >>>>>>>>>>>>>>>>> of Kafka Streams in EOS.***
> > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>> If IsolationLevel being a query-time
> > decision
> > > > >> allows
> > > > >> > > > >> for
> > > > >> > > > >>>>>> this,
> > > > >> > > > >>>>>>>> then
> > > > >> > > > >>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>> would be amazing. I would also vote that
> the
> > > > >> default
> > > > >> > > > >>>> behavior
> > > > >> > > > >>>>>>>>>> should
> > > > >> > > > >>>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>> for
> > > > >> > > > >>>>>>>>>>>>>>>>> reading uncommitted records, because it is
> > > > totally
> > > > >> > > > >>>> possible
> > > > >> > > > >>>>>> for a
> > > > >> > > > >>>>>>>>>>>>> valid
> > > > >> > > > >>>>>>>>>>>>>>>>> application to depend on that behavior,
> and
> > > > >> breaking
> > > > >> > > > >> it
> > > > >> > > > >>>> in a
> > > > >> > > > >>>>>>>> minor
> > > > >> > > > >>>>>>>>>>>>>>>> release
> > > > >> > > > >>>>>>>>>>>>>>>>> might be a bit strong.
> > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>> *** (Note, for the curious reader....) Our
> > > > >> > > > >> use-case/query
> > > > >> > > > >>>>>> pattern
> > > > >> > > > >>>>>>>>>>>> is a
> > > > >> > > > >>>>>>>>>>>>>>>> bit
> > > > >> > > > >>>>>>>>>>>>>>>>> complex, but reading "uncommitted" records
> > is
> > > > >> > > > >> actually
> > > > >> > > > >>>> safe
> > > > >> > > > >>>>>> in
> > > > >> > > > >>>>>>>> our
> > > > >> > > > >>>>>>>>>>>>> case
> > > > >> > > > >>>>>>>>>>>>>>>>> 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 soon as the processor
> > > > >> processes
> > > > >> > > > >> the
> > > > >> > > > >>>>>> result.
> > > > >> > > > >>>>>>>> If
> > > > >> > > > >>>>>>>>>>>>> they
> > > > >> > > > >>>>>>>>>>>>>>>>> can't read uncommitted records, they may
> > > > receive a
> > > > >> > > > >> "201 -
> > > > >> > > > >>>>>>>> Created"
> > > > >> > > > >>>>>>>>>>>>>>>>> response, immediately followed by a "404 -
> > 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 <
> > > > >> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> > > > >> > > > >>>>>>>>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>> Addendum:
> > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>> I think we would also face the same
> problem
> > > > with
> > > > >> the
> > > > >> > > > >>>>>> approach
> > > > >> > > > >>>>>>>> John
> > > > >> > > > >>>>>>>>>>>>>>>> outlined
> > > > >> > > > >>>>>>>>>>>>>>>>>> earlier (using the record cache as a
> > > > transaction
> > > > >> > > > >> buffer
> > > > >> > > > >>>> and
> > > > >> > > > >>>>>>>>>>>> flushing
> > > > >> > > > >>>>>>>>>>>>>>>> it
> > > > >> > > > >>>>>>>>>>>>>>>>>> straight to SST files). This is because
> the
> > > > >> record
> > > > >> > > > >> cache
> > > > >> > > > >>>>>> (the
> > > > >> > > > >>>>>>>>>>>>>>>> ThreadCache
> > > > >> > > > >>>>>>>>>>>>>>>>>> class) is not thread-safe, so every
> commit
> > > > would
> > > > >> > > > >>>> invalidate
> > > > >> > > > >>>>>> open
> > > > >> > > > >>>>>>>>>> IQ
> > > > >> > > > >>>>>>>>>>>>>>>>>> Iterators in the same way that RocksDB
> > > > >> WriteBatches
> > > > >> > > > >> do.
> > > > >> > > > >>>>>>>>>>>>>>>>>> --
> > > > >> > > > >>>>>>>>>>>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick
> > Telford <
> > > > >> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> > > > >> > > > >>>>>>>>>>>>>>>>>> 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.
> > > > >> > > > >>>>>>>>>>>>>>>>>>> 2. Making IsolationLevel a query-time
> > > > >> constraint,
> > > > >> > > > >>>> rather
> > > > >> > > > >>>>>> than
> > > > >> > > > >>>>>>>>>>>>>>>> linking it
> > > > >> > > > >>>>>>>>>>>>>>>>>>> to the processing.guarantee.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>> There's a wrinkle that makes this a
> > challenge:
> > > > >> > > > >>>> Interactive
> > > > >> > > > >>>>>>>>>> Queries
> > > > >> > > > >>>>>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>>> open an Iterator, when using
> transactions
> > and
> > > > >> > > > >>>>>> READ_UNCOMMITTED.
> > > > >> > > > >>>>>>>>>>>>>>>>>>> The problem is that under
> > READ_UNCOMMITTED,
> > > > >> queries
> > > > >> > > > >>>> need
> > > > >> > > > >>>>>> to be
> > > > >> > > > >>>>>>>>>>>> able
> > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>> read records from the currently
> > uncommitted
> > > > >> > > > >> transaction
> > > > >> > > > >>>>>> buffer
> > > > >> > > > >>>>>>>>>>>>>>>>>>> (WriteBatch). This includes for
> Iterators,
> > > > which
> > > > >> > > > >> should
> > > > >> > > > >>>>>> iterate
> > > > >> > > > >>>>>>>>>>>>>>>> both the
> > > > >> > > > >>>>>>>>>>>>>>>>>>> transaction buffer and underlying
> database
> > > > >> (using
> > > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>> The issue is that when the StreamThread
> > > > >> commits, it
> > > > >> > > > >>>> writes
> > > > >> > > > >>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> current
> > > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch to RocksDB *and then clears
> the
> > > > >> > > > >> WriteBatch*.
> > > > >> > > > >>>>>>>> Clearing
> > > > >> > > > >>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch while an Interactive Query
> > holds an
> > > > >> open
> > > > >> > > > >>>>>> Iterator on
> > > > >> > > > >>>>>>>>>> it
> > > > >> > > > >>>>>>>>>>>>>>>> will
> > > > >> > > > >>>>>>>>>>>>>>>>>>> invalidate the Iterator. Worse, it turns
> > out
> > > > >> that
> > > > >> > > > >>>> Iterators
> > > > >> > > > >>>>>>>> over
> > > > >> > > > >>>>>>>>>> a
> > > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch become invalidated not just
> > when
> > > > the
> > > > >> > > > >>>> WriteBatch
> > > > >> > > > >>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>> cleared,
> > > > >> > > > >>>>>>>>>>>>>>>>>> but
> > > > >> > > > >>>>>>>>>>>>>>>>>>> also when the Iterators' current key
> > receives
> > > > a
> > > > >> new
> > > > >> > > > >>>> write.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>> Now that I'm writing this, I remember
> that
> > > > this
> > > > >> is
> > > > >> > > > >> the
> > > > >> > > > >>>>>> major
> > > > >> > > > >>>>>>>>>>>> reason
> > > > >> > > > >>>>>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>> I
> > > > >> > > > >>>>>>>>>>>>>>>>>>> switched the original design from
> having a
> > > > >> > > > >> query-time
> > > > >> > > > >>>>>>>>>>>>>>>> IsolationLevel to
> > > > >> > > > >>>>>>>>>>>>>>>>>>> having the IsolationLevel linked to the
> > > > >> > > > >>>> transactionality
> > > > >> > > > >>>>>> of the
> > > > >> > > > >>>>>>>>>>>>>>>> stores
> > > > >> > > > >>>>>>>>>>>>>>>>>>> themselves.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>> It *might* be possible to resolve this,
> by
> > > > >> having a
> > > > >> > > > >>>>>> "chain" of
> > > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatches, with the StreamThread
> > switching
> > > > >> to a
> > > > >> > > > >> new
> > > > >> > > > >>>>>>>> WriteBatch
> > > > >> > > > >>>>>>>>>>>>>>>>>> whenever
> > > > >> > > > >>>>>>>>>>>>>>>>>>> a new Interactive Query attempts to read
> > from
> > > > >> the
> > > > >> > > > >>>>>> database, but
> > > > >> > > > >>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>> could
> > > > >> > > > >>>>>>>>>>>>>>>>>>> cause some performance problems/memory
> > > > pressure
> > > > >> > > > >> when
> > > > >> > > > >>>>>> subjected
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>> a
> > > > >> > > > >>>>>>>>>>>>>>>> high
> > > > >> > > > >>>>>>>>>>>>>>>>>>> Interactive Query load. It would also
> > reduce
> > > > the
> > > > >> > > > >>>>>> efficiency of
> > > > >> > > > >>>>>>>>>>>>>>>>>> WriteBatches
> > > > >> > > > >>>>>>>>>>>>>>>>>>> on-commit, as we'd have to write N
> > > > WriteBatches,
> > > > >> > > > >> where
> > > > >> > > > >>>> N
> > > > >> > > > >>>>>> is the
> > > > >> > > > >>>>>>>>>>>>>>>> number of
> > > > >> > > > >>>>>>>>>>>>>>>>>>> Interactive Queries since the last
> commit.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>> I realise this is getting into the weeds
> > of
> > > > the
> > > > >> > > > >>>>>> implementation,
> > > > >> > > > >>>>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>> you'd
> > > > >> > > > >>>>>>>>>>>>>>>>>>> rather we focus on the API for now, but
> I
> > > > think
> > > > >> > > > >> it's
> > > > >> > > > >>>>>> important
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>> consider
> > > > >> > > > >>>>>>>>>>>>>>>>>>> how to implement the desired API, in
> case
> > we
> > > > >> come
> > > > >> > > > >> up
> > > > >> > > > >>>> with
> > > > >> > > > >>>>>> an
> > > > >> > > > >>>>>>>> API
> > > > >> > > > >>>>>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>>> cannot be implemented efficiently, or
> > even at
> > > > >> all!
> > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>> Thoughts?
> > > > >> > > > >>>>>>>>>>>>>>>>>>> --
> > > > >> > > > >>>>>>>>>>>>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno
> > Cadonna <
> > > > >> > > > >>>>>>>> cadonna@apache.org
> > > > >> > > > >>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> 6.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> Of course, you are right! My bad!
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> Wiping out the state in the downgrading
> > case
> > > > is
> > > > >> > > > >> fine.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> 3a.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> Focus on the public facing changes for
> > the
> > > > >> KIP. We
> > > > >> > > > >>>> will
> > > > >> > > > >>>>>> manage
> > > > >> > > > >>>>>>>>>> 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.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> Best,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> Bruno
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford
> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Thanks for getting back to me!
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> 2.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> The fact that implementations can
> always
> > > > track
> > > > >> > > > >>>> estimated
> > > > >> > > > >>>>>>>> memory
> > > > >> > > > >>>>>>>>>>>>>>>> usage
> > > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> the wrapper is a good point. I can
> > remove -1
> > > > >> as
> > > > >> > > > >> an
> > > > >> > > > >>>>>> option,
> > > > >> > > > >>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>> I'll
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> clarify
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> the JavaDoc that 0 is not just for
> > > > >> > > > >> non-transactional
> > > > >> > > > >>>>>> stores,
> > > > >> > > > >>>>>>>>>>>>>>>> which is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> currently misleading.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> 6.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> The problem with catching the
> exception
> > in
> > > > the
> > > > >> > > > >>>> downgrade
> > > > >> > > > >>>>>>>>>> process
> > > > >> > > > >>>>>>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> would require new code in the Kafka
> > version
> > > > >> being
> > > > >> > > > >>>>>> downgraded
> > > > >> > > > >>>>>>>>>> to.
> > > > >> > > > >>>>>>>>>>>>>>>> Since
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> users could conceivably downgrade to
> > almost
> > > > >> *any*
> > > > >> > > > >>>> older
> > > > >> > > > >>>>>>>> version
> > > > >> > > > >>>>>>>>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>>> Kafka
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Streams, I'm not sure how we could add
> > that
> > > > >> code?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> The only way I can think of doing it
> > would
> > > > be
> > > > >> to
> > > > >> > > > >>>> provide
> > > > >> > > > >>>>>> a
> > > > >> > > > >>>>>>>>>>>>>>>> dedicated
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> downgrade tool, that goes through
> every
> > > > local
> > > > >> > > > >> store
> > > > >> > > > >>>> and
> > > > >> > > > >>>>>>>> removes
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> offsets column families. But that
> seems
> > like
> > > > >> an
> > > > >> > > > >>>>>> unnecessary
> > > > >> > > > >>>>>>>>>>>>>>>> amount of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> extra
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> code to maintain just to handle a
> > somewhat
> > > > >> niche
> > > > >> > > > >>>>>> situation,
> > > > >> > > > >>>>>>>>>> when
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> alternative (automatically wipe and
> > restore
> > > > >> > > > >> stores)
> > > > >> > > > >>>>>> should be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> acceptable.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes
> > > > you've
> > > > >> > > > >>>> requested.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> 3a.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> I agree that IsolationLevel makes more
> > sense
> > > > >> at
> > > > >> > > > >>>>>> query-time,
> > > > >> > > > >>>>>>>> and
> > > > >> > > > >>>>>>>>>>>> I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> actually
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> initially attempted to place the
> > > > >> IsolationLevel
> > > > >> > > > >> at
> > > > >> > > > >>>>>>>> query-time,
> > > > >> > > > >>>>>>>>>>>>>>>> but I
> > > > >> > > > >>>>>>>>>>>>>>>>>> ran
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> into some problems:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> - The key issue is that, under ALOS
> > we're
> > > > not
> > > > >> > > > >> staging
> > > > >> > > > >>>>>> writes
> > > > >> > > > >>>>>>>> in
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> transactions, so can't perform writes
> > at the
> > > > >> > > > >>>>>> READ_COMMITTED
> > > > >> > > > >>>>>>>>>>>>>>>> isolation
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> level. However, this may be addressed
> > if we
> > > > >> > > > >> decide to
> > > > >> > > > >>>>>>>> *always*
> > > > >> > > > >>>>>>>>>>>>>>>> use
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> transactions as discussed under 3b.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> > > > >> > > > >>>> implementations. I
> > > > >> > > > >>>>>>>>>> remember
> > > > >> > > > >>>>>>>>>>>>>>>>>> having
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> some difficulty understanding the IQv1
> > > > >> internals,
> > > > >> > > > >>>> which
> > > > >> > > > >>>>>> made
> > > > >> > > > >>>>>>>> it
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> difficult
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> to determine what needed to be
> changed.
> > > > >> However,
> > > > >> > > > >> I
> > > > >> > > > >>>>>> *think*
> > > > >> > > > >>>>>>>> this
> > > > >> > > > >>>>>>>>>>>>>>>> can be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> addressed for both implementations by
> > > > wrapping
> > > > >> > > > >> the
> > > > >> > > > >>>>>>>> RocksDBStore
> > > > >> > > > >>>>>>>>>>>>>>>> in an
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> IsolationLevel-dependent wrapper, that
> > > > >> overrides
> > > > >> > > > >> read
> > > > >> > > > >>>>>> methods
> > > > >> > > > >>>>>>>>>>>>>>>> (get,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> etc.)
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> to either read directly from the
> > database or
> > > > >> > > > >> from the
> > > > >> > > > >>>>>> ongoing
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> But IQv1 might still be difficult.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> - If IsolationLevel becomes a query
> > > > >> constraint,
> > > > >> > > > >> then
> > > > >> > > > >>>> all
> > > > >> > > > >>>>>>>> other
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> StateStores
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> will need to respect it, including the
> > > > >> in-memory
> > > > >> > > > >>>> stores.
> > > > >> > > > >>>>>> This
> > > > >> > > > >>>>>>>>>>>>>>>> would
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> require
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> us to adapt in-memory stores to stage
> > their
> > > > >> > > > >> writes so
> > > > >> > > > >>>>>> they
> > > > >> > > > >>>>>>>> can
> > > > >> > > > >>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> isolated
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> from READ_COMMITTTED queries. It would
> > also
> > > > >> > > > >> become an
> > > > >> > > > >>>>>>>> important
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> consideration for third-party stores
> on
> > > > >> upgrade,
> > > > >> > > > >> as
> > > > >> > > > >>>>>> without
> > > > >> > > > >>>>>>>>>>>>>>>> changes,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> they
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> would not support READ_COMMITTED
> queries
> > > > >> > > > >> correctly.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Ultimately, I may need some help
> making
> > the
> > > > >> > > > >> necessary
> > > > >> > > > >>>>>> change
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> IQv1
> > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> support this, but I don't think it's
> > > > >> > > > >> fundamentally
> > > > >> > > > >>>>>>>> impossible,
> > > > >> > > > >>>>>>>>>>>>>>>> if we
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> want
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> to pursue this route.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> 3b.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> The main reason I chose to keep ALOS
> > > > >> > > > >> un-transactional
> > > > >> > > > >>>>>> was to
> > > > >> > > > >>>>>>>>>>>>>>>> minimize
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> behavioural change for most users (I
> > believe
> > > > >> most
> > > > >> > > > >>>> Streams
> > > > >> > > > >>>>>>>> users
> > > > >> > > > >>>>>>>>>>>>>>>> use
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> default configuration, which is ALOS).
> > That
> > > > >> said,
> > > > >> > > > >>>> it's
> > > > >> > > > >>>>>> clear
> > > > >> > > > >>>>>>>>>>>>>>>> that if
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> ALOS
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> also used transactional stores, the
> only
> > > > >> change
> > > > >> > > > >> in
> > > > >> > > > >>>>>> behaviour
> > > > >> > > > >>>>>>>>>>>>>>>> would be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> it would become *more correct*, which
> > could
> > > > be
> > > > >> > > > >>>>>> considered a
> > > > >> > > > >>>>>>>>>> "bug
> > > > >> > > > >>>>>>>>>>>>>>>> fix"
> > > > >> > > > >>>>>>>>>>>>>>>>>> by
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> users, rather than a change they need
> to
> > > > >> handle.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> I believe that performance using
> > > > transactions
> > > > >> > > > >> (aka.
> > > > >> > > > >>>>>> RocksDB
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> WriteBatches)
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> should actually be *better* than the
> > > > >> un-batched
> > > > >> > > > >>>>>> write-path
> > > > >> > > > >>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> currently used[1]. The only
> > "performance"
> > > > >> > > > >>>> consideration
> > > > >> > > > >>>>>> will
> > > > >> > > > >>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> increased memory usage that
> transactions
> > > > >> require.
> > > > >> > > > >>>> Given
> > > > >> > > > >>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>> mitigations
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> for
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> this memory that we have in place, I
> > would
> > > > >> expect
> > > > >> > > > >>>> that
> > > > >> > > > >>>>>> this
> > > > >> > > > >>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>> not a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> problem for most users.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> If we're happy to do so, we can make
> > ALOS
> > > > also
> > > > >> > > > >> use
> > > > >> > > > >>>>>>>>>> transactions.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> Link 1:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>
> > > > >> > > > >>
> > > > >> > >
> > > > >>
> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno
> > > > Cadonna <
> > > > >> > > > >>>>>>>>>>>> cadonna@apache.org
> > > > >> > > > >>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Thanks for the updates and sorry for
> > the
> > > > >> delay
> > > > >> > > > >> on my
> > > > >> > > > >>>>>> side!
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Making the default implementation for
> > > > >> flush() a
> > > > >> > > > >>>> no-op
> > > > >> > > > >>>>>> sounds
> > > > >> > > > >>>>>>>>>>>>>>>> good to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> me.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I think what was bugging me here is
> > that a
> > > > >> > > > >>>> third-party
> > > > >> > > > >>>>>> state
> > > > >> > > > >>>>>>>>>>>>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> needs
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> to implement the state store
> interface.
> > > > That
> > > > >> > > > >> means
> > > > >> > > > >>>> they
> > > > >> > > > >>>>>> need
> > > > >> > > > >>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> implement a wrapper around the actual
> > state
> > > > >> > > > >> store
> > > > >> > > > >>>> as we
> > > > >> > > > >>>>>> do
> > > > >> > > > >>>>>>>> for
> > > > >> > > > >>>>>>>>>>>>>>>>>> RocksDB
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> with RocksDBStore. So, a third-party
> > state
> > > > >> > > > >> store can
> > > > >> > > > >>>>>> always
> > > > >> > > > >>>>>>>>>>>>>>>> estimate
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> uncommitted bytes, if it wants,
> > because the
> > > > >> > > > >> wrapper
> > > > >> > > > >>>> can
> > > > >> > > > >>>>>>>> record
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> added
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> bytes.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> One case I can think of where
> > returning -1
> > > > >> makes
> > > > >> > > > >>>> sense
> > > > >> > > > >>>>>> is
> > > > >> > > > >>>>>>>> when
> > > > >> > > > >>>>>>>>>>>>>>>>>> Streams
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> does not need to estimate the size of
> > the
> > > > >> write
> > > > >> > > > >>>> batch
> > > > >> > > > >>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>> trigger
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> extraordinary commits, because the
> > > > >> third-party
> > > > >> > > > >> state
> > > > >> > > > >>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>> takes care
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memory. But in that case the method
> > could
> > > > >> also
> > > > >> > > > >> just
> > > > >> > > > >>>>>> return
> > > > >> > > > >>>>>>>> 0.
> > > > >> > > > >>>>>>>>>>>>>>>> Even
> > > > >> > > > >>>>>>>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> case would be better solved with a
> > method
> > > > >> that
> > > > >> > > > >>>> returns
> > > > >> > > > >>>>>>>> whether
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> store manages itself the memory used
> > for
> > > > >> > > > >> uncommitted
> > > > >> > > > >>>>>> bytes
> > > > >> > > > >>>>>>>> or
> > > > >> > > > >>>>>>>>>>>>>>>> not.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Said that, I am fine with keeping the
> > -1
> > > > >> return
> > > > >> > > > >>>> value,
> > > > >> > > > >>>>>> I was
> > > > >> > > > >>>>>>>>>>>>>>>> just
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> wondering when and if it will be
> used.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Regarding returning 0 for
> transactional
> > > > state
> > > > >> > > > >> stores
> > > > >> > > > >>>>>> when
> > > > >> > > > >>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> batch
> > > > >> > > > >>>>>>>>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> empty, I was just wondering because
> you
> > > > >> > > > >> explicitly
> > > > >> > > > >>>>>> stated
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> "or {@code 0} if this StateStore does
> > not
> > > > >> > > > >> support
> > > > >> > > > >>>>>>>>>>>> transactions."
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> So it seemed to me returning 0 could
> > only
> > > > >> > > > >> happen for
> > > > >> > > > >>>>>>>>>>>>>>>>>> non-transactional
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> state stores.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a) What do you think if we move the
> > > > isolation
> > > > >> > > > >> level
> > > > >> > > > >>>> to
> > > > >> > > > >>>>>> IQ
> > > > >> > > > >>>>>>>> (v1
> > > > >> > > > >>>>>>>>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>> v2)?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> In the end this is the only component
> > that
> > > > >> > > > >> really
> > > > >> > > > >>>> needs
> > > > >> > > > >>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> specify
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> isolation level. It is similar to the
> > Kafka
> > > > >> > > > >> consumer
> > > > >> > > > >>>>>> that
> > > > >> > > > >>>>>>>> can
> > > > >> > > > >>>>>>>>>>>>>>>> choose
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> with what isolation level to read the
> > input
> > > > >> > > > >> topic.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> For IQv1 the isolation level should
> go
> > into
> > > > >> > > > >>>>>>>>>>>>>>>> StoreQueryParameters. For
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> IQv2, I would add it to the Query
> > > > interface.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> b) Point a) raises the question what
> > should
> > > > >> > > > >> happen
> > > > >> > > > >>>>>> during
> > > > >> > > > >>>>>>>>>>>>>>>>>> at-least-once
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing when the state store does
> > not
> > > > use
> > > > >> > > > >>>>>> transactions?
> > > > >> > > > >>>>>>>>>> John
> > > > >> > > > >>>>>>>>>>>>>>>> in
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> past proposed to also use
> transactions
> > on
> > > > >> state
> > > > >> > > > >>>> stores
> > > > >> > > > >>>>>> for
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> at-least-once. I like that idea,
> > because it
> > > > >> > > > >> avoids
> > > > >> > > > >>>>>>>> aggregating
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>> same
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> records over and over again in the
> > case of
> > > > a
> > > > >> > > > >>>> failure. We
> > > > >> > > > >>>>>>>> had a
> > > > >> > > > >>>>>>>>>>>>>>>> case
> > > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the past where a Streams applications
> > in
> > > > >> > > > >>>> at-least-once
> > > > >> > > > >>>>>> mode
> > > > >> > > > >>>>>>>>>> was
> > > > >> > > > >>>>>>>>>>>>>>>>>> failing
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> continuously for some reasons I do
> not
> > > > >> remember
> > > > >> > > > >>>> before
> > > > >> > > > >>>>>>>>>>>>>>>> committing the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> offsets. After each failover, the app
> > > > >> aggregated
> > > > >> > > > >>>> again
> > > > >> > > > >>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>> again the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> same records. Of course the aggregate
> > > > >> increased
> > > > >> > > > >> to
> > > > >> > > > >>>> very
> > > > >> > > > >>>>>>>> wrong
> > > > >> > > > >>>>>>>>>>>>>>>> values
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> just because of the failover. With
> > > > >> transactions
> > > > >> > > > >> on
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>> state
> > > > >> > > > >>>>>>>>>>>>>>>> stores
> > > > >> > > > >>>>>>>>>>>>>>>>>> we
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> could have avoided this. The app
> would
> > have
> > > > >> > > > >> output
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>> same
> > > > >> > > > >>>>>>>>>>>>>>>> aggregate
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> multiple times (i.e., after each
> > failover)
> > > > >> but
> > > > >> > > > >> at
> > > > >> > > > >>>> least
> > > > >> > > > >>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> value of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> aggregate would not depend on the
> > number of
> > > > >> > > > >>>> failovers.
> > > > >> > > > >>>>>>>>>>>>>>>> Outputting the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> same aggregate multiple times would
> be
> > > > >> incorrect
> > > > >> > > > >>>> under
> > > > >> > > > >>>>>>>>>>>>>>>> exactly-once
> > > > >> > > > >>>>>>>>>>>>>>>>>> but
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> it is OK for at-least-once.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> If it makes sense to add a config to
> > turn
> > > > on
> > > > >> > > > >> and off
> > > > >> > > > >>>>>>>>>>>>>>>> transactions on
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> state stores under at-least-once or
> > just
> > > > use
> > > > >> > > > >>>>>> transactions in
> > > > >> > > > >>>>>>>>>>>>>>>> any case
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a question we should also discuss in
> > this
> > > > >> KIP.
> > > > >> > > > >> It
> > > > >> > > > >>>>>> depends a
> > > > >> > > > >>>>>>>>>> bit
> > > > >> > > > >>>>>>>>>>>>>>>> on
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> performance trade-off. Maybe to be
> > safe, I
> > > > >> would
> > > > >> > > > >>>> add a
> > > > >> > > > >>>>>>>> config.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 4.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Your points are all valid. I tend to
> > say to
> > > > >> > > > >> keep the
> > > > >> > > > >>>>>> metrics
> > > > >> > > > >>>>>>>>>>>>>>>> around
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() until we remove flush()
> > completely
> > > > >> from
> > > > >> > > > >> the
> > > > >> > > > >>>>>>>> interface.
> > > > >> > > > >>>>>>>>>>>>>>>> Calls
> > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() might still exist since
> > existing
> > > > >> > > > >> processors
> > > > >> > > > >>>>>> might
> > > > >> > > > >>>>>>>>>> still
> > > > >> > > > >>>>>>>>>>>>>>>> call
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() explicitly as you mentioned
> in
> > 1).
> > > > >> For
> > > > >> > > > >>>> sure, we
> > > > >> > > > >>>>>> need
> > > > >> > > > >>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> document
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> how the metrics change due to the
> > > > >> transactions
> > > > >> > > > >> in
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>>> upgrade
> > > > >> > > > >>>>>>>>>>>>>>>> notes.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 5.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I see. Then you should describe how
> the
> > > > >> > > > >> .position
> > > > >> > > > >>>> files
> > > > >> > > > >>>>>> are
> > > > >> > > > >>>>>>>>>>>>>>>> handled
> > > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a dedicated section of the KIP or
> > > > incorporate
> > > > >> > > > >> the
> > > > >> > > > >>>>>>>> description
> > > > >> > > > >>>>>>>>>>>>>>>> in the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> "Atomic Checkpointing" section
> instead
> > of
> > > > >> only
> > > > >> > > > >>>>>> mentioning it
> > > > >> > > > >>>>>>>>>> in
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> "Compatibility, 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:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>
> > > > >> > > > >>
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> >
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> It is a bit brittle, but it works.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Best,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford
> > wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for taking the time to review
> > the
> > > > >> KIP.
> > > > >> > > > >> I'm
> > > > >> > > > >>>> back
> > > > >> > > > >>>>>> from
> > > > >> > > > >>>>>>>>>>>>>>>> leave
> > > > >> > > > >>>>>>>>>>>>>>>>>> now
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> intend to move this forwards as
> > quickly
> > > > as I
> > > > >> > > > >> can.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Addressing your points:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Because flush() is part of the
> > StateStore
> > > > >> API,
> > > > >> > > > >> it's
> > > > >> > > > >>>>>> exposed
> > > > >> > > > >>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>> custom
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors, which might be making
> > calls to
> > > > >> > > > >> flush().
> > > > >> > > > >>>>>> This
> > > > >> > > > >>>>>>>> was
> > > > >> > > > >>>>>>>>>>>>>>>>>> actually
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> case in a few integration tests.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> To maintain as much compatibility as
> > > > >> possible,
> > > > >> > > > >> I'd
> > > > >> > > > >>>>>> prefer
> > > > >> > > > >>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>> make
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> this
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> an UnsupportedOperationException, as
> > it
> > > > will
> > > > >> > > > >> cause
> > > > >> > > > >>>>>>>> previously
> > > > >> > > > >>>>>>>>>>>>>>>>>> working
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors to start throwing
> > exceptions at
> > > > >> > > > >> runtime.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that it doesn't make sense
> > for it
> > > > to
> > > > >> > > > >> proxy
> > > > >> > > > >>>>>>>> commit(),
> > > > >> > > > >>>>>>>>>>>>>>>> though,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> as
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> that would cause it to violate the
> > > > >> "StateStores
> > > > >> > > > >>>> commit
> > > > >> > > > >>>>>> only
> > > > >> > > > >>>>>>>>>>>>>>>> when the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> Task
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commits" rule.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Instead, I think we should make
> this a
> > > > >> no-op.
> > > > >> > > > >> That
> > > > >> > > > >>>> way,
> > > > >> > > > >>>>>>>>>>>>>>>> existing
> > > > >> > > > >>>>>>>>>>>>>>>>>> user
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors will continue to work
> > > > as-before,
> > > > >> > > > >> without
> > > > >> > > > >>>>>>>> violation
> > > > >> > > > >>>>>>>>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> consistency that would be caused by
> > > > >> premature
> > > > >> > > > >>>>>> flush/commit
> > > > >> > > > >>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> StateStore
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> data to disk.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> What do you think?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> As stated in the JavaDoc, when a
> > > > StateStore
> > > > >> > > > >>>>>> implementation
> > > > >> > > > >>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, but is unable to
> > estimate
> > > > the
> > > > >> > > > >>>>>> uncommitted
> > > > >> > > > >>>>>>>>>>>> memory
> > > > >> > > > >>>>>>>>>>>>>>>>>> usage,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> method will return -1.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> The intention here is to permit
> > > > third-party
> > > > >> > > > >>>>>> implementations
> > > > >> > > > >>>>>>>>>>>>>>>> that may
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> able to estimate memory usage.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has
> > been
> > > > >> > > > >> written to
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>> yet. I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> thought
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> that was implied by "This method
> will
> > > > >> return an
> > > > >> > > > >>>>>>>> approximation
> > > > >> > > > >>>>>>>>>>>>>>>> of the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memory
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> would be freed by the next call to
> > {@link
> > > > >> > > > >>>>>> #commit(Map)}"
> > > > >> > > > >>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>> "@return
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> The
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> approximate size of all records
> > awaiting
> > > > >> {@link
> > > > >> > > > >>>>>>>>>>>> #commit(Map)}",
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> however,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> can add it explicitly to the JavaDoc
> > if
> > > > you
> > > > >> > > > >> think
> > > > >> > > > >>>> this
> > > > >> > > > >>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>> unclear?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I realise this is probably the most
> > > > >> contentious
> > > > >> > > > >>>> point
> > > > >> > > > >>>>>> in my
> > > > >> > > > >>>>>>>>>>>>>>>> design,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I'm
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> open to changing it if I'm unable to
> > > > >> convince
> > > > >> > > > >> you
> > > > >> > > > >>>> of
> > > > >> > > > >>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> benefits.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s)
> are
> > > > >> directly
> > > > >> > > > >>>> provided
> > > > >> > > > >>>>>>>>>>>>>>>> StateStores
> > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> query, and it may be important for
> > users
> > > > to
> > > > >> > > > >>>>>>>> programmatically
> > > > >> > > > >>>>>>>>>>>>>>>> know
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> which
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> mode the StateStore is operating
> > under. If
> > > > >> we
> > > > >> > > > >>>> simply
> > > > >> > > > >>>>>>>> provide
> > > > >> > > > >>>>>>>>>>>> an
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "eosEnabled" boolean (as used
> > throughout
> > > > the
> > > > >> > > > >>>> internal
> > > > >> > > > >>>>>>>> streams
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> engine), or
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> similar, then users will need to
> > > > understand
> > > > >> the
> > > > >> > > > >>>>>> operation
> > > > >> > > > >>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> consequences
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> of each available processing mode
> and
> > how
> > > > it
> > > > >> > > > >>>> pertains
> > > > >> > > > >>>>>> to
> > > > >> > > > >>>>>>>>>> their
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Interactive Query users aren't the
> > only
> > > > >> people
> > > > >> > > > >> that
> > > > >> > > > >>>>>> care
> > > > >> > > > >>>>>>>>>> about
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> processing.mode/IsolationLevel of a
> > > > >> StateStore:
> > > > >> > > > >>>>>>>> implementers
> > > > >> > > > >>>>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>>> custom
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStores also need to understand
> > the
> > > > >> > > > >> behaviour
> > > > >> > > > >>>>>> expected
> > > > >> > > > >>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>> their
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> implementation. KIP-892 introduces
> > some
> > > > >> > > > >> assumptions
> > > > >> > > > >>>>>> into
> > > > >> > > > >>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> Streams
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Engine
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> about how StateStores operate under
> > each
> > > > >> > > > >> processing
> > > > >> > > > >>>>>> mode,
> > > > >> > > > >>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>> it's
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> important that custom
> implementations
> > > > >> adhere to
> > > > >> > > > >>>> those
> > > > >> > > > >>>>>>>>>>>>>>>> assumptions in
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> order
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> to maintain the consistency
> > guarantees.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> IsolationLevels provide a high-level
> > > > >> contract
> > > > >> > > > >> on
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>>>>> behaviour
> > > > >> > > > >>>>>>>>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStore: a user knows that under
> > > > >> > > > >> READ_COMMITTED,
> > > > >> > > > >>>>>> they
> > > > >> > > > >>>>>>>> will
> > > > >> > > > >>>>>>>>>>>>>>>> see
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> writes
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> only after the Task has committed,
> and
> > > > under
> > > > >> > > > >>>>>>>> READ_UNCOMMITTED
> > > > >> > > > >>>>>>>>>>>>>>>> they
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> will
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> see
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> writes immediately. No understanding
> > of
> > > > the
> > > > >> > > > >>>> details of
> > > > >> > > > >>>>>> each
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing.mode
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> is required, either for IQ users or
> > > > >> StateStore
> > > > >> > > > >>>>>>>> implementers.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> An argument can be made that these
> > > > >> contractual
> > > > >> > > > >>>>>> guarantees
> > > > >> > > > >>>>>>>> can
> > > > >> > > > >>>>>>>>>>>>>>>> simply
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> documented for the processing.mode
> > (i.e.
> > > > >> that
> > > > >> > > > >>>>>> exactly-once
> > > > >> > > > >>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> exactly-once-v2 behave like
> > READ_COMMITTED
> > > > >> and
> > > > >> > > > >>>>>>>> at-least-once
> > > > >> > > > >>>>>>>>>>>>>>>> behaves
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> like
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are
> > several
> > > > >> small
> > > > >> > > > >>>> issues
> > > > >> > > > >>>>>> with
> > > > >> > > > >>>>>>>>>>>>>>>> this I'd
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> prefer
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> to avoid:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - Where would we document
> > these
> > > > >> > > > >> contracts,
> > > > >> > > > >>>> in
> > > > >> > > > >>>>>> a way
> > > > >> > > > >>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> difficult
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          for users/implementers to
> > > > >> miss/ignore?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It's not clear to users
> > that
> > > > the
> > > > >> > > > >>>> processing
> > > > >> > > > >>>>>> mode
> > > > >> > > > >>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> communicating
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          an expectation of read
> > isolation,
> > > > >> > > > >> unless
> > > > >> > > > >>>> they
> > > > >> > > > >>>>>> read
> > > > >> > > > >>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> documentation. Users
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          rarely consult
> documentation
> > > > unless
> > > > >> > > > >> they
> > > > >> > > > >>>> feel
> > > > >> > > > >>>>>> they
> > > > >> > > > >>>>>>>>>> need
> > > > >> > > > >>>>>>>>>>>>>>>> to, so
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> it's
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> likely
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          this detail would get
> missed
> > by
> > > > >> many
> > > > >> > > > >> users.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It tightly couples
> > processing
> > > > >> modes
> > > > >> > > > >> to
> > > > >> > > > >>>> read
> > > > >> > > > >>>>>>>>>> isolation.
> > > > >> > > > >>>>>>>>>>>>>>>> Adding
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> new
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          processing modes, or
> > changing the
> > > > >> read
> > > > >> > > > >>>>>> isolation of
> > > > >> > > > >>>>>>>>>>>>>>>> existing
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          modes would be
> > > > >> difficult/impossible.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Ultimately, the cost of introducing
> > > > >> > > > >>>> IsolationLevels is
> > > > >> > > > >>>>>>>> just a
> > > > >> > > > >>>>>>>>>>>>>>>> single
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> method, since we re-use the existing
> > > > >> > > > >> IsolationLevel
> > > > >> > > > >>>>>> enum
> > > > >> > > > >>>>>>>> from
> > > > >> > > > >>>>>>>>>>>>>>>> Kafka.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> This
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> gives us a clear place to document
> the
> > > > >> > > > >> contractual
> > > > >> > > > >>>>>>>> guarantees
> > > > >> > > > >>>>>>>>>>>>>>>>>> expected
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> of/provided by StateStores, that is
> > > > >> accessible
> > > > >> > > > >>>> both by
> > > > >> > > > >>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>> StateStore
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> itself, and by IQ users.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> (Writing this I've just realised
> that
> > the
> > > > >> > > > >>>> StateStore
> > > > >> > > > >>>>>> and IQ
> > > > >> > > > >>>>>>>>>>>>>>>> APIs
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> actually
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> don't provide access to
> > StateStoreContext
> > > > >> that
> > > > >> > > > >> IQ
> > > > >> > > > >>>> users
> > > > >> > > > >>>>>>>> would
> > > > >> > > > >>>>>>>>>>>>>>>> have
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> direct
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> access to... Perhaps StateStore
> should
> > > > >> expose
> > > > >> > > > >>>>>>>>>> isolationLevel()
> > > > >> > > > >>>>>>>>>>>>>>>>>> itself
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> too?)
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 4.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming
> the
> > > > >> metrics
> > > > >> > > > >>>> in-place
> > > > >> > > > >>>>>>>>>>>>>>>> either, as
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> it's a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> backwards incompatible change. My
> > concern
> > > > is
> > > > >> > > > >> that,
> > > > >> > > > >>>> if
> > > > >> > > > >>>>>> we
> > > > >> > > > >>>>>>>>>> leave
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> existing
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics in place, they will
> be
> > > > >> > > > >> confusing to
> > > > >> > > > >>>>>> users.
> > > > >> > > > >>>>>>>>>>>>>>>> Right
> > > > >> > > > >>>>>>>>>>>>>>>>>> now,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics record explicit
> > flushes to
> > > > >> > > > >> disk,
> > > > >> > > > >>>> but
> > > > >> > > > >>>>>> under
> > > > >> > > > >>>>>>>>>>>>>>>> KIP-892,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> even
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() will not explicitly flush
> > data to
> > > > >> > > > >> disk -
> > > > >> > > > >>>>>> RocksDB
> > > > >> > > > >>>>>>>>>> will
> > > > >> > > > >>>>>>>>>>>>>>>>>> decide
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> on
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> when to flush memtables to disk
> > itself.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> If we keep the existing "flush"
> > metrics,
> > > > >> we'd
> > > > >> > > > >> have
> > > > >> > > > >>>> two
> > > > >> > > > >>>>>>>>>>>> options,
> > > > >> > > > >>>>>>>>>>>>>>>>>> which
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> both
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> seem pretty bad to me:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          1. Have them record calls
> to
> > > > >> commit(),
> > > > >> > > > >>>> which
> > > > >> > > > >>>>>> would
> > > > >> > > > >>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> misleading, as
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          data is no longer
> explicitly
> > > > >> "flushed"
> > > > >> > > > >> to
> > > > >> > > > >>>> disk
> > > > >> > > > >>>>>> by
> > > > >> > > > >>>>>>>> this
> > > > >> > > > >>>>>>>>>>>>>>>> call.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          2. Have them record nothing
> > at
> > > > all,
> > > > >> > > > >> which
> > > > >> > > > >>>> is
> > > > >> > > > >>>>>>>>>> equivalent
> > > > >> > > > >>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> removing
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          metrics, except that users
> > will
> > > > see
> > > > >> > the
> > > > >> > > > >>>> metric
> > > > >> > > > >>>>>>>> still
> > > > >> > > > >>>>>>>>>>>>>>>> exists and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> so
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> assume
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          that the metric is correct,
> > and
> > > > >> that
> > > > >> > > > >>>> there's a
> > > > >> > > > >>>>>>>> problem
> > > > >> > > > >>>>>>>>>>>>>>>> with
> > > > >> > > > >>>>>>>>>>>>>>>>>> their
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> system
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          when there isn't.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that removing them is also a
> > bad
> > > > >> > > > >> solution,
> > > > >> > > > >>>> and
> > > > >> > > > >>>>>> I'd
> > > > >> > > > >>>>>>>>>>>>>>>> like some
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> guidance on the best path forward
> > here.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 5.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Position files are updated on every
> > write
> > > > >> to a
> > > > >> > > > >>>>>> StateStore.
> > > > >> > > > >>>>>>>>>>>>>>>> Since our
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> writes
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> are now buffered until commit(), we
> > can't
> > > > >> > > > >> update
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>>> Position
> > > > >> > > > >>>>>>>>>>>>>>>> file
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> until
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() has been called, otherwise
> it
> > > > >> would be
> > > > >> > > > >>>>>>>> inconsistent
> > > > >> > > > >>>>>>>>>>>>>>>> with
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> data
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> in the event of a rollback.
> > Consequently,
> > > > we
> > > > >> > > > >> need
> > > > >> > > > >>>> to
> > > > >> > > > >>>>>> manage
> > > > >> > > > >>>>>>>>>>>>>>>> these
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> offsets
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> the same way we manage the
> checkpoint
> > > > >> offsets,
> > > > >> > > > >> and
> > > > >> > > > >>>>>> ensure
> > > > >> > > > >>>>>>>>>>>>>>>> they're
> > > > >> > > > >>>>>>>>>>>>>>>>>> only
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> written on commit().
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 6.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Agreed, although I'm not exactly
> sure
> > yet
> > > > >> what
> > > > >> > > > >>>> tests to
> > > > >> > > > >>>>>>>>>> write.
> > > > >> > > > >>>>>>>>>>>>>>>> How
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> explicit
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> do we need to be here in the KIP?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is
> > > > >> designed
> > > > >> > > > >> to be
> > > > >> > > > >>>>>>>> seamless,
> > > > >> > > > >>>>>>>>>>>>>>>> and we
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> should
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> definitely add some tests around
> that.
> > > > >> > > > >> Downgrade,
> > > > >> > > > >>>> it
> > > > >> > > > >>>>>>>>>>>>>>>> transpires,
> > > > >> > > > >>>>>>>>>>>>>>>>>> isn't
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> currently possible, as the extra
> > column
> > > > >> family
> > > > >> > > > >> for
> > > > >> > > > >>>>>> offset
> > > > >> > > > >>>>>>>>>>>>>>>> storage is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> incompatible with the pre-KIP-892
> > > > >> > > > >> implementation:
> > > > >> > > > >>>> when
> > > > >> > > > >>>>>> you
> > > > >> > > > >>>>>>>>>>>>>>>> open a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> database, you must open all
> available
> > > > column
> > > > >> > > > >>>> families
> > > > >> > > > >>>>>> or
> > > > >> > > > >>>>>>>>>>>>>>>> receive an
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> error.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> What currently happens on downgrade
> is
> > > > that
> > > > >> it
> > > > >> > > > >>>>>> attempts to
> > > > >> > > > >>>>>>>>>>>>>>>> open the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> store,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> throws an error about the offsets
> > column
> > > > >> > > > >> family not
> > > > >> > > > >>>>>> being
> > > > >> > > > >>>>>>>>>>>>>>>> opened,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> which
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> triggers a wipe and rebuild of the
> > Task.
> > > > >> Given
> > > > >> > > > >> that
> > > > >> > > > >>>>>>>>>> downgrades
> > > > >> > > > >>>>>>>>>>>>>>>>>> should
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> uncommon, I think this is acceptable
> > > > >> > > > >> behaviour, as
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> end-state is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> consistent, even if it results in an
> > > > >> > > > >> undesirable
> > > > >> > > > >>>> state
> > > > >> > > > >>>>>>>>>>>> restore.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Should I document the
> > upgrade/downgrade
> > > > >> > > > >> behaviour
> > > > >> > > > >>>>>>>> explicitly
> > > > >> > > > >>>>>>>>>>>>>>>> in the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> KIP?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> --
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno
> > > > Cadonna
> > > > >> <
> > > > >> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Nick!
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates!
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Why does StateStore#flush() default
> > to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > StateStore#commit(Collections.emptyMap())?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Since calls to flush() will not
> exist
> > > > >> anymore
> > > > >> > > > >>>> after
> > > > >> > > > >>>>>> this
> > > > >> > > > >>>>>>>> KIP
> > > > >> > > > >>>>>>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> released, I would rather throw an
> > > > >> unsupported
> > > > >> > > > >>>>>> operation
> > > > >> > > > >>>>>>>>>>>>>>>> exception
> > > > >> > > > >>>>>>>>>>>>>>>>>> by
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> default.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> When would a state store return -1
> > from
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > StateStore#approximateNumUncommittedBytes()
> > > > >> > > > >> while
> > > > >> > > > >>>>>> being
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> transactional?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Wouldn't
> > > > >> > > > >>>> StateStore#approximateNumUncommittedBytes()
> > > > >> > > > >>>>>> also
> > > > >> > > > >>>>>>>>>>>>>>>> return 0
> > > > >> > > > >>>>>>>>>>>>>>>>>> if
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the state store is transactional
> but
> > > > >> nothing
> > > > >> > > > >> has
> > > > >> > > > >>>> been
> > > > >> > > > >>>>>>>>>> written
> > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> state store yet?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Sorry for bringing this up again.
> > Does
> > > > this
> > > > >> > > > >> KIP
> > > > >> > > > >>>> really
> > > > >> > > > >>>>>>>> need
> > > > >> > > > >>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> introduce
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> > > > >> > > > >>>> StateStoreContext
> > > > >> > > > >>>>>> has
> > > > >> > > > >>>>>>>>>>>>>>>> already
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> appConfigs() which basically
> exposes
> > the
> > > > >> same
> > > > >> > > > >>>>>> information,
> > > > >> > > > >>>>>>>>>>>>>>>> i.e., if
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> EOS
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> is enabled or not.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> In one of your previous e-mails you
> > > > wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> "My idea was to try to keep the
> > > > StateStore
> > > > >> > > > >>>> interface
> > > > >> > > > >>>>>> as
> > > > >> > > > >>>>>>>>>>>>>>>> loosely
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> coupled
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as
> possible,
> > to
> > > > >> give
> > > > >> > > > >>>>>> implementers
> > > > >> > > > >>>>>>>>>>>> more
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> freedom,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and reduce the amount of internal
> > > > knowledge
> > > > >> > > > >>>> required."
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> While I understand the intent, I
> > doubt
> > > > >> that it
> > > > >> > > > >>>>>> decreases
> > > > >> > > > >>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> coupling of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> a StateStore interface and the
> > Streams
> > > > >> engine.
> > > > >> > > > >>>>>>>>>> READ_COMMITTED
> > > > >> > > > >>>>>>>>>>>>>>>> only
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> applies to IQ but not to reads by
> > > > >> processors.
> > > > >> > > > >>>> Thus,
> > > > >> > > > >>>>>>>>>>>>>>>> implementers
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> need to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand how Streams accesses the
> > state
> > > > >> > > > >> stores.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> I would like to hear what others
> > think
> > > > >> about
> > > > >> > > > >> this.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 4.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Great exposing new metrics for
> > > > >> transactional
> > > > >> > > > >> state
> > > > >> > > > >>>>>> stores!
> > > > >> > > > >>>>>>>>>>>>>>>>>> However, I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> would prefer to add new metrics and
> > > > >> deprecate
> > > > >> > > > >> (in
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>>>> docs)
> > > > >> > > > >>>>>>>>>>>>>>>> the 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 also need to
> > adapt/add
> > > > >> > > > >> integration
> > > > >> > > > >>>>>> tests
> > > > >> > > > >>>>>>>>>>>>>>>> besides
> > > > >> > > > >>>>>>>>>>>>>>>>>> unit
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> tests. Additionally, we probably
> need
> > > > >> > > > >> integration
> > > > >> > > > >>>> or
> > > > >> > > > >>>>>>>> system
> > > > >> > > > >>>>>>>>>>>>>>>> tests
> > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> verify that upgrades and downgrades
> > > > between
> > > > >> > > > >>>>>> transactional
> > > > >> > > > >>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> non-transactional state stores work
> > as
> > > > >> > > > >> expected.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford
> > wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> One more thing: I noted John's
> > > > suggestion
> > > > >> in
> > > > >> > > > >> the
> > > > >> > > > >>>> KIP,
> > > > >> > > > >>>>>>>> under
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> "Rejected
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> Alternatives". I still think it's
> an
> > > > idea
> > > > >> > > > >> worth
> > > > >> > > > >>>>>> pursuing,
> > > > >> > > > >>>>>>>>>>>>>>>> but I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> believe
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> that it's out of the scope of this
> > KIP,
> > > > >> > > > >> because
> > > > >> > > > >>>> it
> > > > >> > > > >>>>>>>> solves a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> different
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> set
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> of problems to this KIP, and the
> > scope
> > > > of
> > > > >> > > > >> this
> > > > >> > > > >>>> one
> > > > >> > > > >>>>>> has
> > > > >> > > > >>>>>>>>>>>>>>>> already
> > > > >> > > > >>>>>>>>>>>>>>>>>> grown
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> quite
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> large!
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick
> > > > >> Telford <
> > > > >> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP (
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>
> > > > >> > > > >>
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> )
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with the latest changes; mostly
> > > > bringing
> > > > >> > > > >> back
> > > > >> > > > >>>>>> "Atomic
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> Checkpointing"
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> (for
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> what feels like the 10th time!).
> I
> > > > think
> > > > >> > > > >> the one
> > > > >> > > > >>>>>> thing
> > > > >> > > > >>>>>>>>>>>>>>>> missing is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> some
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> changes to metrics (notably the
> > store
> > > > >> > > > >> "flush"
> > > > >> > > > >>>>>> metrics
> > > > >> > > > >>>>>>>> will
> > > > >> > > > >>>>>>>>>>>>>>>> need
> > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> renamed to "commit").
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> The reason I brought back Atomic
> > > > >> > > > >> Checkpointing
> > > > >> > > > >>>> was
> > > > >> > > > >>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> decouple
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flush
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from store commit. This is
> > important,
> > > > >> > > > >> because
> > > > >> > > > >>>> with
> > > > >> > > > >>>>>>>>>>>>>>>> Transactional
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> StateStores, we now need to call
> > > > "flush"
> > > > >> on
> > > > >> > > > >>>> *every*
> > > > >> > > > >>>>>> Task
> > > > >> > > > >>>>>>>>>>>>>>>> commit,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> just when the StateStore is
> > closing,
> > > > >> > > > >> otherwise
> > > > >> > > > >>>> our
> > > > >> > > > >>>>>>>>>>>>>>>> transaction
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> buffer
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> will
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> never be written and persisted,
> > instead
> > > > >> > > > >> growing
> > > > >> > > > >>>>>>>> unbounded!
> > > > >> > > > >>>>>>>>>>>> I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> experimented
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with some simple solutions, like
> > > > forcing
> > > > >> a
> > > > >> > > > >> store
> > > > >> > > > >>>>>> flush
> > > > >> > > > >>>>>>>>>>>>>>>> whenever
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer was likely to
> > exceed
> > > > >> its
> > > > >> > > > >>>>>> configured
> > > > >> > > > >>>>>>>>>>>>>>>> size, but
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> this
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> brittle: it prevented the
> > transaction
> > > > >> buffer
> > > > >> > > > >>>> from
> > > > >> > > > >>>>>> being
> > > > >> > > > >>>>>>>>>>>>>>>>>> configured
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> unbounded, and it still would
> have
> > > > >> required
> > > > >> > > > >>>> explicit
> > > > >> > > > >>>>>>>>>>>>>>>> flushes of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> RocksDB,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yielding sub-optimal performance
> > and
> > > > >> memory
> > > > >> > > > >>>>>> utilization.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to
> > be the
> > > > >> > > > >> "right"
> > > > >> > > > >>>> way
> > > > >> > > > >>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> resolve
> > > > >> > > > >>>>>>>>>>>>>>>>>> this
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> problem. By ensuring that the
> > changelog
> > > > >> > > > >> offsets
> > > > >> > > > >>>> that
> > > > >> > > > >>>>>>>>>>>>>>>> correspond
> > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> most
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> recently written records are
> always
> > > > >> > > > >> atomically
> > > > >> > > > >>>>>> written
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (by writing them to the same
> > > > transaction
> > > > >> > > > >>>> buffer),
> > > > >> > > > >>>>>> we can
> > > > >> > > > >>>>>>>>>>>>>>>> avoid
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> forcibly
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to
> > disk,
> > > > >> > > > >> letting
> > > > >> > > > >>>>>> RocksDB
> > > > >> > > > >>>>>>>>>>>>>>>> flush
> > > > >> > > > >>>>>>>>>>>>>>>>>> them
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> only
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when necessary, without losing
> any
> > of
> > > > our
> > > > >> > > > >>>>>> consistency
> > > > >> > > > >>>>>>>>>>>>>>>> guarantees.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> See
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> updated KIP for more info.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I have fully implemented these
> > changes,
> > > > >> > > > >>>> although I'm
> > > > >> > > > >>>>>>>> still
> > > > >> > > > >>>>>>>>>>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> entirely
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> happy with the implementation for
> > > > >> segmented
> > > > >> > > > >>>>>> StateStores,
> > > > >> > > > >>>>>>>>>> so
> > > > >> > > > >>>>>>>>>>>>>>>> I
> > > > >> > > > >>>>>>>>>>>>>>>>>> plan
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> refactor that. Despite that, all
> > tests
> > > > >> > > > >> pass. If
> > > > >> > > > >>>>>> you'd
> > > > >> > > > >>>>>>>> like
> > > > >> > > > >>>>>>>>>>>>>>>> to try
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> out
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> or
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> review this highly experimental
> and
> > > > >> > > > >> incomplete
> > > > >> > > > >>>>>> branch,
> > > > >> > > > >>>>>>>>>> it's
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> available
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> here:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>
> > https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > > > >> > > > >>>>>>>> .
> > > > >> > > > >>>>>>>>>>>>>>>> Note:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> it's
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> built
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I
> had a
> > > > >> stable
> > > > >> > > > >> base
> > > > >> > > > >>>> to
> > > > >> > > > >>>>>> build
> > > > >> > > > >>>>>>>>>>>>>>>> and test
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> it
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> on,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and to enable easy
> apples-to-apples
> > > > >> > > > >> comparisons
> > > > >> > > > >>>> in a
> > > > >> > > > >>>>>>>> live
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> environment. I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> plan to rebase it against trunk
> > once
> > > > it's
> > > > >> > > > >> nearer
> > > > >> > > > >>>>>>>>>> completion
> > > > >> > > > >>>>>>>>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>> has
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> been
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> proven on our main application.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I would really appreciate help in
> > > > >> reviewing
> > > > >> > > > >> and
> > > > >> > > > >>>>>> testing:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session
> and
> > > > >> Window)
> > > > >> > > > >>>> stores
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Global stores
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> As I do not currently use either
> of
> > > > >> these,
> > > > >> > > > >> so my
> > > > >> > > > >>>>>> primary
> > > > >> > > > >>>>>>>>>>>>>>>> test
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> environment
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> doesn't test these areas.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I'm going on Parental Leave
> > starting
> > > > next
> > > > >> > > > >> week
> > > > >> > > > >>>> for
> > > > >> > > > >>>>>> a few
> > > > >> > > > >>>>>>>>>>>>>>>> weeks,
> > > > >> > > > >>>>>>>>>>>>>>>>>> so
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> will
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> not have time to move this
> forward
> > > > until
> > > > >> > > > >> late
> > > > >> > > > >>>>>> August.
> > > > >> > > > >>>>>>>> That
> > > > >> > > > >>>>>>>>>>>>>>>> said,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> your
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> feedback is welcome and
> > appreciated, I
> > > > >> just
> > > > >> > > > >>>> won't be
> > > > >> > > > >>>>>>>> able
> > > > >> > > > >>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> respond
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> as
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> quickly as usual.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick
> > > > >> Telford <
> > > > >> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, that's correct, although
> the
> > > > impact
> > > > >> > > > >> on IQ
> > > > >> > > > >>>> is
> > > > >> > > > >>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>> something
> > > > >> > > > >>>>>>>>>>>>>>>>>> I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> had
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> considered.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating
> the
> > > > state
> > > > >> > > > >> store
> > > > >> > > > >>>>>> from the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval
> and
> > > > >> writing
> > > > >> > > > >> the
> > > > >> > > > >>>>>>>> checkpoint
> > > > >> > > > >>>>>>>>>>>>>>>> (thus,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured
> > amount
> > > > >> of
> > > > >> > > > >> data
> > > > >> > > > >>>>>> and/or
> > > > >> > > > >>>>>>>>>>>>>>>> number of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are
> > you
> > > > >> > > > >> suggesting
> > > > >> > > > >>>>>> that we
> > > > >> > > > >>>>>>>>>>>>>>>> add an
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> additional
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> config for the max number of
> > commit
> > > > >> > > > >> intervals
> > > > >> > > > >>>>>> between
> > > > >> > > > >>>>>>>>>>>>>>>>>> checkpoints?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> That
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> way, we would checkpoint
> *either*
> > when
> > > > >> the
> > > > >> > > > >>>>>> transaction
> > > > >> > > > >>>>>>>>>>>>>>>> buffers
> > > > >> > > > >>>>>>>>>>>>>>>>>> are
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> nearly
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain
> > number
> > > > of
> > > > >> > > > >> commit
> > > > >> > > > >>>>>>>> intervals
> > > > >> > > > >>>>>>>>>>>>>>>> have
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> elapsed,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> whichever comes first?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> That certainly seems reasonable,
> > > > >> although
> > > > >> > > > >> this
> > > > >> > > > >>>>>>>> re-ignites
> > > > >> > > > >>>>>>>>>>>>>>>> an
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> earlier
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> debate about whether a config
> > should
> > > > be
> > > > >> > > > >>>> measured in
> > > > >> > > > >>>>>>>>>>>>>>>> "number of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> commit
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> intervals", instead of just an
> > > > absolute
> > > > >> > > > >> time.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue
> > is
> > > > the
> > > > >> > > > >> reason
> > > > >> > > > >>>> I
> > > > >> > > > >>>>>> was
> > > > >> > > > >>>>>>>>>>>>>>>> pursuing
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Atomic
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples
> > memtable
> > > > >> > > > >> flush
> > > > >> > > > >>>> from
> > > > >> > > > >>>>>>>>>>>>>>>> checkpointing,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> which
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> enables us to just checkpoint on
> > every
> > > > >> > > > >> commit
> > > > >> > > > >>>>>> without
> > > > >> > > > >>>>>>>> any
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> performance
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is
> > > > >> definitely
> > > > >> > > > >> the
> > > > >> > > > >>>>>> "best"
> > > > >> > > > >>>>>>>>>>>>>>>> solution,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> but
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm not
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sure if this is enough to bring
> it
> > > > back
> > > > >> > > > >> into
> > > > >> > > > >>>> this
> > > > >> > > > >>>>>> KIP.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm currently working on moving
> > all
> > > > the
> > > > >> > > > >>>>>> transactional
> > > > >> > > > >>>>>>>>>>>> logic
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> directly
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> into
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does
> > away
> > > > >> with
> > > > >> > > > >> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore#newTransaction
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> method, and reduces the number
> of
> > new
> > > > >> > > > >> classes
> > > > >> > > > >>>>>>>> introduced,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> significantly
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reducing the complexity. If it
> > works,
> > > > >> and
> > > > >> > > > >> the
> > > > >> > > > >>>>>>>> complexity
> > > > >> > > > >>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> drastically
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reduced, I may try bringing back
> > > > Atomic
> > > > >> > > > >>>> Checkpoints
> > > > >> > > > >>>>>>>> into
> > > > >> > > > >>>>>>>>>>>>>>>> this
> > > > >> > > > >>>>>>>>>>>>>>>>>> KIP.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27,
> Bruno
> > > > >> Cadonna
> > > > >> > > > >> <
> > > > >> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very
> > > > >> interesting!
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> As far as I understand, you
> want
> > to
> > > > >> > > > >> atomically
> > > > >> > > > >>>>>> update
> > > > >> > > > >>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> state
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> from the transaction buffer,
> > flush
> > > > the
> > > > >> > > > >>>> memtable
> > > > >> > > > >>>>>> of a
> > > > >> > > > >>>>>>>>>>>> state
> > > > >> > > > >>>>>>>>>>>>>>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write the checkpoint not after
> > the
> > > > >> commit
> > > > >> > > > >> time
> > > > >> > > > >>>>>> elapsed
> > > > >> > > > >>>>>>>>>>>> but
> > > > >> > > > >>>>>>>>>>>>>>>>>> after
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reached a
> size
> > > > that
> > > > >> > > > >> would
> > > > >> > > > >>>> lead
> > > > >> > > > >>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> exceeding
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > statestore.transaction.buffer.max.bytes
> > > > >> > > > >>>> before the
> > > > >> > > > >>>>>>>> next
> > > > >> > > > >>>>>>>>>>>>>>>> commit
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> interval
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> ends.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> That means, the Kafka
> transaction
> > > > would
> > > > >> > > > >> commit
> > > > >> > > > >>>>>> every
> > > > >> > > > >>>>>>>>>>>>>>>> commit
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> interval
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store will only be
> > > > atomically
> > > > >> > > > >>>> updated
> > > > >> > > > >>>>>>>> roughly
> > > > >> > > > >>>>>>>>>>>>>>>> every
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> statestore.transaction.buffer.max.bytes of
> > > > >> > > > >>>> data.
> > > > >> > > > >>>>>> Also
> > > > >> > > > >>>>>>>> IQ
> > > > >> > > > >>>>>>>>>>>>>>>> would
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> then
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> only
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see new data roughly every
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> statestore.transaction.buffer.max.bytes.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> After a failure the state store
> > needs
> > > > >> to
> > > > >> > > > >>>> restore
> > > > >> > > > >>>>>> up to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> statestore.transaction.buffer.max.bytes.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Is this correct?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating
> > the
> > > > >> state
> > > > >> > > > >> store
> > > > >> > > > >>>>>> from
> > > > >> > > > >>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval
> and
> > > > >> writing
> > > > >> > > > >> the
> > > > >> > > > >>>>>>>> checkpoint
> > > > >> > > > >>>>>>>>>>>>>>>> (thus,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured
> > amount
> > > > >> of
> > > > >> > > > >> data
> > > > >> > > > >>>>>> and/or
> > > > >> > > > >>>>>>>>>>>>>>>> number of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we
> > would
> > > > have
> > > > >> > > > >> the
> > > > >> > > > >>>> same
> > > > >> > > > >>>>>> delay
> > > > >> > > > >>>>>>>>>>>> for
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> records
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> appearing in output topics and
> IQ
> > > > >> because
> > > > >> > > > >> both
> > > > >> > > > >>>>>> would
> > > > >> > > > >>>>>>>>>>>>>>>> appear
> > > > >> > > > >>>>>>>>>>>>>>>>>> when
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka transaction is committed.
> > > > >> However,
> > > > >> > > > >>>> after a
> > > > >> > > > >>>>>>>> failure
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it might restore data that is
> > already
> > > > >> in
> > > > >> > > > >> the
> > > > >> > > > >>>> state
> > > > >> > > > >>>>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>>>> because
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last
> > > > stable
> > > > >> > > > >> offset
> > > > >> > > > >>>>>> (i.e.
> > > > >> > > > >>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> last
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> committed
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset) of the changelog
> topics.
> > > > >> Restoring
> > > > >> > > > >>>> data
> > > > >> > > > >>>>>> that
> > > > >> > > > >>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>> already
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> in
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> state store is idempotent, so
> eos
> > > > >> should
> > > > >> > > > >> not
> > > > >> > > > >>>>>> violated.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> This solution needs at least
> one
> > new
> > > > >> > > > >> config to
> > > > >> > > > >>>>>> specify
> > > > >> > > > >>>>>>>>>>>>>>>> when a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> should be written.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> A small correction to your
> > previous
> > > > >> e-mail
> > > > >> > > > >>>> that
> > > > >> > > > >>>>>> does
> > > > >> > > > >>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>> change
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> anything
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> you said: Under alos the
> default
> > > > commit
> > > > >> > > > >>>> interval
> > > > >> > > > >>>>>> is 30
> > > > >> > > > >>>>>>>>>>>>>>>> seconds,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> five
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> seconds.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford
> > > > wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've begun performance testing
> > my
> > > > >> branch
> > > > >> > > > >> on
> > > > >> > > > >>>> our
> > > > >> > > > >>>>>>>> staging
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> environment,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting it through its paces
> in
> > our
> > > > >> > > > >>>> non-trivial
> > > > >> > > > >>>>>>>>>>>>>>>> application.
> > > > >> > > > >>>>>>>>>>>>>>>>>> I'm
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing the same increased
> > flush
> > > > >> rate
> > > > >> > > > >> that
> > > > >> > > > >>>> we
> > > > >> > > > >>>>>> saw
> > > > >> > > > >>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> last
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> time
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> attempted to use a version of
> > this
> > > > >> KIP,
> > > > >> > > > >> but
> > > > >> > > > >>>> this
> > > > >> > > > >>>>>>>> time,
> > > > >> > > > >>>>>>>>>> I
> > > > >> > > > >>>>>>>>>>>>>>>>>> think I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> know
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> why.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892,
> > StreamTask#postCommit,
> > > > >> > > > >> which is
> > > > >> > > > >>>>>> called
> > > > >> > > > >>>>>>>> at
> > > > >> > > > >>>>>>>>>>>>>>>> the end
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Task
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit process, has the
> > following
> > > > >> > > > >> behaviour:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under ALOS:
> > checkpoint
> > > > >> the
> > > > >> > > > >> state
> > > > >> > > > >>>>>> stores.
> > > > >> > > > >>>>>>>>>> This
> > > > >> > > > >>>>>>>>>>>>>>>>>> includes
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            flushing memtables
> in
> > > > >> RocksDB.
> > > > >> > > > >>>> This is
> > > > >> > > > >>>>>>>>>>>> acceptable
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> because the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> default
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit.interval.ms
> > is 5
> > > > >> > > > >> seconds,
> > > > >> > > > >>>> so
> > > > >> > > > >>>>>>>> forcibly
> > > > >> > > > >>>>>>>>>>>>>>>> flushing
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> every 5
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            seconds is
> > acceptable for
> > > > >> most
> > > > >> > > > >>>>>>>> applications.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under EOS:
> > > > checkpointing
> > > > >> is
> > > > >> > > > >> not
> > > > >> > > > >>>> done,
> > > > >> > > > >>>>>>>>>> *unless*
> > > > >> > > > >>>>>>>>>>>>>>>> it's
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> being
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> forced, due
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to e.g. the Task
> > closing
> > > > or
> > > > >> > > > >> being
> > > > >> > > > >>>>>> revoked.
> > > > >> > > > >>>>>>>>>> This
> > > > >> > > > >>>>>>>>>>>>>>>> means
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> under
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> normal
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing
> > conditions,
> > > > the
> > > > >> > > > >> state
> > > > >> > > > >>>> stores
> > > > >> > > > >>>>>>>> will
> > > > >> > > > >>>>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpointed,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            not have memtables
> > > > flushed
> > > > >> at
> > > > >> > > > >> all ,
> > > > >> > > > >>>>>> unless
> > > > >> > > > >>>>>>>>>>>> RocksDB
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> decides to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush them on
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            its own.
> > Checkpointing
> > > > >> stores
> > > > >> > > > >> and
> > > > >> > > > >>>>>>>>>> force-flushing
> > > > >> > > > >>>>>>>>>>>>>>>> their
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            done when a Task is
> > being
> > > > >> > > > >> closed.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to
> > > > checkpoint
> > > > >> > > > >>>> stores on
> > > > >> > > > >>>>>> at
> > > > >> > > > >>>>>>>>>>>> least
> > > > >> > > > >>>>>>>>>>>>>>>>>> *some*
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> normal
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task commits, in order to
> write
> > the
> > > > >> > > > >> RocksDB
> > > > >> > > > >>>>>>>> transaction
> > > > >> > > > >>>>>>>>>>>>>>>>>> buffers
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure
> the
> > > > >> offsets
> > > > >> > > > >> are
> > > > >> > > > >>>>>> synced to
> > > > >> > > > >>>>>>>>>>>>>>>> disk to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> prevent
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> restores from getting out of
> > hand.
> > > > >> > > > >>>> Consequently,
> > > > >> > > > >>>>>> my
> > > > >> > > > >>>>>>>>>>>>>>>> current
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on
> *every*
> > > > Task
> > > > >> > > > >> commit,
> > > > >> > > > >>>>>> which
> > > > >> > > > >>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>> far too
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> frequent.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This causes checkpoints every
> > 10,000
> > > > >> > > > >> records,
> > > > >> > > > >>>>>> which
> > > > >> > > > >>>>>>>> is
> > > > >> > > > >>>>>>>>>> a
> > > > >> > > > >>>>>>>>>>>>>>>>>> change
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> in
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
> > > > >> > > > >> performance
> > > > >> > > > >>>>>> problems
> > > > >> > > > >>>>>>>> for
> > > > >> > > > >>>>>>>>>>>>>>>> some
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> applications.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm looking into possible
> > solutions,
> > > > >> and
> > > > >> > > > >> I'm
> > > > >> > > > >>>>>>>> currently
> > > > >> > > > >>>>>>>>>>>>>>>> leaning
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> towards
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> using the
> > > > >> > > > >>>> statestore.transaction.buffer.max.bytes
> > > > >> > > > >>>>>>>>>>>>>>>>>> configuration
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are
> > likely
> > > > to
> > > > >> > > > >>>> exceed it.
> > > > >> > > > >>>>>>>> This
> > > > >> > > > >>>>>>>>>>>>>>>> would
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> complement the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
> > > > >> > > > >> functionality
> > > > >> > > > >>>> that
> > > > >> > > > >>>>>> this
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> configuration
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provides, in the following
> way:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Currently, we use
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> force an
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            early Task commit
> if
> > > > >> > processing
> > > > >> > > > >>>> more
> > > > >> > > > >>>>>>>> records
> > > > >> > > > >>>>>>>>>>>> would
> > > > >> > > > >>>>>>>>>>>>>>>>>> cause
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> our
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> state
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            transactions to
> > exceed
> > > > the
> > > > >> > > > >> memory
> > > > >> > > > >>>>>> assigned
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> them.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - New
> functionality:
> > > > when a
> > > > >> > > > >> Task
> > > > >> > > > >>>> *does*
> > > > >> > > > >>>>>>>>>> commit,
> > > > >> > > > >>>>>>>>>>>>>>>> we will
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the stores (and
> hence
> > > > flush
> > > > >> > the
> > > > >> > > > >>>>>> transaction
> > > > >> > > > >>>>>>>>>>>>>>>> buffers)
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> unless
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> expect to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            cross the
> > > > >> > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> threshold
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the next
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06,
> > Nick
> > > > >> > > > >> Telford <
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> By "less predictable for
> > users", I
> > > > >> > > > >> meant in
> > > > >> > > > >>>>>> terms of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> understanding
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance profile under
> > various
> > > > >> > > > >>>>>> circumstances. The
> > > > >> > > > >>>>>>>>>>>>>>>> more
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> complex
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, the more difficult
> it
> > > > would
> > > > >> > > > >> be for
> > > > >> > > > >>>>>> users
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> understand
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance they see. For
> > example,
> > > > >> > > > >> spilling
> > > > >> > > > >>>>>> records
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> disk
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> when
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a
> > > > >> threshold
> > > > >> > > > >>>> would, I
> > > > >> > > > >>>>>>>>>> expect,
> > > > >> > > > >>>>>>>>>>>>>>>>>> reduce
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> write
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in
> > write
> > > > >> > > > >>>> throughput
> > > > >> > > > >>>>>> could
> > > > >> > > > >>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> unexpected,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> > > > >> > > > >>>> diagnose/understand for
> > > > >> > > > >>>>>>>>>> users.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the
> > "early
> > > > >> > > > >> commit"
> > > > >> > > > >>>>>> concept is
> > > > >> > > > >>>>>>>>>>>>>>>>>> relatively
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to
> > > > >> document,
> > > > >> > > > >> and
> > > > >> > > > >>>>>>>>>>>> conceptually
> > > > >> > > > >>>>>>>>>>>>>>>>>> fairly
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> obvious to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users. We could probably add
> a
> > > > >> metric to
> > > > >> > > > >>>> make it
> > > > >> > > > >>>>>>>>>> easier
> > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it happens though.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is
> > *essentially*
> > > > >> an
> > > > >> > > > >>>> indirect
> > > > >> > > > >>>>>> way
> > > > >> > > > >>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>>> telling
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether they should be
> > > > transactional.
> > > > >> > > > >>>>>> READ_COMMITTED
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> essentially
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> requires
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, because it
> > dictates
> > > > >> that
> > > > >> > > > >> two
> > > > >> > > > >>>>>> threads
> > > > >> > > > >>>>>>>>>>>>>>>> calling
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not
> > see
> > > > >> writes
> > > > >> > > > >>>> from
> > > > >> > > > >>>>>> the
> > > > >> > > > >>>>>>>>>> other
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> until
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they have been committed.
> With
> > > > >> > > > >>>>>> READ_UNCOMMITTED, all
> > > > >> > > > >>>>>>>>>>>>>>>> bets are
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> off,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to
> > observe
> > > > >> > > > >> written
> > > > >> > > > >>>>>> records
> > > > >> > > > >>>>>>>> at
> > > > >> > > > >>>>>>>>>>>>>>>> any
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> time,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> which is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially "no
> transactions".
> > That
> > > > >> > > > >> said,
> > > > >> > > > >>>>>>>> StateStores
> > > > >> > > > >>>>>>>>>>>>>>>> are
> > > > >> > > > >>>>>>>>>>>>>>>>>> free
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implement
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these guarantees however they
> > can,
> > > > >> > > > >> which is
> > > > >> > > > >>>> a
> > > > >> > > > >>>>>> bit
> > > > >> > > > >>>>>>>> more
> > > > >> > > > >>>>>>>>>>>>>>>>>> relaxed
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> than
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dictating "you must use
> > > > >> transactions".
> > > > >> > > > >> For
> > > > >> > > > >>>>>> example,
> > > > >> > > > >>>>>>>>>>>> with
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement these as
> > READ_COMMITTED
> > > > ==
> > > > >> > > > >>>> WBWI-based
> > > > >> > > > >>>>>>>>>>>>>>>>>> "transactions",
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct
> > writes
> > > > to
> > > > >> the
> > > > >> > > > >>>>>> database.
> > > > >> > > > >>>>>>>> But
> > > > >> > > > >>>>>>>>>>>>>>>> with
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> other
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> storage
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> engines, it might be
> > preferable to
> > > > >> > > > >> *always*
> > > > >> > > > >>>> use
> > > > >> > > > >>>>>>>>>>>>>>>> transactions,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> even
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be
> > > > storage
> > > > >> > > > >> engines
> > > > >> > > > >>>>>> that
> > > > >> > > > >>>>>>>>>> don't
> > > > >> > > > >>>>>>>>>>>>>>>>>> provide
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, but the
> isolation
> > > > >> > > > >> guarantees
> > > > >> > > > >>>> can
> > > > >> > > > >>>>>> be
> > > > >> > > > >>>>>>>> met
> > > > >> > > > >>>>>>>>>>>>>>>> using a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> different
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> technique.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep
> the
> > > > >> > > > >> StateStore
> > > > >> > > > >>>>>> interface
> > > > >> > > > >>>>>>>> as
> > > > >> > > > >>>>>>>>>>>>>>>>>> loosely
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> coupled
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as
> > > > possible,
> > > > >> to
> > > > >> > > > >> give
> > > > >> > > > >>>>>>>>>>>>>>>> implementers
> > > > >> > > > >>>>>>>>>>>>>>>>>> more
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> freedom, and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal
> > > > >> knowledge
> > > > >> > > > >>>>>> required.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> > > > >> > > > >>>> "IsolationLevel"
> > > > >> > > > >>>>>> might
> > > > >> > > > >>>>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> right
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can
> always
> > make
> > > > >> it
> > > > >> > > > >> much
> > > > >> > > > >>>> more
> > > > >> > > > >>>>>>>>>>>>>>>> explicit if
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> required, e.g.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7-8.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I can make these changes
> either
> > > > later
> > > > >> > > > >> today
> > > > >> > > > >>>> or
> > > > >> > > > >>>>>>>>>>>> tomorrow.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Small update:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on
> > trunk and
> > > > >> > > > >> fixed a
> > > > >> > > > >>>>>> bunch of
> > > > >> > > > >>>>>>>>>>>>>>>> issues
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all
> the
> > > > tests
> > > > >> > > > >> pass,
> > > > >> > > > >>>>>> which is
> > > > >> > > > >>>>>>>>>>>>>>>>>> promising,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> but
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> it
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to undergo some
> > performance
> > > > >> > > > >> testing. I
> > > > >> > > > >>>>>> haven't
> > > > >> > > > >>>>>>>>>>>>>>>> (yet)
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> worked
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> on
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> removing the
> `newTransaction()`
> > > > >> stuff,
> > > > >> > > > >> but I
> > > > >> > > > >>>>>> would
> > > > >> > > > >>>>>>>>>>>>>>>> expect
> > > > >> > > > >>>>>>>>>>>>>>>>>> that,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make
> > no
> > > > >> > > > >>>> difference. The
> > > > >> > > > >>>>>>>>>> branch
> > > > >> > > > >>>>>>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> available
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> at
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> > > > >> > > > >>>>>>>>>> if
> > > > >> > > > >>>>>>>>>>>>>>>>>> anyone
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interested in taking an early
> > look.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59,
> > Bruno
> > > > >> > > > >> Cadonna
> > > > >> > > > >>>> <
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> cadonna@apache.org>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That
> > was
> > > > >> > > > >> actually
> > > > >> > > > >>>> also
> > > > >> > > > >>>>>> my
> > > > >> > > > >>>>>>>>>>>>>>>> point. I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understood
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that John was proposing the
> > > > >> ingestion
> > > > >> > > > >> path
> > > > >> > > > >>>> as
> > > > >> > > > >>>>>> a way
> > > > >> > > > >>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> avoid
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> early
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I
> > > > misinterpreted
> > > > >> the
> > > > >> > > > >>>> intent.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that
> > > > >> actually
> > > > >> > > > >> it is
> > > > >> > > > >>>>>> public
> > > > >> > > > >>>>>>>>>>>>>>>> API. My
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> question
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern
> affects
> > > > >> normal
> > > > >> > > > >>>>>> processing.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that checking
> > for
> > > > the
> > > > >> > > > >> size
> > > > >> > > > >>>> of the
> > > > >> > > > >>>>>>>>>>>>>>>> transaction
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> buffer
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early
> > commit
> > > > >> > > > >> affects
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>> whole
> > > > >> > > > >>>>>>>>>>>>>>>>>> processing
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams. The
> transactionality
> > of a
> > > > >> > > > >> state
> > > > >> > > > >>>> store
> > > > >> > > > >>>>>> is
> > > > >> > > > >>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> confined to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store itself, but
> spills
> > > > over
> > > > >> and
> > > > >> > > > >>>>>> changes the
> > > > >> > > > >>>>>>>>>>>>>>>> behavior
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree
> > with
> > > > >> you
> > > > >> > > > >> that
> > > > >> > > > >>>> it
> > > > >> > > > >>>>>> is a
> > > > >> > > > >>>>>>>>>>>>>>>> decent
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compromise. I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the
> > > > downsides
> > > > >> > > > >> and
> > > > >> > > > >>>> list
> > > > >> > > > >>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> options to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> overcome
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you
> > that
> > > > all
> > > > >> > > > >>>> options
> > > > >> > > > >>>>>> seem
> > > > >> > > > >>>>>>>>>>>> quite
> > > > >> > > > >>>>>>>>>>>>>>>>>> heavy
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compared
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not
> > understand
> > > > >> > > > >> what you
> > > > >> > > > >>>>>> mean
> > > > >> > > > >>>>>>>> with
> > > > >> > > > >>>>>>>>>>>>>>>> "less
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> predictable
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I found the discussions
> about
> > the
> > > > >> > > > >>>> alternatives
> > > > >> > > > >>>>>>>> really
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> interesting.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> But I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also think that your plan
> > sounds
> > > > >> good
> > > > >> > > > >> and
> > > > >> > > > >>>> we
> > > > >> > > > >>>>>> should
> > > > >> > > > >>>>>>>>>>>>>>>> continue
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> with
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> it!
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply
> > to my
> > > > >> > > > >> e-mail on
> > > > >> > > > >>>>>> June
> > > > >> > > > >>>>>>>>>> 20th:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the
> > > > reasoning
> > > > >> > > > >> behind
> > > > >> > > > >>>>>> putting
> > > > >> > > > >>>>>>>>>>>>>>>> isolation
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> level
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store context.
> > Thanks!
> > > > >> Should
> > > > >> > > > >>>> that
> > > > >> > > > >>>>>> also
> > > > >> > > > >>>>>>>> be
> > > > >> > > > >>>>>>>>>> a
> > > > >> > > > >>>>>>>>>>>>>>>> way
> > > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> give
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store the
> > opportunity to
> > > > >> > > > >> decide
> > > > >> > > > >>>>>> whether
> > > > >> > > > >>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> turn on
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more
> > > > >> concerned
> > > > >> > > > >> about
> > > > >> > > > >>>>>> how do
> > > > >> > > > >>>>>>>>>> you
> > > > >> > > > >>>>>>>>>>>>>>>> know
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> if a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be
> > > > written
> > > > >> > > > >> under
> > > > >> > > > >>>> EOS,
> > > > >> > > > >>>>>> if
> > > > >> > > > >>>>>>>> you
> > > > >> > > > >>>>>>>>>>>>>>>> do not
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> have a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to know if the state store
> is
> > > > >> > > > >>>> transactional or
> > > > >> > > > >>>>>> not.
> > > > >> > > > >>>>>>>>>> If
> > > > >> > > > >>>>>>>>>>>>>>>> a
> > > > >> > > > >>>>>>>>>>>>>>>>>> state
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional, the
> checkpoint
> > file
> > > > >> can
> > > > >> > > > >> be
> > > > >> > > > >>>>>> written
> > > > >> > > > >>>>>>>>>>>>>>>> during
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> normal
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the
> > state
> > > > >> > > > >> store
> > > > >> > > > >>>> is not
> > > > >> > > > >>>>>>>>>>>>>>>>>> transactional,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be
> > > > written
> > > > >> > > > >> under
> > > > >> > > > >>>> EOS.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My point was about not only
> > > > >> > > > >> considering the
> > > > >> > > > >>>>>> bytes
> > > > >> > > > >>>>>>>> in
> > > > >> > > > >>>>>>>>>>>>>>>> memory
> > > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> config
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > statestore.uncommitted.max.bytes,
> > > > >> but
> > > > >> > > > >> also
> > > > >> > > > >>>>>> bytes
> > > > >> > > > >>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>> might
> > > > >> > > > >>>>>>>>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> spilled
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was
> > > > wondering
> > > > >> > > > >>>> whether you
> > > > >> > > > >>>>>>>>>> should
> > > > >> > > > >>>>>>>>>>>>>>>>>> remove
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number
> of
> > > > >> memory
> > > > >> > > > >>>> bytes to
> > > > >> > > > >>>>>> be
> > > > >> > > > >>>>>>>>>> used
> > > > >> > > > >>>>>>>>>>>>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted
> state-store
> > > > >> > > > >> records." My
> > > > >> > > > >>>>>>>> thinking
> > > > >> > > > >>>>>>>>>>>>>>>> was
> > > > >> > > > >>>>>>>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> even
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store spills
> uncommitted
> > > > >> bytes to
> > > > >> > > > >>>> disk,
> > > > >> > > > >>>>>>>>>> limiting
> > > > >> > > > >>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> overall
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> bytes
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking
> > about
> > > > it
> > > > >> > > > >> again
> > > > >> > > > >>>> and
> > > > >> > > > >>>>>>>>>>>>>>>> considering
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> recent
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not
> make
> > too
> > > > >> much
> > > > >> > > > >>>> sense
> > > > >> > > > >>>>>>>> anymore.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I like the name
> > > > >> > > > >>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> that
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> you
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 8.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A high-level description
> > (without
> > > > >> > > > >>>>>> implementation
> > > > >> > > > >>>>>>>>>>>>>>>> details) of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> how
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the
> > commit of
> > > > >> > > > >> changelog
> > > > >> > > > >>>>>>>>>>>>>>>> transactions,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions and
> checkpointing
> > > > >> would be
> > > > >> > > > >>>> great.
> > > > >> > > > >>>>>>>> Would
> > > > >> > > > >>>>>>>>>>>> be
> > > > >> > > > >>>>>>>>>>>>>>>>>> great
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> if
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> you
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could also add some
> sentences
> > > > about
> > > > >> the
> > > > >> > > > >>>>>> behavior in
> > > > >> > > > >>>>>>>>>>>>>>>> case of
> > > > >> > > > >>>>>>>>>>>>>>>>>> a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> failure.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For instance how does a
> > > > >> transactional
> > > > >> > > > >> state
> > > > >> > > > >>>>>> store
> > > > >> > > > >>>>>>>>>>>>>>>> recover
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> after a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure or what happens with
> > the
> > > > >> > > > >>>> transaction
> > > > >> > > > >>>>>>>> buffer,
> > > > >> > > > >>>>>>>>>>>>>>>> etc.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> (that
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> what
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in
> > point
> > > > 9.)
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick
> > Telford
> > > > >> wrote:
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same
> > issue
> > > > >> that
> > > > >> > > > >>>>>>>>>>>>>>>> WriteBatchWithIndex
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or
> > > > likely
> > > > >> to
> > > > >> > > > >>>> exceed)
> > > > >> > > > >>>>>>>>>>>>>>>> configured
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> memory
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needs to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big
> > concerns.
> > > > >> > > > >>>> Ultimately,
> > > > >> > > > >>>>>> any
> > > > >> > > > >>>>>>>>>>>>>>>> approach
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> based
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> on
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cracking
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and
> > using
> > > > >> it in
> > > > >> > > > >>>> ways
> > > > >> > > > >>>>>> it's
> > > > >> > > > >>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>> really
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> designed
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> likely to have some
> unforseen
> > > > >> > > > >> performance
> > > > >> > > > >>>> or
> > > > >> > > > >>>>>>>>>>>>>>>> consistency
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> issues.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for
> > > > removing
> > > > >> > > > >> these
> > > > >> > > > >>>>>> early
> > > > >> > > > >>>>>>>>>>>>>>>> commits?
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> While
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> not
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent
> > compromise
> > > > >> to
> > > > >> > > > >>>> ensure
> > > > >> > > > >>>>>>>>>>>>>>>> consistency
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> whilst
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maintaining
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good and predictable
> > performance.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested
> ideas
> > > > seem
> > > > >> > > > >> *very*
> > > > >> > > > >>>>>>>>>>>>>>>> complicated, and
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> might
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less
> > predictable
> > > > for
> > > > >> > > > >> users
> > > > >> > > > >>>> as a
> > > > >> > > > >>>>>>>>>>>>>>>> consequence.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that
> the
> > > > scope
> > > > >> of
> > > > >> > > > >> this
> > > > >> > > > >>>>>> KIP is
> > > > >> > > > >>>>>>>>>>>>>>>> growing a
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> bit
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> out
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to
> > > > discuss
> > > > >> > > > >> ideas
> > > > >> > > > >>>> for
> > > > >> > > > >>>>>>>> future
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> improvements, I
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > >> > > > >>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow
> the
> > > > scope
> > > > >> > > > >> down
> > > > >> > > > >>>> to a
> > > > >> > > > >>>>>>>> design
> > > > >> > > > >>>>>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> achieves
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> most
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives
> (constant
> > > > sized
> > > > >> > > > >>>>>> restorations
> > > > >> > > > >>>>>>>>>>>>>>>> during
> > > > >> > > > >>>>>>>>>>>>>>>>>> dirty
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors).
> Any
> > > > >> design
> > > > >> > > > >> that
> > > > >> > > > >>>>>> this KIP
> > > > >> > > > >>>>>>>>>>>>>>>> produces
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> can
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ultimately
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future,
> > > > >> especially
> > > > >> > > > >> if
> > > > >> > > > >>>> the
> > > > >> > > > >>>>>> bulk
> > > > >> > > > >>>>>>>> of
> > > > >> > > > >>>>>>>>>>>>>>>> it is
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> internal
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some
> time
> > next
> > > > >> week
> > > > >> > > > >>>> trying
> > > > >> > > > >>>>>> to
> > > > >> > > > >>>>>>>>>>>>>>>> re-work
> > > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design
> to
> > > > >> remove
> > > > >> > > > >> the
> > > > >> > > > >>>>>>>>>>>>>>>> newTransaction()
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>> method,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> such
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation
> > > > detail
> > > > >> of
> > > > >> > > > >>>>>>>> RocksDBStore.
> > > > >> > > > >>>>>>>>>>>>>>>> That
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> way, if
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something
> > in
> > > > the
> > > > >> > > > >> future,
> > > > >> > > > >>>>>> like
> > > > >> > > > >>>>>>>> the
> > > > >> > > > >>>>>>>>>>>>>>>> SST
> > > > >> > > > >>>>>>>>>>>>>>>>>> file
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> management
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we
> > can do
> > > > so
> > > > >> > > > >> with
> > > > >> > > > >>>>>> little/no
> > > > >> > > > >>>>>>>>>>>> API
> > > > >> > > > >>>>>>>>>>>>>>>>>>>> changes.
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>>
> > > > >> > > > >>>>>>>>>>
> > > > >> > > > >>>>>>>>>
> > > > >> > > > >>>>>>>>
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>>>
> > > > >> > > > >>>>
> > > > >> > > > >>
> > > > >> > > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > >
> >
>

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

Posted by Sophie Blee-Goldman <so...@responsive.dev>.
Yeah I had missed the internal Streams configs when I first looked
and was going to suggest the "enable.x.y" format based on the client
configs until I noticed we had internal configs of the form x.y.enabled

But I fully agree that it's silly to align ourselves with internal config
names
and we should just follow the established pattern in public configs.

So yeah, I'm on board with enable.transactional.stores

...I think that's the last open question then?

On Mon, Apr 15, 2024 at 5:32 AM Lucas Brutschy
<lb...@confluent.io.invalid> wrote:

> Hi Nick,
>
> the java clients do have some properties that are of the form
> `enable....` (`enable.auto.comit` and `enable.idempotance`), so I
> would go with this form. State updater and processing thread configs
> use the `....enabled` form, but they are internal and could be changed
> to the `enable...` form for consistency. But, not intending to start a
> big discussion here. Both options are fine with me.
>
> Thanks for the updates!
>
>
> On Sun, Apr 14, 2024 at 10:58 PM Sophie Blee-Goldman
> <so...@responsive.dev> wrote:
> >
> > Makes sense to me! Regarding the new config name -- doesn't seem like
> > we have any *public *configs in StreamsConfig that set a standard yet, so
> > imo we're free to pick whatever we think sounds good.
> >
> > However we do have a few internal configs that follow the pattern
> > *feature.name.enabled* so if we want to play it safe and adhere to the
> > established pattern, we should call it
> "transactional.statestores.enabled"
> >
> > I also personally think x.y.enabled sounds better than enable.x.y, but
> > honestly I don't feel too strongly either way. I don't think it's worth
> > prolonging
> > the discussion any further over
> >
> > I'll take a look at KIP-892 as soon as I can since it's a prereq for this
> > one.
> > Thanks for the updates!
> >
> >
> > On Thu, Apr 11, 2024 at 12:02 PM Nick Telford <ni...@gmail.com>
> > wrote:
> >
> > > Hi everyone,
> > >
> > > After various off-list discussions, it's become clear there are still
> some
> > > contentious points to this KIP.
> > >
> > > 1.
> > > We agreed to change the "default.state.isolation.level" config
> property to
> > > be a transactional state store feature toggle. What should we name
> this?
> > > "enable.transactional.statestores"? What's the convention for this
> kind of
> > > flag?
> > >
> > > 2.
> > > The "atomic checkpointing" section has been broken out into KIP-1035
> (with
> > > one significant change, and otherwise just more explanation). This is
> now a
> > > hard dependency of KIP-892.
> > >
> > > Regards,
> > > Nick
> > >
> > > On Mon, 6 Nov 2023 at 17:00, Nick Telford <ni...@gmail.com>
> wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > Sorry for the delay.
> > > >
> > > > I've updated the KIP based on our discussion. I've also added some
> > > details
> > > > on "Transactional support under READ_UNCOMMITTED" and  "Query-time
> > > > Isolation Levels" in the "Rejected Alternatives" section, to
> highlight
> > > the
> > > > potential to improve things in a future KIP.
> > > >
> > > > If there are no more requested changes or objections, I'll kick off
> the
> > > > vote thread tomorrow, approximately 24 hours from now.
> > > >
> > > > While I'd love for this KIP to make it into 3.7.0, I can't commit to
> it
> > > > being code-complete by the December 20th deadline. While the branch I
> > > have
> > > > is broadly working, there are areas that will likely require
> improvement,
> > > > and others that are untested:
> > > >
> > > > - Untested outside of unit tests: Windowed stores, Versioned stores,
> > > > Global stores, IQv2. They should all work as expected, but the
> Position
> > > > file logic might be a bit janky.
> > > > - Requires improvement: Position data handling. Very clunky right
> now.
> > > > - Requires implementation: The "feature flag", where we disable
> > > > transactionality under EOS+READ_UNCOMMITTED. I've started this, but
> it's
> > > > far from simple. Plus this will require fleshing out the test suite
> > > quite a
> > > > bit.
> > > >
> > > > For the curious, my active development branch has been rebased
> against
> > > > 3.6.0: https://github.com/nicktelford/kafka/tree/KIP-892-3.6.0
> > > >
> > > > Regards,
> > > > Nick
> > > >
> > > > On Mon, 30 Oct 2023 at 23:32, Sophie Blee-Goldman <
> sophie@responsive.dev
> > > >
> > > > wrote:
> > > >
> > > >> Hey Nick, sounds like things are moving along here. I think you're
> > > already
> > > >> aware, but just as a reminder to us all, the KIP freeze deadline
> for 3.7
> > > >> is
> > > >> approaching and is currently set for Nov 18. I think we can all
> agree
> > > that
> > > >> it would be great to have this KIP accepted by then, and it sounds
> like
> > > >> much of the implementation is already completed, in which case this
> > > might
> > > >> be able to make it into 3.7.
> > > >>
> > > >> It sounds like this discussion is wrapping up so I just want to
> > > summarize
> > > >> my understanding of the current plan regarding configs since it
> doesn't
> > > >> seem like the KIP has been updated to include this yet.
> > > >>
> > > >> Basically we're all in agreement to go with option #1 that you
> presented
> > > >> earlier, right? Assuming so, I just want to clear up some details
> around
> > > >> the default behavior. What happens if:
> > > >> 1. User sets EOS and sets READ_UNCOMMITTED: txn state stores will be
> > > >> disabled/feature-flagged
> > > >> 2. User sets EOS and does not set anything for the isolation level:
> > > >> although the default is READ_UNCOMMITTED, enabling EOS will change
> the
> > > >> default to READ_COMMITTED and txn state stores will be used
> > > >>
> > > >> As for the future, it sounds like when READ_UNCOMMITTED mode is
> > > >> implemented, we will basically just remove this "feature flag" and
> txn
> > > >> state stores will always be used for all EOS cases. EOS users will
> be
> > > able
> > > >> to configure the isolation level independently, although it will
> still
> > > >> default to READ_COMMITTED when EOS is enabled and it wasn't
> explicitly
> > > >> configured.
> > > >>
> > > >> Is this understanding correct? I think this was the right choice
> > > >> regardless, as it will give people a way to disable the txn stores
> in an
> > > >> emergency -- as a project we went a long time with little pressure
> to
> > > >> feature flag things, and our users paid the cost for that. Even if
> we
> > > >> managed to ship something without bugs, it was often only after an
> > > intense
> > > >> period of successive blocker bugs that delayed the entire released
> for
> > > >> weeks. Other times, major bugs slipped through and some versions
> became
> > > >> unusable for certain use cases. So having some way to disable the
> txn
> > > >> state
> > > >> stores when EOS is used feels like a good strategy, since you just
> never
> > > >> know what might get through all the testing we do.
> > > >>
> > > >> If this ends up slipping to 4.0 and you manage to implement the
> > > >> READ_UNCOMMITTED mode within the same release, I think it's worth
> > > >> considering to add in an additional feature flag, even if it's just
> a
> > > >> backdoor internal config (eg as we did in KIP-441 with the internal
> task
> > > >> assignor config). But for now let's assume the first release this
> KIP
> > > >> appears in will have the behavior as described above, with
> > > >> READ_UNCOMMITTED
> > > >> mode acting as a feature flag
> > > >>
> > > >> Nick -- please let us know when you've updated the KIP to clarify
> the
> > > >> config behavior, and are ready for a vote!
> > > >>
> > > >>
> > > >> On Sun, Oct 29, 2023 at 12:02 PM Colt McNealy <co...@littlehorse.io>
> > > >> wrote:
> > > >>
> > > >> > Guozhang—I agree, I am in favor of moving forward with the KIP now
> > > that
> > > >> the
> > > >> > Transactional State Stores will be behind a feature flag.
> > > >> >
> > > >> > Nick—I just did a bit more light testing of your branch
> > > `KIP-892-3.5.0`
> > > >> > with your most recent changes. I couldn't detect a performance
> > > >> difference
> > > >> > versus trunk (in the past there was a slight degradation of
> > > performance
> > > >> on
> > > >> > the restoration path, but that has been fixed). I don't believe
> that
> > > >> your
> > > >> > branch has the state updater thread enabled, so I didn't test that
> > > path
> > > >> too
> > > >> > heavily.
> > > >> >
> > > >> > As expected, however, our internal correctness tests failed due
> to the
> > > >> IQ
> > > >> > read-your-own-writes issue we discussed previously. The community
> as a
> > > >> > whole would vastly benefit from this KIP getting over the finish
> line
> > > in
> > > >> > 3.7.0, and so long as it is behind a feature flag so that we at
> > > >> LittleHorse
> > > >> > can still guarantee RYOW for our users, I think it's purely a win
> 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 <
> > > >> guozhang.wang.us@gmail.com
> > > >> > >
> > > >> > wrote:
> > > >> >
> > > >> > > I'd agree with you guys that as long as we are in agreement
> about
> > > the
> > > >> > > configuration semantics, that would be a big win to move
> forward for
> > > >> > > this KIP. As for the TaskCorruptedException handling like wiping
> > > state
> > > >> > > stores, we can discuss that in the PR rather than in the KIP.
> > > >> > >
> > > >> > > Just to clarify, I'm onboard with the latest proposal, and
> probably
> > > we
> > > >> > > can move on for voting on this KIP now?
> > > >> > >
> > > >> > > Guozhang
> > > >> > >
> > > >> > > On Thu, Oct 19, 2023 at 5:33 AM Bruno Cadonna <
> cadonna@apache.org>
> > > >> > wrote:
> > > >> > > >
> > > >> > > > Hi Nick,
> > > >> > > >
> > > >> > > > What you and Lucas wrote about the different configurations of
> > > >> ALOS/EOS
> > > >> > > > and READ_COMMITTED/READ_UNCOMMITTED make sense to me. My
> earlier
> > > >> > > > concerns about changelogs diverging from the content of the
> local
> > > >> state
> > > >> > > > stores turned out to not apply. So I think, we can move on
> with
> > > >> those
> > > >> > > > configurations.
> > > >> > > >
> > > >> > > > Regarding the TaskCorruptedException and wiping out the state
> > > stores
> > > >> > > > under EOS, couldn't we abort the transaction on the state
> store
> > > and
> > > >> > > > close the task dirty? If the Kafka transaction was indeed
> > > committed,
> > > >> > the
> > > >> > > > store would restore the missing part from the changelog
> topic. If
> > > >> the
> > > >> > > > Kafka transaction was not committed, changelog topic and state
> > > store
> > > >> > are
> > > >> > > > in-sync.
> > > >> > > >
> > > >> > > > In any case, IMO those are implementation details that we do
> not
> > > >> need
> > > >> > to
> > > >> > > > discuss and solve in the KIP discussion. We can solve them on
> the
> > > >> PR.
> > > >> > > > The important thing is that the processing guarantees hold.
> > > >> > > >
> > > >> > > > Best,
> > > >> > > > Bruno
> > > >> > > >
> > > >> > > > On 10/18/23 3:56 PM, Nick Telford wrote:
> > > >> > > > > Hi Lucas,
> > > >> > > > >
> > > >> > > > > TaskCorruptedException is how Streams signals that the Task
> > > state
> > > >> > > needs to
> > > >> > > > > be wiped, so we can't retain that exception without also
> wiping
> > > >> state
> > > >> > > on
> > > >> > > > > timeouts.
> > > >> > > > >
> > > >> > > > > Regards,
> > > >> > > > > Nick
> > > >> > > > >
> > > >> > > > > On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <
> > > >> lbrutschy@confluent.io
> > > >> > > .invalid>
> > > >> > > > > wrote:
> > > >> > > > >
> > > >> > > > >> Hi Nick,
> > > >> > > > >>
> > > >> > > > >> I think indeed the better behavior would be to retry
> > > >> > commitTransaction
> > > >> > > > >> until we risk running out of time to meet `
> > > max.poll.interval.ms
> > > >> `.
> > > >> > > > >>
> > > >> > > > >> However, if it's handled as a `TaskCorruptedException` at
> the
> > > >> > moment,
> > > >> > > > >> I would do the same in this KIP, and leave exception
> handling
> > > >> > > > >> improvements to future work. 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 <
> > > >> > nick.telford@gmail.com>
> > > >> > > > >> wrote:
> > > >> > > > >>>
> > > >> > > > >>> Hi Lucas,
> > > >> > > > >>>
> > > >> > > > >>> Yeah, this is pretty much the direction I'm thinking of
> going
> > > in
> > > >> > > now. You
> > > >> > > > >>> make an interesting point about committing on-error under
> > > >> > > > >>> ALOS/READ_COMMITTED, although I haven't had a chance to
> think
> > > >> > > through the
> > > >> > > > >>> implications yet.
> > > >> > > > >>>
> > > >> > > > >>> Something that I ran into earlier this week is an issue
> with
> > > the
> > > >> > new
> > > >> > > > >>> handling of TimeoutException. Without TX stores,
> > > >> TimeoutException
> > > >> > > under
> > > >> > > > >> EOS
> > > >> > > > >>> throws a TaskCorruptedException, which wipes the stores.
> > > >> However,
> > > >> > > with TX
> > > >> > > > >>> stores, TimeoutException is now just bubbled up and dealt
> with
> > > >> as
> > > >> > it
> > > >> > > is
> > > >> > > > >>> under ALOS. The problem arises when the
> > > >> Producer#commitTransaction
> > > >> > > call
> > > >> > > > >>> times out: Streams attempts to ignore the error and
> continue
> > > >> > > producing,
> > > >> > > > >>> which causes the next call to Producer#send to throw
> > > >> > > > >>> "IllegalStateException: Cannot attempt operation `send`
> > > because
> > > >> the
> > > >> > > > >>> previous call to `commitTransaction` timed out and must be
> > > >> > retried".
> > > >> > > > >>>
> > > >> > > > >>> I'm not sure what we should do here: retrying the
> > > >> commitTransaction
> > > >> > > seems
> > > >> > > > >>> logical, but what if it times out again? Where do we draw
> the
> > > >> line
> > > >> > > and
> > > >> > > > >>> shutdown the instance?
> > > >> > > > >>>
> > > >> > > > >>> Regards,
> > > >> > > > >>> Nick
> > > >> > > > >>>
> > > >> > > > >>> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <
> > > >> > lbrutschy@confluent.io
> > > >> > > > >> .invalid>
> > > >> > > > >>> wrote:
> > > >> > > > >>>
> > > >> > > > >>>> Hi all,
> > > >> > > > >>>>
> > > >> > > > >>>> I think I liked your suggestion of allowing EOS with
> > > >> > > READ_UNCOMMITTED,
> > > >> > > > >>>> but keep wiping the state on error, and I'd vote for this
> > > >> solution
> > > >> > > > >>>> when introducing `default.state.isolation.level`. This
> way,
> > > >> we'd
> > > >> > > have
> > > >> > > > >>>> the most low-risk roll-out of this feature (no behavior
> > > change
> > > >> > > without
> > > >> > > > >>>> reconfiguration), with the possibility of switching to
> the
> > > most
> > > >> > > sane /
> > > >> > > > >>>> battle-tested default settings in 4.0. Essentially, we'd
> > > have a
> > > >> > > > >>>> feature flag but call it `default.state.isolation.level`
> and
> > > >> don't
> > > >> > > > >>>> have to deprecate it later.
> > > >> > > > >>>>
> > > >> > > > >>>> So the possible configurations would then be this:
> > > >> > > > >>>>
> > > >> > > > >>>> 1. ALOS/READ_UNCOMMITTED (default) = processing uses
> > > >> direct-to-DB,
> > > >> > > IQ
> > > >> > > > >>>> reads from DB.
> > > >> > > > >>>> 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ
> reads
> > > >> from
> > > >> > > > >>>> WriteBatch/DB. Flush on error (see note below).
> > > >> > > > >>>> 3. EOS/READ_UNCOMMITTED (default) = processing uses
> > > >> direct-to-DB,
> > > >> > IQ
> > > >> > > > >>>> reads from DB. Wipe state on error.
> > > >> > > > >>>> 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ
> reads
> > > >> from
> > > >> > > > >>>> WriteBatch/DB.
> > > >> > > > >>>>
> > > >> > > > >>>> I believe the feature is important enough that we will
> see
> > > good
> > > >> > > > >>>> adoption even without changing the default. In 4.0, when
> we
> > > >> have
> > > >> > > seen
> > > >> > > > >>>> this being adopted and is battle-tested, we make
> > > READ_COMMITTED
> > > >> > the
> > > >> > > > >>>> default for EOS, or even READ_COMITTED always the
> default,
> > > >> > depending
> > > >> > > > >>>> on our experiences. And we could add a clever
> implementation
> > > of
> > > >> > > > >>>> READ_UNCOMITTED with WriteBatches later.
> > > >> > > > >>>>
> > > >> > > > >>>> The only smell here is that
> `default.state.isolation.level`
> > > >> > wouldn't
> > > >> > > > >>>> be purely an IQ setting, but it would also (slightly)
> change
> > > >> the
> > > >> > > > >>>> behavior of the processing, but that seems unavoidable as
> > > long
> > > >> as
> > > >> > we
> > > >> > > > >>>> haven't solve READ_UNCOMITTED IQ with WriteBatches.
> > > >> > > > >>>>
> > > >> > > > >>>> Minor: As for Bruno's point 4, I think if we are
> concerned
> > > >> about
> > > >> > > this
> > > >> > > > >>>> 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).
> > > >> > > > >>>>
> > > >> > > > >>>> Cheers,
> > > >> > > > >>>> Lucas
> > > >> > > > >>>>
> > > >> > > > >>>> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <
> > > >> > > nick.telford@gmail.com>
> > > >> > > > >>>> wrote:
> > > >> > > > >>>>>
> > > >> > > > >>>>> Hi Guozhang,
> > > >> > > > >>>>>
> > > >> > > > >>>>> The KIP as it stands introduces a new configuration,
> > > >> > > > >>>>> default.state.isolation.level, which is independent of
> > > >> > > > >> processing.mode.
> > > >> > > > >>>>> It's intended that this new configuration be used to
> > > >> configure a
> > > >> > > > >> global
> > > >> > > > >>>> IQ
> > > >> > > > >>>>> isolation level in the short term, with a future KIP
> > > >> introducing
> > > >> > > the
> > > >> > > > >>>>> capability to change the isolation level on a per-query
> > > basis,
> > > >> > > > >> falling
> > > >> > > > >>>> back
> > > >> > > > >>>>> to the "default" defined by this config. That's why I
> called
> > > >> it
> > > >> > > > >>>> "default",
> > > >> > > > >>>>> for future-proofing.
> > > >> > > > >>>>>
> > > >> > > > >>>>> However, it currently includes the caveat that
> > > >> READ_UNCOMMITTED
> > > >> > is
> > > >> > > > >> not
> > > >> > > > >>>>> available under EOS. I think this is the coupling you
> are
> > > >> > alluding
> > > >> > > > >> to?
> > > >> > > > >>>>>
> > > >> > > > >>>>> This isn't intended to be a restriction of the API, but
> is
> > > >> > > currently
> > > >> > > > >> a
> > > >> > > > >>>>> technical limitation. However, after discussing with
> some
> > > >> users
> > > >> > > about
> > > >> > > > >>>>> use-cases that would require READ_UNCOMMITTED under
> EOS, I'm
> > > >> > > > >> inclined to
> > > >> > > > >>>>> remove that clause and put in the necessary work to make
> > > that
> > > >> > > > >> combination
> > > >> > > > >>>>> possible now.
> > > >> > > > >>>>>
> > > >> > > > >>>>> I currently see two possible approaches:
> > > >> > > > >>>>>
> > > >> > > > >>>>>     1. Disable TX StateStores internally when the
> > > >> IsolationLevel
> > > >> > is
> > > >> > > > >>>>>     READ_UNCOMMITTED and the processing.mode is EOS.
> This is
> > > >> more
> > > >> > > > >>>> difficult
> > > >> > > > >>>>>     than it sounds, as there are many assumptions being
> made
> > > >> > > > >> throughout
> > > >> > > > >>>> the
> > > >> > > > >>>>>     internals about the guarantees StateStores provide.
> It
> > > >> would
> > > >> > > > >>>> definitely add
> > > >> > > > >>>>>     a lot of extra "if (read_uncommitted && eos)"
> branches,
> > > >> > > > >> complicating
> > > >> > > > >>>>>     maintenance and testing.
> > > >> > > > >>>>>     2. Invest the time *now* to make READ_UNCOMMITTED
> of EOS
> > > >> > > > >> StateStores
> > > >> > > > >>>>>     possible. I have some ideas on how this could be
> > > achieved,
> > > >> > but
> > > >> > > > >> they
> > > >> > > > >>>> would
> > > >> > > > >>>>>     need testing and could introduce some additional
> issues.
> > > >> The
> > > >> > > > >> benefit
> > > >> > > > >>>> of
> > > >> > > > >>>>>     this approach is that it would make query-time
> > > >> > IsolationLevels
> > > >> > > > >> much
> > > >> > > > >>>> simpler
> > > >> > > > >>>>>     to implement in the future.
> > > >> > > > >>>>>
> > > >> > > > >>>>> Unfortunately, both will require considerable work that
> will
> > > >> > > further
> > > >> > > > >>>> delay
> > > >> > > > >>>>> this KIP, which was the reason I placed the restriction
> in
> > > the
> > > >> > KIP
> > > >> > > > >> in the
> > > >> > > > >>>>> first place.
> > > >> > > > >>>>>
> > > >> > > > >>>>> Regards,
> > > >> > > > >>>>> Nick
> > > >> > > > >>>>>
> > > >> > > > >>>>> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
> > > >> > > > >> guozhang.wang.us@gmail.com>
> > > >> > > > >>>>> wrote:
> > > >> > > > >>>>>
> > > >> > > > >>>>>> Hello Nick,
> > > >> > > > >>>>>>
> > > >> > > > >>>>>> First of all, thanks a lot for the great effort you've
> put
> > > in
> > > >> > > > >> driving
> > > >> > > > >>>>>> this KIP! I really like it coming through finally, as
> many
> > > >> > people
> > > >> > > > >> in
> > > >> > > > >>>>>> the community have raised this. At the same time I
> honestly
> > > >> > feel a
> > > >> > > > >> bit
> > > >> > > > >>>>>> ashamed for not putting enough of my time supporting
> it and
> > > >> > > > >> pushing it
> > > >> > > > >>>>>> through the finish line (you raised this KIP almost a
> year
> > > >> ago).
> > > >> > > > >>>>>>
> > > >> > > > >>>>>> I briefly passed through the DISCUSS thread so far, not
> > > sure
> > > >> > I've
> > > >> > > > >> 100
> > > >> > > > >>>>>> percent digested all the bullet points. But with the
> goal
> > > of
> > > >> > > > >> trying to
> > > >> > > > >>>>>> help take it through the finish line in mind, I'd want
> to
> > > >> throw
> > > >> > > > >>>>>> thoughts on top of my head only on the point #4 above
> > > which I
> > > >> > felt
> > > >> > > > >> may
> > > >> > > > >>>>>> be the main hurdle for the current KIP to drive to a
> > > >> consensus
> > > >> > > now.
> > > >> > > > >>>>>>
> > > >> > > > >>>>>> The general question I asked myself is, whether we
> want to
> > > >> > couple
> > > >> > > > >> "IQ
> > > >> > > > >>>>>> reading mode" with "processing mode". While
> technically I
> > > >> tend
> > > >> > to
> > > >> > > > >>>>>> agree with you that, it's feels like a bug if some
> single
> > > >> user
> > > >> > > > >> chose
> > > >> > > > >>>>>> "EOS" for processing mode while choosing "read
> uncommitted"
> > > >> for
> > > >> > IQ
> > > >> > > > >>>>>> reading mode, at the same time, I'm thinking if it's
> > > possible
> > > >> > that
> > > >> > > > >>>>>> there could be two different persons (or even two
> teams)
> > > that
> > > >> > > > >> would be
> > > >> > > > >>>>>> using the stream API to build the app, and the IQ API
> to
> > > >> query
> > > >> > the
> > > >> > > > >>>>>> running state of the app. I know this is less of a
> > > technical
> > > >> > thing
> > > >> > > > >> but
> > > >> > > > >>>>>> rather a more design stuff, but if it could be ever the
> > > case,
> > > >> > I'm
> > > >> > > > >>>>>> wondering if the personale using the IQ API knows
> about the
> > > >> > risks
> > > >> > > > >> of
> > > >> > > > >>>>>> using read uncommitted but still chose so for the
> favor of
> > > >> > > > >>>>>> performance, no matter if the underlying stream
> processing
> > > >> mode
> > > >> > > > >>>>>> configured by another personale is EOS or not. In that
> > > >> regard,
> > > >> > I'm
> > > >> > > > >>>>>> leaning towards a "leaving the door open, and close it
> > > later
> > > >> if
> > > >> > we
> > > >> > > > >>>>>> found it's a bad idea" aspect with a configuration
> 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 can still
> > > >> enforce
> > > >> > > > >>>>>> coupling them.
> > > >> > > > >>>>>>
> > > >> > > > >>>>>> Again, just my 2c here. Thanks again for the great
> patience
> > > >> and
> > > >> > > > >>>>>> diligence on this KIP.
> > > >> > > > >>>>>>
> > > >> > > > >>>>>>
> > > >> > > > >>>>>> Guozhang
> > > >> > > > >>>>>>
> > > >> > > > >>>>>>
> > > >> > > > >>>>>>
> > > >> > > > >>>>>> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
> > > >> > > > >> nick.telford@gmail.com>
> > > >> > > > >>>>>> wrote:
> > > >> > > > >>>>>>>
> > > >> > > > >>>>>>> Hi Bruno,
> > > >> > > > >>>>>>>
> > > >> > > > >>>>>>> 4.
> > > >> > > > >>>>>>> I'll hold off on making that change until we have a
> > > >> consensus
> > > >> > as
> > > >> > > > >> to
> > > >> > > > >>>> what
> > > >> > > > >>>>>>> configuration to use to control all of this, as it'll
> be
> > > >> > > > >> affected by
> > > >> > > > >>>> the
> > > >> > > > >>>>>>> decision on EOS isolation levels.
> > > >> > > > >>>>>>>
> > > >> > > > >>>>>>> 5.
> > > >> > > > >>>>>>> Done. I've chosen "committedOffsets".
> > > >> > > > >>>>>>>
> > > >> > > > >>>>>>> Regards,
> > > >> > > > >>>>>>> Nick
> > > >> > > > >>>>>>>
> > > >> > > > >>>>>>> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <
> > > >> > cadonna@apache.org>
> > > >> > > > >>>> wrote:
> > > >> > > > >>>>>>>
> > > >> > > > >>>>>>>> Hi Nick,
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>>> 1.
> > > >> > > > >>>>>>>> Yeah, you are probably right that it does not make
> too
> > > much
> > > >> > > > >> sense.
> > > >> > > > >>>>>>>> Thanks for the clarification!
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>>> 4.
> > > >> > > > >>>>>>>> Yes, sorry for the back and forth, but I think for
> the
> > > >> sake of
> > > >> > > > >> the
> > > >> > > > >>>> KIP
> > > >> > > > >>>>>>>> it is better to let the ALOS behavior as it is for
> now
> > > due
> > > >> to
> > > >> > > > >> the
> > > >> > > > >>>>>>>> possible issues you would run into. Maybe we can
> find a
> > > >> > > > >> solution
> > > >> > > > >>>> in the
> > > >> > > > >>>>>>>> future. Now the question returns to whether we really
> > > need
> > > >> > > > >>>>>>>> default.state.isolation.level. Maybe the config
> could be
> > > >> the
> > > >> > > > >>>> feature
> > > >> > > > >>>>>>>> flag Sophie requested.
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>>> 5.
> > > >> > > > >>>>>>>> There 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
> > > >> committedOffsetFor().
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>>> Best,
> > > >> > > > >>>>>>>> Bruno
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>>> On 10/13/23 3:21 PM, Nick Telford wrote:
> > > >> > > > >>>>>>>>> Hi Bruno,
> > > >> > > > >>>>>>>>>
> > > >> > > > >>>>>>>>> Thanks for getting back to me.
> > > >> > > > >>>>>>>>>
> > > >> > > > >>>>>>>>> 1.
> > > >> > > > >>>>>>>>> I think this should be possible. Are you thinking
> of the
> > > >> > > > >>>> situation
> > > >> > > > >>>>>> where
> > > >> > > > >>>>>>>> a
> > > >> > > > >>>>>>>>> user may downgrade to a previous version of Kafka
> > > >> Streams? In
> > > >> > > > >>>> that
> > > >> > > > >>>>>> case,
> > > >> > > > >>>>>>>>> sadly, the RocksDBStore would get wiped by the older
> > > >> version
> > > >> > > > >> of
> > > >> > > > >>>> Kafka
> > > >> > > > >>>>>>>>> Streams anyway, because that version wouldn't
> understand
> > > >> the
> > > >> > > > >>>> extra
> > > >> > > > >>>>>> column
> > > >> > > > >>>>>>>>> family (that holds offsets), so the missing Position
> > > file
> > > >> > > > >> would
> > > >> > > > >>>>>>>>> automatically get rebuilt when the store is rebuilt
> from
> > > >> the
> > > >> > > > >>>>>> changelog.
> > > >> > > > >>>>>>>>> Are there other situations than downgrade where a
> > > >> > > > >> transactional
> > > >> > > > >>>> store
> > > >> > > > >>>>>>>> could
> > > >> > > > >>>>>>>>> be replaced by a non-transactional one? I can't
> think of
> > > >> any.
> > > >> > > > >>>>>>>>>
> > > >> > > > >>>>>>>>> 2.
> > > >> > > > >>>>>>>>> Ahh yes, the Test Plan - my Kryptonite! This section
> > > >> > > > >> definitely
> > > >> > > > >>>>>> needs to
> > > >> > > > >>>>>>>> be
> > > >> > > > >>>>>>>>> fleshed out. I'll work on that. How much detail do
> you
> > > >> need?
> > > >> > > > >>>>>>>>>
> > > >> > > > >>>>>>>>> 3.
> > > >> > > > >>>>>>>>> See my previous email discussing this.
> > > >> > > > >>>>>>>>>
> > > >> > > > >>>>>>>>> 4.
> > > >> > > > >>>>>>>>> Hmm, this is an interesting point. Are you
> suggesting
> > > that
> > > >> > > > >> under
> > > >> > > > >>>> ALOS
> > > >> > > > >>>>>>>>> READ_COMMITTED should not be supported?
> > > >> > > > >>>>>>>>>
> > > >> > > > >>>>>>>>> Regards,
> > > >> > > > >>>>>>>>> Nick
> > > >> > > > >>>>>>>>>
> > > >> > > > >>>>>>>>> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
> > > >> > > > >> cadonna@apache.org>
> > > >> > > > >>>>>> wrote:
> > > >> > > > >>>>>>>>>
> > > >> > > > >>>>>>>>>> Hi Nick,
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>> I think the KIP is converging!
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>> 1.
> > > >> > > > >>>>>>>>>> I am wondering whether it makes sense to write the
> > > >> position
> > > >> > > > >> file
> > > >> > > > >>>>>> during
> > > >> > > > >>>>>>>>>> close as we do for the checkpoint file, so that in
> case
> > > >> the
> > > >> > > > >>>> state
> > > >> > > > >>>>>> store
> > > >> > > > >>>>>>>>>> is replaced with a non-transactional state store
> the
> > > >> > > > >>>>>> non-transactional
> > > >> > > > >>>>>>>>>> state store finds the position file. I think, this
> is
> > > not
> > > >> > > > >>>> strictly
> > > >> > > > >>>>>>>>>> needed, but would be a nice behavior instead of
> just
> > > >> > > > >> deleting
> > > >> > > > >>>> the
> > > >> > > > >>>>>>>>>> position file.
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>> 2.
> > > >> > > > >>>>>>>>>> The test plan does not mention integration tests.
> Do
> > > you
> > > >> not
> > > >> > > > >>>> need to
> > > >> > > > >>>>>>>>>> extend existing ones and add new ones. Also for
> > > upgrading
> > > >> > > > >> and
> > > >> > > > >>>>>>>>>> downgrading you might need integration and/or
> system
> > > >> tests.
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>> 3.
> > > >> > > > >>>>>>>>>> I think Sophie made a point. Although, IQ reading
> from
> > > >> > > > >>>> uncommitted
> > > >> > > > >>>>>> data
> > > >> > > > >>>>>>>>>> under EOS might be considered a bug by some people.
> > > Thus,
> > > >> > > > >> your
> > > >> > > > >>>> KIP
> > > >> > > > >>>>>> would
> > > >> > > > >>>>>>>>>> fix a bug rather than changing the intended
> behavior.
> > > >> > > > >> However, I
> > > >> > > > >>>>>> also
> > > >> > > > >>>>>>>>>> see that a feature flag would help users that rely
> on
> > > >> this
> > > >> > > > >> buggy
> > > >> > > > >>>>>>>>>> behavior (at least until AK 4.0).
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>> 4.
> > > >> > > > >>>>>>>>>> This is related to the previous point. I assume
> that
> > > the
> > > >> > > > >>>> difference
> > > >> > > > >>>>>>>>>> between READ_COMMITTED and READ_UNCOMMITTED for
> ALOS is
> > > >> > > > >> that in
> > > >> > > > >>>> the
> > > >> > > > >>>>>>>>>> former you enable transactions on the state store
> and
> > > in
> > > >> the
> > > >> > > > >>>> latter
> > > >> > > > >>>>>> you
> > > >> > > > >>>>>>>>>> disable them. If my assumption is correct, I think
> that
> > > >> is
> > > >> > > > >> an
> > > >> > > > >>>> issue.
> > > >> > > > >>>>>>>>>> Let's assume under ALOS Streams fails over a
> couple of
> > > >> times
> > > >> > > > >>>> more or
> > > >> > > > >>>>>>>>>> less at the same step in processing after value 3
> is
> > > >> added
> > > >> > > > >> to an
> > > >> > > > >>>>>>>>>> aggregation but the offset of the corresponding
> input
> > > >> record
> > > >> > > > >>>> was not
> > > >> > > > >>>>>>>>>> committed. Without transactions disabled, the
> > > aggregation
> > > >> > > > >> value
> > > >> > > > >>>>>> would
> > > >> > > > >>>>>>>>>> increase by 3 for each failover. With transactions
> > > >> enabled,
> > > >> > > > >>>> value 3
> > > >> > > > >>>>>>>>>> would only be added to the aggregation once when
> the
> > > >> offset
> > > >> > > > >> of
> > > >> > > > >>>> the
> > > >> > > > >>>>>> input
> > > >> > > > >>>>>>>>>> record is committed and the transaction finally
> > > >> completes.
> > > >> > > > >> So
> > > >> > > > >>>> the
> > > >> > > > >>>>>>>>>> content of the state store would change depending
> on
> > > the
> > > >> > > > >>>>>> configuration
> > > >> > > > >>>>>>>>>> for IQ. IMO, the content of the state store should
> be
> > > >> > > > >>>> independent
> > > >> > > > >>>>>> from
> > > >> > > > >>>>>>>>>> IQ. Given this issue, I propose to not use
> transactions
> > > >> with
> > > >> > > > >>>> ALOS at
> > > >> > > > >>>>>>>>>> all. I was a big proponent of using transactions
> with
> > > >> ALOS,
> > > >> > > > >> but
> > > >> > > > >>>> I
> > > >> > > > >>>>>>>>>> realized that transactions with ALOS is not as
> easy as
> > > >> > > > >> enabling
> > > >> > > > >>>>>>>>>> transactions on state stores. Another aspect that
> is
> > > >> > > > >>>> problematic is
> > > >> > > > >>>>>> that
> > > >> > > > >>>>>>>>>> the changelog topic which actually replicates the
> state
> > > >> > > > >> store
> > > >> > > > >>>> is not
> > > >> > > > >>>>>>>>>> transactional under ALOS. Thus, it might happen
> that
> > > the
> > > >> > > > >> state
> > > >> > > > >>>>>> store and
> > > >> > > > >>>>>>>>>> the changelog differ in their content. All of this
> is
> > > >> maybe
> > > >> > > > >>>> solvable
> > > >> > > > >>>>>>>>>> somehow, but for the sake of this KIP, I would
> leave it
> > > >> for
> > > >> > > > >> the
> > > >> > > > >>>>>> future.
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>> Best,
> > > >> > > > >>>>>>>>>> Bruno
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > > >> > > > >>>>>>>>>>> Hey Nick! First of all thanks for taking up this
> > > awesome
> > > >> > > > >>>> feature,
> > > >> > > > >>>>>> I'm
> > > >> > > > >>>>>>>>>> sure
> > > >> > > > >>>>>>>>>>> every single
> > > >> > > > >>>>>>>>>>> Kafka Streams user and dev would agree that it is
> > > sorely
> > > >> > > > >>>> needed.
> > > >> > > > >>>>>>>>>>>
> > > >> > > > >>>>>>>>>>> I've just been catching up on the KIP and
> surrounding
> > > >> > > > >>>> discussion,
> > > >> > > > >>>>>> so
> > > >> > > > >>>>>>>>>> please
> > > >> > > > >>>>>>>>>>> forgive me
> > > >> > > > >>>>>>>>>>> for any misunderstandings or misinterpretations
> of the
> > > >> > > > >> current
> > > >> > > > >>>>>> plan and
> > > >> > > > >>>>>>>>>>> don't hesitate to
> > > >> > > > >>>>>>>>>>> correct me.
> > > >> > > > >>>>>>>>>>>
> > > >> > > > >>>>>>>>>>> Before I jump in, I just want to say that having
> seen
> > > >> this
> > > >> > > > >>>> drag on
> > > >> > > > >>>>>> for
> > > >> > > > >>>>>>>> so
> > > >> > > > >>>>>>>>>>> long, my singular
> > > >> > > > >>>>>>>>>>> goal in responding is to help this KIP past a
> > > perceived
> > > >> > > > >>>> impasse so
> > > >> > > > >>>>>> we
> > > >> > > > >>>>>>>> can
> > > >> > > > >>>>>>>>>>> finally move on
> > > >> > > > >>>>>>>>>>> to voting and implementing it. Long discussions
> are to
> > > >> be
> > > >> > > > >>>> expected
> > > >> > > > >>>>>> for
> > > >> > > > >>>>>>>>>>> major features like
> > > >> > > > >>>>>>>>>>> this but it's completely on us as the Streams
> devs to
> > > >> make
> > > >> > > > >> sure
> > > >> > > > >>>>>> there
> > > >> > > > >>>>>>>> is
> > > >> > > > >>>>>>>>>> an
> > > >> > > > >>>>>>>>>>> end in sight
> > > >> > > > >>>>>>>>>>> for any ongoing discussion.
> > > >> > > > >>>>>>>>>>>
> > > >> > > > >>>>>>>>>>> With that said, it's my understanding that the
> KIP as
> > > >> > > > >> currently
> > > >> > > > >>>>>>>> proposed
> > > >> > > > >>>>>>>>>> is
> > > >> > > > >>>>>>>>>>> just not tenable
> > > >> > > > >>>>>>>>>>> for Kafka Streams, and would prevent some EOS
> users
> > > from
> > > >> > > > >>>> upgrading
> > > >> > > > >>>>>> to
> > > >> > > > >>>>>>>> the
> > > >> > > > >>>>>>>>>>> version it
> > > >> > > > >>>>>>>>>>> first appears in. Given that we can't predict or
> > > >> guarantee
> > > >> > > > >>>> whether
> > > >> > > > >>>>>> any
> > > >> > > > >>>>>>>> of
> > > >> > > > >>>>>>>>>>> the followup KIPs
> > > >> > > > >>>>>>>>>>> would be completed in the same release cycle as
> this
> > > >> one,
> > > >> > > > >> we
> > > >> > > > >>>> need
> > > >> > > > >>>>>> to
> > > >> > > > >>>>>>>> make
> > > >> > > > >>>>>>>>>>> sure that the
> > > >> > > > >>>>>>>>>>> feature is either compatible with all current
> users or
> > > >> else
> > > >> > > > >>>>>>>>>> feature-flagged
> > > >> > > > >>>>>>>>>>> so that they may
> > > >> > > > >>>>>>>>>>> opt in/out.
> > > >> > > > >>>>>>>>>>>
> > > >> > > > >>>>>>>>>>> Therefore, IIUC we need to have either (or both)
> of
> > > >> these
> > > >> > > > >> as
> > > >> > > > >>>>>>>>>>> fully-implemented config options:
> > > >> > > > >>>>>>>>>>> 1. default.state.isolation.level
> > > >> > > > >>>>>>>>>>> 2. enable.transactional.state.stores
> > > >> > > > >>>>>>>>>>>
> > > >> > > > >>>>>>>>>>> This way EOS users for whom read_committed
> semantics
> > > are
> > > >> > > > >> not
> > > >> > > > >>>>>> viable can
> > > >> > > > >>>>>>>>>>> still upgrade,
> > > >> > > > >>>>>>>>>>> and either use the isolation.level config to
> leverage
> > > >> the
> > > >> > > > >> new
> > > >> > > > >>>> txn
> > > >> > > > >>>>>> state
> > > >> > > > >>>>>>>>>>> stores without sacrificing
> > > >> > > > >>>>>>>>>>> their application semantics, or else simply keep
> the
> > > >> > > > >>>> transactional
> > > >> > > > >>>>>>>> state
> > > >> > > > >>>>>>>>>>> stores disabled until we
> > > >> > > > >>>>>>>>>>> are able to fully implement the isolation level
> > > >> > > > >> configuration
> > > >> > > > >>>> at
> > > >> > > > >>>>>> either
> > > >> > > > >>>>>>>>>> an
> > > >> > > > >>>>>>>>>>> application or query level.
> > > >> > > > >>>>>>>>>>>
> > > >> > > > >>>>>>>>>>> Frankly you are the expert here and know much more
> > > about
> > > >> > > > >> the
> > > >> > > > >>>>>> tradeoffs
> > > >> > > > >>>>>>>> in
> > > >> > > > >>>>>>>>>>> both semantics and
> > > >> > > > >>>>>>>>>>> effort level of implementing one of these configs
> vs
> > > the
> > > >> > > > >>>> other. In
> > > >> > > > >>>>>> my
> > > >> > > > >>>>>>>>>>> opinion, either option would
> > > >> > > > >>>>>>>>>>> be fine and I would leave the decision of which
> one to
> > > >> > > > >> include
> > > >> > > > >>>> in
> > > >> > > > >>>>>> this
> > > >> > > > >>>>>>>>>> KIP
> > > >> > > > >>>>>>>>>>> completely up to you.
> > > >> > > > >>>>>>>>>>> I just don't see a way for the KIP to proceed
> without
> > > >> some
> > > >> > > > >>>>>> variation of
> > > >> > > > >>>>>>>>>> the
> > > >> > > > >>>>>>>>>>> above that would allow
> > > >> > > > >>>>>>>>>>> EOS users to opt-out of read_committed.
> > > >> > > > >>>>>>>>>>>
> > > >> > > > >>>>>>>>>>> (If it's all the same to you, I would recommend
> 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 that gets released.
> > > >> > > > >>>>>>>>>>> So personally, my choice would be to add the
> feature
> > > >> flag
> > > >> > > > >> and
> > > >> > > > >>>>>> leave it
> > > >> > > > >>>>>>>>>> off
> > > >> > > > >>>>>>>>>>> by default. If all 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
> > > >> > > > >>>>>>>>>>>
> > > >> > > > >>>>>>>>>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > > >> > > > >>>>>> nick.telford@gmail.com>
> > > >> > > > >>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>> Hi Bruno,
> > > >> > > > >>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>> Agreed, I can live with that for now.
> > > >> > > > >>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>> In an effort to keep the scope of this KIP from
> > > >> > > > >> expanding, I'm
> > > >> > > > >>>>>> leaning
> > > >> > > > >>>>>>>>>>>> towards just providing a configurable
> > > >> > > > >>>>>> default.state.isolation.level
> > > >> > > > >>>>>>>> and
> > > >> > > > >>>>>>>>>>>> removing IsolationLevel from the
> StateStoreContext.
> > > >> This
> > > >> > > > >>>> would be
> > > >> > > > >>>>>>>>>>>> compatible with adding support for query-time
> > > >> > > > >> IsolationLevels
> > > >> > > > >>>> in
> > > >> > > > >>>>>> the
> > > >> > > > >>>>>>>>>>>> future, whilst providing a way for users to
> select an
> > > >> > > > >>>> isolation
> > > >> > > > >>>>>> level
> > > >> > > > >>>>>>>>>> now.
> > > >> > > > >>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>> The big problem with this, however, is that if a
> user
> > > >> > > > >> selects
> > > >> > > > >>>>>>>>>>>> processing.mode
> > > >> > > > >>>>>>>>>>>> = "exactly-once(-v2|-beta)", and
> > > >> > > > >>>> default.state.isolation.level =
> > > >> > > > >>>>>>>>>>>> "READ_UNCOMMITTED", we need to guarantee that the
> > > data
> > > >> > > > >> isn't
> > > >> > > > >>>>>> written
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>> disk until commit() is called, but we also need
> to
> > > >> permit
> > > >> > > > >> IQ
> > > >> > > > >>>>>> threads
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>> read from the ongoing transaction.
> > > >> > > > >>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>> A simple solution would be to (temporarily)
> forbid
> > > this
> > > >> > > > >>>>>> combination of
> > > >> > > > >>>>>>>>>>>> configuration, and have
> default.state.isolation.level
> > > >> > > > >>>>>> automatically
> > > >> > > > >>>>>>>>>> switch
> > > >> > > > >>>>>>>>>>>> to READ_COMMITTED when processing.mode is
> anything
> > > >> other
> > > >> > > > >> than
> > > >> > > > >>>>>>>>>>>> at-least-once. Do you think this would be
> acceptable?
> > > >> > > > >>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>> In a later KIP, we can add support for query-time
> > > >> > > > >> isolation
> > > >> > > > >>>>>> levels and
> > > >> > > > >>>>>>>>>>>> solve this particular problem there, which would
> > > relax
> > > >> > > > >> this
> > > >> > > > >>>>>>>> restriction.
> > > >> > > > >>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>> Regards,
> > > >> > > > >>>>>>>>>>>> Nick
> > > >> > > > >>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> > > >> > > > >>>> cadonna@apache.org>
> > > >> > > > >>>>>>>>>> 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,
> > > >> > > > >>>>>>>>>>>>> Bruno
> > > >> > > > >>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > > >> > > > >>>>>>>>>>>>>> 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 <
> > > >> > > > >>>>>> nick.telford@gmail.com
> > > >> > > > >>>>>>>>>
> > > >> > > > >>>>>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> Hi everyone,
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> I agree that having IsolationLevel be
> determined
> > > at
> > > >> > > > >>>> query-time
> > > >> > > > >>>>>> is
> > > >> > > > >>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>> ideal design, but there are a few sticking
> points:
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> 1.
> > > >> > > > >>>>>>>>>>>>>>> There needs to be some way to communicate the
> > > >> > > > >>>> IsolationLevel
> > > >> > > > >>>>>> down
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>> RocksDBStore itself, so that the query can
> respect
> > > >> it.
> > > >> > > > >>>> Since
> > > >> > > > >>>>>> stores
> > > >> > > > >>>>>>>>>>>> are
> > > >> > > > >>>>>>>>>>>>>>> "layered" in functionality (i.e.
> > > ChangeLoggingStore,
> > > >> > > > >>>>>> MeteredStore,
> > > >> > > > >>>>>>>>>>>>> etc.),
> > > >> > > > >>>>>>>>>>>>>>> we need some way to deliver that information
> to
> > > the
> > > >> > > > >> bottom
> > > >> > > > >>>>>> layer.
> > > >> > > > >>>>>>>> For
> > > >> > > > >>>>>>>>>>>>> IQv2,
> > > >> > > > >>>>>>>>>>>>>>> we can use the existing State#query() method,
> but
> > > >> IQv1
> > > >> > > > >> has
> > > >> > > > >>>> no
> > > >> > > > >>>>>> way
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>> do
> > > >> > > > >>>>>>>>>>>>>>> this.
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> A simple approach, which would potentially
> open up
> > > >> > > > >> other
> > > >> > > > >>>>>> options,
> > > >> > > > >>>>>>>>>>>> would
> > > >> > > > >>>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>> to add something like:
> ReadOnlyKeyValueStore<K, V>
> > > >> > > > >>>>>>>>>>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > > >> > > > >>>>>>>> ReadOnlyKeyValueStore
> > > >> > > > >>>>>>>>>>>>> (and
> > > >> > > > >>>>>>>>>>>>>>> similar to ReadOnlyWindowStore,
> > > >> ReadOnlySessionStore,
> > > >> > > > >>>> etc.).
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> 2.
> > > >> > > > >>>>>>>>>>>>>>> As mentioned above, RocksDB WriteBatches are
> not
> > > >> > > > >>>> thread-safe,
> > > >> > > > >>>>>> which
> > > >> > > > >>>>>>>>>>>>> causes
> > > >> > > > >>>>>>>>>>>>>>> a problem if we want to provide
> READ_UNCOMMITTED
> > > >> > > > >>>> Iterators. I
> > > >> > > > >>>>>> also
> > > >> > > > >>>>>>>>>>>> had a
> > > >> > > > >>>>>>>>>>>>>>> look at RocksDB Transactions[1], but they
> solve a
> > > >> very
> > > >> > > > >>>>>> different
> > > >> > > > >>>>>>>>>>>>> problem,
> > > >> > > > >>>>>>>>>>>>>>> and have the same thread-safety issue.
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> One possible approach that I mentioned is
> chaining
> > > >> > > > >>>>>> WriteBatches:
> > > >> > > > >>>>>>>>>> every
> > > >> > > > >>>>>>>>>>>>>>> time a new Interactive Query is received (i.e.
> > > >> > > > >>>> readOnlyView,
> > > >> > > > >>>>>> see
> > > >> > > > >>>>>>>>>>>> above,
> > > >> > > > >>>>>>>>>>>>>>> is called) we "freeze" the existing
> WriteBatch,
> > > and
> > > >> > > > >> start a
> > > >> > > > >>>>>> new one
> > > >> > > > >>>>>>>>>>>> for
> > > >> > > > >>>>>>>>>>>>> new
> > > >> > > > >>>>>>>>>>>>>>> writes. The Interactive Query queries the
> "chain"
> > > of
> > > >> > > > >>>> previous
> > > >> > > > >>>>>>>>>>>>> WriteBatches
> > > >> > > > >>>>>>>>>>>>>>> + the underlying database; while the
> StreamThread
> > > >> > > > >> starts
> > > >> > > > >>>>>> writing to
> > > >> > > > >>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>> *new* WriteBatch. On-commit, the StreamThread
> > > would
> > > >> > > > >> write
> > > >> > > > >>>> *all*
> > > >> > > > >>>>>>>>>>>>>>> WriteBatches in the chain to the database
> (that
> > > have
> > > >> > > > >> not
> > > >> > > > >>>> yet
> > > >> > > > >>>>>> been
> > > >> > > > >>>>>>>>>>>>> written).
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> WriteBatches would be closed/freed only when
> they
> > > >> have
> > > >> > > > >> been
> > > >> > > > >>>>>> both
> > > >> > > > >>>>>>>>>>>>>>> committed, and all open Interactive Queries on
> > > them
> > > >> > > > >> have
> > > >> > > > >>>> been
> > > >> > > > >>>>>>>> closed.
> > > >> > > > >>>>>>>>>>>>> This
> > > >> > > > >>>>>>>>>>>>>>> would require some reference counting.
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> Obviously a drawback of this approach is the
> > > >> potential
> > > >> > > > >> for
> > > >> > > > >>>>>>>> increased
> > > >> > > > >>>>>>>>>>>>>>> memory usage: if an Interactive Query is
> > > long-lived,
> > > >> > > > >> for
> > > >> > > > >>>>>> example by
> > > >> > > > >>>>>>>>>>>>> doing a
> > > >> > > > >>>>>>>>>>>>>>> full scan over a large database, or even just
> > > >> pausing
> > > >> > > > >> in
> > > >> > > > >>>> the
> > > >> > > > >>>>>> middle
> > > >> > > > >>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>> an
> > > >> > > > >>>>>>>>>>>>>>> iteration, then the existing chain of
> WriteBatches
> > > >> > > > >> could be
> > > >> > > > >>>>>> kept
> > > >> > > > >>>>>>>>>>>> around
> > > >> > > > >>>>>>>>>>>>> for
> > > >> > > > >>>>>>>>>>>>>>> a long time, potentially forever.
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> --
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> A.
> > > >> > > > >>>>>>>>>>>>>>> Going off on a tangent, it looks like in
> addition
> > > to
> > > >> > > > >>>> supporting
> > > >> > > > >>>>>>>>>>>>>>> READ_COMMITTED queries, we could go further
> and
> > > >> support
> > > >> > > > >>>>>>>>>>>> REPEATABLE_READ
> > > >> > > > >>>>>>>>>>>>>>> queries (i.e. where subsequent reads to the
> same
> > > key
> > > >> > > > >> in the
> > > >> > > > >>>>>> same
> > > >> > > > >>>>>>>>>>>>>>> Interactive Query are guaranteed to yield the
> same
> > > >> > > > >> value)
> > > >> > > > >>>> by
> > > >> > > > >>>>>> making
> > > >> > > > >>>>>>>>>>>> use
> > > >> > > > >>>>>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>> RocksDB Snapshots[2]. These are fairly
> > > lightweight,
> > > >> so
> > > >> > > > >> the
> > > >> > > > >>>>>>>>>> performance
> > > >> > > > >>>>>>>>>>>>>>> impact is likely to be negligible, but they do
> > > >> require
> > > >> > > > >>>> that the
> > > >> > > > >>>>>>>>>>>>> Interactive
> > > >> > > > >>>>>>>>>>>>>>> Query session can be explicitly closed.
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> This could be achieved if we made the above
> > > >> > > > >> readOnlyView
> > > >> > > > >>>>>> interface
> > > >> > > > >>>>>>>>>>>> look
> > > >> > > > >>>>>>>>>>>>>>> more like:
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueView<K, V>
> implements
> > > >> > > > >>>>>>>>>>>> ReadOnlyKeyValueStore<K,
> > > >> > > > >>>>>>>>>>>>>>> V>, AutoCloseable {}
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > > >> > > > >>>>>>>>>>>>>>>         ...
> > > >> > > > >>>>>>>>>>>>>>>         ReadOnlyKeyValueView<K, V>
> > > >> > > > >>>> readOnlyView(IsolationLevel
> > > >> > > > >>>>>>>>>>>>> isolationLevel);
> > > >> > > > >>>>>>>>>>>>>>> }
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> But this would be a breaking change, as
> existing
> > > >> IQv1
> > > >> > > > >>>> queries
> > > >> > > > >>>>>> are
> > > >> > > > >>>>>>>>>>>>>>> guaranteed to never call store.close(), and
> > > >> therefore
> > > >> > > > >> these
> > > >> > > > >>>>>> would
> > > >> > > > >>>>>>>>>> leak
> > > >> > > > >>>>>>>>>>>>>>> memory under REPEATABLE_READ.
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> B.
> > > >> > > > >>>>>>>>>>>>>>> One thing that's notable: MyRocks states that
> they
> > > >> > > > >> support
> > > >> > > > >>>>>>>>>>>>> READ_COMMITTED
> > > >> > > > >>>>>>>>>>>>>>> and REPEATABLE_READ, but they make no mention
> of
> > > >> > > > >>>>>>>>>>>> READ_UNCOMMITTED[3][4].
> > > >> > > > >>>>>>>>>>>>>>> This could be because doing so is technically
> > > >> > > > >>>>>> difficult/impossible
> > > >> > > > >>>>>>>>>>>> using
> > > >> > > > >>>>>>>>>>>>>>> the primitives available in RocksDB.
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> --
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> Lucas, to address your points:
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> U1.
> > > >> > > > >>>>>>>>>>>>>>> It's only "SHOULD" to permit alternative (i.e.
> > > >> > > > >> non-RocksDB)
> > > >> > > > >>>>>>>>>>>>>>> implementations of StateStore that do not
> support
> > > >> > > > >> atomic
> > > >> > > > >>>>>> writes.
> > > >> > > > >>>>>>>>>>>>> Obviously
> > > >> > > > >>>>>>>>>>>>>>> in those cases, the guarantees Kafka Streams
> > > >> > > > >>>> provides/expects
> > > >> > > > >>>>>> would
> > > >> > > > >>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>> relaxed. Do you think we should require all
> > > >> > > > >>>> implementations to
> > > >> > > > >>>>>>>>>> support
> > > >> > > > >>>>>>>>>>>>>>> atomic writes?
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> U2.
> > > >> > > > >>>>>>>>>>>>>>> Stores can support multiple IsolationLevels.
> As
> > > >> we've
> > > >> > > > >>>> discussed
> > > >> > > > >>>>>>>>>> above,
> > > >> > > > >>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>> ideal scenario would be to specify the
> > > >> IsolationLevel
> > > >> > > > >> at
> > > >> > > > >>>>>>>> query-time.
> > > >> > > > >>>>>>>>>>>>>>> Failing that, I think the second-best
> approach is
> > > to
> > > >> > > > >>>> define the
> > > >> > > > >>>>>>>>>>>>>>> IsolationLevel for *all* queries based on the
> > > >> > > > >>>> processing.mode,
> > > >> > > > >>>>>>>> which
> > > >> > > > >>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>> what the default
> > > StateStoreContext#isolationLevel()
> > > >> > > > >>>> achieves.
> > > >> > > > >>>>>> Would
> > > >> > > > >>>>>>>>>>>> you
> > > >> > > > >>>>>>>>>>>>>>> prefer an alternative?
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> While the existing implementation is
> equivalent to
> > > >> > > > >>>>>>>> READ_UNCOMMITTED,
> > > >> > > > >>>>>>>>>>>>> this
> > > >> > > > >>>>>>>>>>>>>>> can yield unexpected results/errors under
> EOS, if
> > > a
> > > >> > > > >>>>>> transaction is
> > > >> > > > >>>>>>>>>>>>> rolled
> > > >> > > > >>>>>>>>>>>>>>> back. While this would be a change in
> behaviour
> > > for
> > > >> > > > >> users,
> > > >> > > > >>>> it
> > > >> > > > >>>>>> would
> > > >> > > > >>>>>>>>>>>> look
> > > >> > > > >>>>>>>>>>>>>>> more like a bug fix than a breaking change.
> That
> > > >> said,
> > > >> > > > >> we
> > > >> > > > >>>>>> *could*
> > > >> > > > >>>>>>>>>> make
> > > >> > > > >>>>>>>>>>>>> it
> > > >> > > > >>>>>>>>>>>>>>> configurable, and default to the existing
> > > behaviour
> > > >> > > > >>>>>>>>>> (READ_UNCOMMITTED)
> > > >> > > > >>>>>>>>>>>>>>> instead of inferring it from the
> processing.mode?
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> N1, N2.
> > > >> > > > >>>>>>>>>>>>>>> These were only primitives to avoid boxing
> costs,
> > > >> but
> > > >> > > > >> since
> > > >> > > > >>>>>> this is
> > > >> > > > >>>>>>>>>>>> not
> > > >> > > > >>>>>>>>>>>>> a
> > > >> > > > >>>>>>>>>>>>>>> performance sensitive area, it should be fine
> to
> > > >> > > > >> change if
> > > >> > > > >>>>>> that's
> > > >> > > > >>>>>>>>>>>>> desirable.
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> N3.
> > > >> > > > >>>>>>>>>>>>>>> It's because the store "manages its own
> offsets",
> > > >> which
> > > >> > > > >>>>>> includes
> > > >> > > > >>>>>>>> both
> > > >> > > > >>>>>>>>>>>>>>> committing the offset, *and providing it* via
> > > >> > > > >>>>>> getCommittedOffset().
> > > >> > > > >>>>>>>>>>>>>>> Personally, I think "managesOffsets" conveys
> this
> > > >> best,
> > > >> > > > >>>> but I
> > > >> > > > >>>>>> don't
> > > >> > > > >>>>>>>>>>>> mind
> > > >> > > > >>>>>>>>>>>>>>> changing it if the nomenclature is unclear.
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> Sorry for the massive emails/essays!
> > > >> > > > >>>>>>>>>>>>>>> --
> > > >> > > > >>>>>>>>>>>>>>> Nick
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> 1:
> > > >> > > > >> https://github.com/facebook/rocksdb/wiki/Transactions
> > > >> > > > >>>>>>>>>>>>>>> 2:
> > > >> https://github.com/facebook/rocksdb/wiki/Snapshot
> > > >> > > > >>>>>>>>>>>>>>> 3:
> > > >> > > > >>>>>>>>
> > > >> > > > >>
> > > https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > >> > > > >>>>>>>>>>>>>>> 4:
> > > >> > > > >>>>
> https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > > >> > > > >>>>>>>>>>>>>>> <lb...@confluent.io.invalid> wrote:
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>> Hi Nick,
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>> since I last read it in April, the KIP has
> become
> > > >> much
> > > >> > > > >>>>>> cleaner and
> > > >> > > > >>>>>>>>>>>>>>>> easier to read. Great work!
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>> It feels to me the last big open point is
> whether
> > > >> we
> > > >> > > > >> can
> > > >> > > > >>>>>> implement
> > > >> > > > >>>>>>>>>>>>>>>> isolation level as a query parameter. I
> > > understand
> > > >> > > > >> that
> > > >> > > > >>>> there
> > > >> > > > >>>>>> are
> > > >> > > > >>>>>>>>>>>>>>>> implementation concerns, but as Colt says, it
> > > would
> > > >> > > > >> be a
> > > >> > > > >>>> great
> > > >> > > > >>>>>>>>>>>>>>>> addition, and would also simplify the
> migration
> > > >> path
> > > >> > > > >> for
> > > >> > > > >>>> this
> > > >> > > > >>>>>>>>>> change.
> > > >> > > > >>>>>>>>>>>>>>>> Is the implementation problem you mentioned
> > > caused
> > > >> by
> > > >> > > > >> the
> > > >> > > > >>>>>>>> WriteBatch
> > > >> > > > >>>>>>>>>>>>>>>> not having a notion of a snapshot, as the
> > > >> underlying
> > > >> > > > >> DB
> > > >> > > > >>>>>> iterator
> > > >> > > > >>>>>>>>>>>> does?
> > > >> > > > >>>>>>>>>>>>>>>> In that case, I am not sure a chain of
> > > WriteBatches
> > > >> > > > >> as you
> > > >> > > > >>>>>> propose
> > > >> > > > >>>>>>>>>>>>>>>> would fully solve the problem, but maybe I
> didn't
> > > >> dig
> > > >> > > > >>>> enough
> > > >> > > > >>>>>> into
> > > >> > > > >>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>> details to fully understand it.
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>> If it's not possible to implement it now,
> would
> > > it
> > > >> be
> > > >> > > > >> an
> > > >> > > > >>>>>> option to
> > > >> > > > >>>>>>>>>>>>>>>> make sure in this KIP that we do not fully
> close
> > > >> the
> > > >> > > > >> door
> > > >> > > > >>>> on
> > > >> > > > >>>>>>>>>>>> per-query
> > > >> > > > >>>>>>>>>>>>>>>> isolation levels in the interface, as it may
> be
> > > >> > > > >> possible
> > > >> > > > >>>> to
> > > >> > > > >>>>>>>>>> implement
> > > >> > > > >>>>>>>>>>>>>>>> the missing primitives in RocksDB or Speedb
> in
> > > the
> > > >> > > > >> future.
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>> Understanding:
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>> * U1) Why is it only "SHOULD" for
> > > changelogOffsets
> > > >> to
> > > >> > > > >> be
> > > >> > > > >>>>>> persisted
> > > >> > > > >>>>>>>>>>>>>>>> atomically with the records?
> > > >> > > > >>>>>>>>>>>>>>>> * U2) Don't understand the default
> implementation
> > > >> of
> > > >> > > > >>>>>>>>>>>> `isolationLevel`.
> > > >> > > > >>>>>>>>>>>>>>>> The isolation level should be a property of
> the
> > > >> > > > >> underlying
> > > >> > > > >>>>>> store,
> > > >> > > > >>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>> not be defined by the default config?
> Existing
> > > >> stores
> > > >> > > > >>>> probably
> > > >> > > > >>>>>>>> don't
> > > >> > > > >>>>>>>>>>>>>>>> guarantee READ_COMMITTED, so the default
> should
> > > be
> > > >> to
> > > >> > > > >>>> return
> > > >> > > > >>>>>>>>>>>>>>>> READ_UNCOMMITTED.
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>> Nits:
> > > >> > > > >>>>>>>>>>>>>>>> * N1) Could `getComittedOffset` use an
> > > >> `OptionalLong`
> > > >> > > > >>>> return
> > > >> > > > >>>>>> type,
> > > >> > > > >>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> avoid the `null`?
> > > >> > > > >>>>>>>>>>>>>>>> * N2) Could `apporixmateNumUncomittedBytes`
> use
> > > an
> > > >> > > > >>>>>> `OptionalLong`
> > > >> > > > >>>>>>>>>>>>>>>> return type, to avoid the `-1`?
> > > >> > > > >>>>>>>>>>>>>>>> * N3) I don't understand why `managesOffsets`
> > > uses
> > > >> the
> > > >> > > > >>>>>> 'manage'
> > > >> > > > >>>>>>>>>> verb,
> > > >> > > > >>>>>>>>>>>>>>>> whereas all other methods use the "commits"
> verb.
> > > >> I'd
> > > >> > > > >>>> suggest
> > > >> > > > >>>>>>>>>>>>>>>> `commitsOffsets`.
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>> Either way, it feels this KIP is very close
> to
> > > the
> > > >> > > > >> finish
> > > >> > > > >>>>>> line,
> > > >> > > > >>>>>>>> I'm
> > > >> > > > >>>>>>>>>>>>>>>> looking forward to seeing this in production!
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>> Cheers,
> > > >> > > > >>>>>>>>>>>>>>>> Lucas
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy
> <
> > > >> > > > >>>>>> colt@littlehorse.io
> > > >> > > > >>>>>>>>>
> > > >> > > > >>>>>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>> Making IsolationLevel a query-time
> constraint,
> > > >> > > > >> rather
> > > >> > > > >>>> than
> > > >> > > > >>>>>>>> linking
> > > >> > > > >>>>>>>>>>>> it
> > > >> > > > >>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>> the processing.guarantee.
> > > >> > > > >>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>> As I understand it, would this allow even a
> user
> > > >> of
> > > >> > > > >> EOS
> > > >> > > > >>>> to
> > > >> > > > >>>>>>>> control
> > > >> > > > >>>>>>>>>>>>>>>> whether
> > > >> > > > >>>>>>>>>>>>>>>>> reading committed or uncommitted records?
> If so,
> > > >> I am
> > > >> > > > >>>> highly
> > > >> > > > >>>>>> in
> > > >> > > > >>>>>>>>>>>> favor
> > > >> > > > >>>>>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>> this.
> > > >> > > > >>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>> I know that I was one of the early people to
> > > point
> > > >> > > > >> out
> > > >> > > > >>>> the
> > > >> > > > >>>>>>>> current
> > > >> > > > >>>>>>>>>>>>>>>>> shortcoming that IQ reads uncommitted
> records,
> > > but
> > > >> > > > >> just
> > > >> > > > >>>> this
> > > >> > > > >>>>>>>>>>>> morning I
> > > >> > > > >>>>>>>>>>>>>>>>> realized a pattern we use which means that
> (for
> > > >> > > > >> certain
> > > >> > > > >>>>>> queries)
> > > >> > > > >>>>>>>>>> our
> > > >> > > > >>>>>>>>>>>>>>>> system
> > > >> > > > >>>>>>>>>>>>>>>>> needs to be able to read uncommitted
> records,
> > > >> which
> > > >> > > > >> is
> > > >> > > > >>>> the
> > > >> > > > >>>>>>>> current
> > > >> > > > >>>>>>>>>>>>>>>> behavior
> > > >> > > > >>>>>>>>>>>>>>>>> of Kafka Streams in EOS.***
> > > >> > > > >>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>> If IsolationLevel being a query-time
> decision
> > > >> allows
> > > >> > > > >> for
> > > >> > > > >>>>>> this,
> > > >> > > > >>>>>>>> then
> > > >> > > > >>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>> would be amazing. I would also vote that the
> > > >> default
> > > >> > > > >>>> behavior
> > > >> > > > >>>>>>>>>> should
> > > >> > > > >>>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>> for
> > > >> > > > >>>>>>>>>>>>>>>>> reading uncommitted records, because it is
> > > totally
> > > >> > > > >>>> possible
> > > >> > > > >>>>>> for a
> > > >> > > > >>>>>>>>>>>>> valid
> > > >> > > > >>>>>>>>>>>>>>>>> application to depend on that behavior, and
> > > >> breaking
> > > >> > > > >> it
> > > >> > > > >>>> in a
> > > >> > > > >>>>>>>> minor
> > > >> > > > >>>>>>>>>>>>>>>> release
> > > >> > > > >>>>>>>>>>>>>>>>> might be a bit strong.
> > > >> > > > >>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>> *** (Note, for the curious reader....) Our
> > > >> > > > >> use-case/query
> > > >> > > > >>>>>> pattern
> > > >> > > > >>>>>>>>>>>> is a
> > > >> > > > >>>>>>>>>>>>>>>> bit
> > > >> > > > >>>>>>>>>>>>>>>>> complex, but reading "uncommitted" records
> is
> > > >> > > > >> actually
> > > >> > > > >>>> safe
> > > >> > > > >>>>>> in
> > > >> > > > >>>>>>>> our
> > > >> > > > >>>>>>>>>>>>> case
> > > >> > > > >>>>>>>>>>>>>>>>> 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 soon as the processor
> > > >> processes
> > > >> > > > >> the
> > > >> > > > >>>>>> result.
> > > >> > > > >>>>>>>> If
> > > >> > > > >>>>>>>>>>>>> they
> > > >> > > > >>>>>>>>>>>>>>>>> can't read uncommitted records, they may
> > > receive a
> > > >> > > > >> "201 -
> > > >> > > > >>>>>>>> Created"
> > > >> > > > >>>>>>>>>>>>>>>>> response, immediately followed by a "404 -
> 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 <
> > > >> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> > > >> > > > >>>>>>>>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>> Addendum:
> > > >> > > > >>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>> I think we would also face the same problem
> > > with
> > > >> the
> > > >> > > > >>>>>> approach
> > > >> > > > >>>>>>>> John
> > > >> > > > >>>>>>>>>>>>>>>> outlined
> > > >> > > > >>>>>>>>>>>>>>>>>> earlier (using the record cache as a
> > > transaction
> > > >> > > > >> buffer
> > > >> > > > >>>> and
> > > >> > > > >>>>>>>>>>>> flushing
> > > >> > > > >>>>>>>>>>>>>>>> it
> > > >> > > > >>>>>>>>>>>>>>>>>> straight to SST files). This is because the
> > > >> record
> > > >> > > > >> cache
> > > >> > > > >>>>>> (the
> > > >> > > > >>>>>>>>>>>>>>>> ThreadCache
> > > >> > > > >>>>>>>>>>>>>>>>>> class) is not thread-safe, so every commit
> > > would
> > > >> > > > >>>> invalidate
> > > >> > > > >>>>>> open
> > > >> > > > >>>>>>>>>> IQ
> > > >> > > > >>>>>>>>>>>>>>>>>> Iterators in the same way that RocksDB
> > > >> WriteBatches
> > > >> > > > >> do.
> > > >> > > > >>>>>>>>>>>>>>>>>> --
> > > >> > > > >>>>>>>>>>>>>>>>>> Nick
> > > >> > > > >>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick
> Telford <
> > > >> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> > > >> > > > >>>>>>>>>>>>>>>>>> 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.
> > > >> > > > >>>>>>>>>>>>>>>>>>> 2. Making IsolationLevel a query-time
> > > >> constraint,
> > > >> > > > >>>> rather
> > > >> > > > >>>>>> than
> > > >> > > > >>>>>>>>>>>>>>>> linking it
> > > >> > > > >>>>>>>>>>>>>>>>>>> to the processing.guarantee.
> > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>> There's a wrinkle that makes this a
> challenge:
> > > >> > > > >>>> Interactive
> > > >> > > > >>>>>>>>>> Queries
> > > >> > > > >>>>>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>>> open an Iterator, when using transactions
> and
> > > >> > > > >>>>>> READ_UNCOMMITTED.
> > > >> > > > >>>>>>>>>>>>>>>>>>> The problem is that under
> READ_UNCOMMITTED,
> > > >> queries
> > > >> > > > >>>> need
> > > >> > > > >>>>>> to be
> > > >> > > > >>>>>>>>>>>> able
> > > >> > > > >>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>> read records from the currently
> uncommitted
> > > >> > > > >> transaction
> > > >> > > > >>>>>> buffer
> > > >> > > > >>>>>>>>>>>>>>>>>>> (WriteBatch). This includes for Iterators,
> > > which
> > > >> > > > >> should
> > > >> > > > >>>>>> iterate
> > > >> > > > >>>>>>>>>>>>>>>> both the
> > > >> > > > >>>>>>>>>>>>>>>>>>> transaction buffer and underlying database
> > > >> (using
> > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>> The issue is that when the StreamThread
> > > >> commits, it
> > > >> > > > >>>> writes
> > > >> > > > >>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>> current
> > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch to RocksDB *and then clears the
> > > >> > > > >> WriteBatch*.
> > > >> > > > >>>>>>>> Clearing
> > > >> > > > >>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch while an Interactive Query
> holds an
> > > >> open
> > > >> > > > >>>>>> Iterator on
> > > >> > > > >>>>>>>>>> it
> > > >> > > > >>>>>>>>>>>>>>>> will
> > > >> > > > >>>>>>>>>>>>>>>>>>> invalidate the Iterator. Worse, it turns
> out
> > > >> that
> > > >> > > > >>>> Iterators
> > > >> > > > >>>>>>>> over
> > > >> > > > >>>>>>>>>> a
> > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch become invalidated not just
> when
> > > the
> > > >> > > > >>>> WriteBatch
> > > >> > > > >>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>> cleared,
> > > >> > > > >>>>>>>>>>>>>>>>>> but
> > > >> > > > >>>>>>>>>>>>>>>>>>> also when the Iterators' current key
> receives
> > > a
> > > >> new
> > > >> > > > >>>> write.
> > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>> Now that I'm writing this, I remember that
> > > this
> > > >> is
> > > >> > > > >> the
> > > >> > > > >>>>>> major
> > > >> > > > >>>>>>>>>>>> reason
> > > >> > > > >>>>>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>> I
> > > >> > > > >>>>>>>>>>>>>>>>>>> switched the original design from having a
> > > >> > > > >> query-time
> > > >> > > > >>>>>>>>>>>>>>>> IsolationLevel to
> > > >> > > > >>>>>>>>>>>>>>>>>>> having the IsolationLevel linked to the
> > > >> > > > >>>> transactionality
> > > >> > > > >>>>>> of the
> > > >> > > > >>>>>>>>>>>>>>>> stores
> > > >> > > > >>>>>>>>>>>>>>>>>>> themselves.
> > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>> It *might* be possible to resolve this, by
> > > >> having a
> > > >> > > > >>>>>> "chain" of
> > > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatches, with the StreamThread
> switching
> > > >> to a
> > > >> > > > >> new
> > > >> > > > >>>>>>>> WriteBatch
> > > >> > > > >>>>>>>>>>>>>>>>>> whenever
> > > >> > > > >>>>>>>>>>>>>>>>>>> a new Interactive Query attempts to read
> from
> > > >> the
> > > >> > > > >>>>>> database, but
> > > >> > > > >>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>> could
> > > >> > > > >>>>>>>>>>>>>>>>>>> cause some performance problems/memory
> > > pressure
> > > >> > > > >> when
> > > >> > > > >>>>>> subjected
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>> a
> > > >> > > > >>>>>>>>>>>>>>>> high
> > > >> > > > >>>>>>>>>>>>>>>>>>> Interactive Query load. It would also
> reduce
> > > the
> > > >> > > > >>>>>> efficiency of
> > > >> > > > >>>>>>>>>>>>>>>>>> WriteBatches
> > > >> > > > >>>>>>>>>>>>>>>>>>> on-commit, as we'd have to write N
> > > WriteBatches,
> > > >> > > > >> where
> > > >> > > > >>>> N
> > > >> > > > >>>>>> is the
> > > >> > > > >>>>>>>>>>>>>>>> number of
> > > >> > > > >>>>>>>>>>>>>>>>>>> Interactive Queries since the last commit.
> > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>> I realise this is getting into the weeds
> of
> > > the
> > > >> > > > >>>>>> implementation,
> > > >> > > > >>>>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>> you'd
> > > >> > > > >>>>>>>>>>>>>>>>>>> rather we focus on the API for now, but I
> > > think
> > > >> > > > >> it's
> > > >> > > > >>>>>> important
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>> consider
> > > >> > > > >>>>>>>>>>>>>>>>>>> how to implement the desired API, in case
> we
> > > >> come
> > > >> > > > >> up
> > > >> > > > >>>> with
> > > >> > > > >>>>>> an
> > > >> > > > >>>>>>>> API
> > > >> > > > >>>>>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>>> cannot be implemented efficiently, or
> even at
> > > >> all!
> > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>> Thoughts?
> > > >> > > > >>>>>>>>>>>>>>>>>>> --
> > > >> > > > >>>>>>>>>>>>>>>>>>> Nick
> > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno
> Cadonna <
> > > >> > > > >>>>>>>> cadonna@apache.org
> > > >> > > > >>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>> 6.
> > > >> > > > >>>>>>>>>>>>>>>>>>>> Of course, you are right! My bad!
> > > >> > > > >>>>>>>>>>>>>>>>>>>> Wiping out the state in the downgrading
> case
> > > is
> > > >> > > > >> fine.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>> 3a.
> > > >> > > > >>>>>>>>>>>>>>>>>>>> Focus on the public facing changes for
> the
> > > >> KIP. We
> > > >> > > > >>>> will
> > > >> > > > >>>>>> manage
> > > >> > > > >>>>>>>>>> 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.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>> Best,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> Bruno
> > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> Thanks for getting back to me!
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> 2.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> The fact that implementations can always
> > > track
> > > >> > > > >>>> estimated
> > > >> > > > >>>>>>>> memory
> > > >> > > > >>>>>>>>>>>>>>>> usage
> > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> the wrapper is a good point. I can
> remove -1
> > > >> as
> > > >> > > > >> an
> > > >> > > > >>>>>> option,
> > > >> > > > >>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>> I'll
> > > >> > > > >>>>>>>>>>>>>>>>>>>> clarify
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> the JavaDoc that 0 is not just for
> > > >> > > > >> non-transactional
> > > >> > > > >>>>>> stores,
> > > >> > > > >>>>>>>>>>>>>>>> which is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> currently misleading.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> 6.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> The problem with catching the exception
> in
> > > the
> > > >> > > > >>>> downgrade
> > > >> > > > >>>>>>>>>> process
> > > >> > > > >>>>>>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> would require new code in the Kafka
> version
> > > >> being
> > > >> > > > >>>>>> downgraded
> > > >> > > > >>>>>>>>>> to.
> > > >> > > > >>>>>>>>>>>>>>>> Since
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> users could conceivably downgrade to
> almost
> > > >> *any*
> > > >> > > > >>>> older
> > > >> > > > >>>>>>>> version
> > > >> > > > >>>>>>>>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>>> Kafka
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> Streams, I'm not sure how we could add
> that
> > > >> code?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> The only way I can think of doing it
> would
> > > be
> > > >> to
> > > >> > > > >>>> provide
> > > >> > > > >>>>>> a
> > > >> > > > >>>>>>>>>>>>>>>> dedicated
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> downgrade tool, that goes through every
> > > local
> > > >> > > > >> store
> > > >> > > > >>>> and
> > > >> > > > >>>>>>>> removes
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> offsets column families. But that seems
> like
> > > >> an
> > > >> > > > >>>>>> unnecessary
> > > >> > > > >>>>>>>>>>>>>>>> amount of
> > > >> > > > >>>>>>>>>>>>>>>>>>>> extra
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> code to maintain just to handle a
> somewhat
> > > >> niche
> > > >> > > > >>>>>> situation,
> > > >> > > > >>>>>>>>>> when
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> alternative (automatically wipe and
> restore
> > > >> > > > >> stores)
> > > >> > > > >>>>>> should be
> > > >> > > > >>>>>>>>>>>>>>>>>>>> acceptable.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes
> > > you've
> > > >> > > > >>>> requested.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> 3a.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> I agree that IsolationLevel makes more
> sense
> > > >> at
> > > >> > > > >>>>>> query-time,
> > > >> > > > >>>>>>>> and
> > > >> > > > >>>>>>>>>>>> I
> > > >> > > > >>>>>>>>>>>>>>>>>>>> actually
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> initially attempted to place the
> > > >> IsolationLevel
> > > >> > > > >> at
> > > >> > > > >>>>>>>> query-time,
> > > >> > > > >>>>>>>>>>>>>>>> but I
> > > >> > > > >>>>>>>>>>>>>>>>>> ran
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> into some problems:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> - The key issue is that, under ALOS
> we're
> > > not
> > > >> > > > >> staging
> > > >> > > > >>>>>> writes
> > > >> > > > >>>>>>>> in
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> transactions, so can't perform writes
> at the
> > > >> > > > >>>>>> READ_COMMITTED
> > > >> > > > >>>>>>>>>>>>>>>> isolation
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> level. However, this may be addressed
> if we
> > > >> > > > >> decide to
> > > >> > > > >>>>>>>> *always*
> > > >> > > > >>>>>>>>>>>>>>>> use
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> transactions as discussed under 3b.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> > > >> > > > >>>> implementations. I
> > > >> > > > >>>>>>>>>> remember
> > > >> > > > >>>>>>>>>>>>>>>>>> having
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> some difficulty understanding the IQv1
> > > >> internals,
> > > >> > > > >>>> which
> > > >> > > > >>>>>> made
> > > >> > > > >>>>>>>> it
> > > >> > > > >>>>>>>>>>>>>>>>>>>> difficult
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> to determine what needed to be changed.
> > > >> However,
> > > >> > > > >> I
> > > >> > > > >>>>>> *think*
> > > >> > > > >>>>>>>> this
> > > >> > > > >>>>>>>>>>>>>>>> can be
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> addressed for both implementations by
> > > wrapping
> > > >> > > > >> the
> > > >> > > > >>>>>>>> RocksDBStore
> > > >> > > > >>>>>>>>>>>>>>>> in an
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> IsolationLevel-dependent wrapper, that
> > > >> overrides
> > > >> > > > >> read
> > > >> > > > >>>>>> methods
> > > >> > > > >>>>>>>>>>>>>>>> (get,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> etc.)
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> to either read directly from the
> database or
> > > >> > > > >> from the
> > > >> > > > >>>>>> ongoing
> > > >> > > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> But IQv1 might still be difficult.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> - If IsolationLevel becomes a query
> > > >> constraint,
> > > >> > > > >> then
> > > >> > > > >>>> all
> > > >> > > > >>>>>>>> other
> > > >> > > > >>>>>>>>>>>>>>>>>>>> StateStores
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> will need to respect it, including the
> > > >> in-memory
> > > >> > > > >>>> stores.
> > > >> > > > >>>>>> This
> > > >> > > > >>>>>>>>>>>>>>>> would
> > > >> > > > >>>>>>>>>>>>>>>>>>>> require
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> us to adapt in-memory stores to stage
> their
> > > >> > > > >> writes so
> > > >> > > > >>>>>> they
> > > >> > > > >>>>>>>> can
> > > >> > > > >>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>>>> isolated
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> from READ_COMMITTTED queries. It would
> also
> > > >> > > > >> become an
> > > >> > > > >>>>>>>> important
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> consideration for third-party stores on
> > > >> upgrade,
> > > >> > > > >> as
> > > >> > > > >>>>>> without
> > > >> > > > >>>>>>>>>>>>>>>> changes,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> they
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> would not support READ_COMMITTED queries
> > > >> > > > >> correctly.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> Ultimately, I may need some help making
> the
> > > >> > > > >> necessary
> > > >> > > > >>>>>> change
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> IQv1
> > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> support this, but I don't think it's
> > > >> > > > >> fundamentally
> > > >> > > > >>>>>>>> impossible,
> > > >> > > > >>>>>>>>>>>>>>>> if we
> > > >> > > > >>>>>>>>>>>>>>>>>>>> want
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> to pursue this route.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> 3b.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> The main reason I chose to keep ALOS
> > > >> > > > >> un-transactional
> > > >> > > > >>>>>> was to
> > > >> > > > >>>>>>>>>>>>>>>> minimize
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> behavioural change for most users (I
> believe
> > > >> most
> > > >> > > > >>>> Streams
> > > >> > > > >>>>>>>> users
> > > >> > > > >>>>>>>>>>>>>>>> use
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> default configuration, which is ALOS).
> That
> > > >> said,
> > > >> > > > >>>> it's
> > > >> > > > >>>>>> clear
> > > >> > > > >>>>>>>>>>>>>>>> that if
> > > >> > > > >>>>>>>>>>>>>>>>>>>> ALOS
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> also used transactional stores, the only
> > > >> change
> > > >> > > > >> in
> > > >> > > > >>>>>> behaviour
> > > >> > > > >>>>>>>>>>>>>>>> would be
> > > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> it would become *more correct*, which
> could
> > > be
> > > >> > > > >>>>>> considered a
> > > >> > > > >>>>>>>>>> "bug
> > > >> > > > >>>>>>>>>>>>>>>> fix"
> > > >> > > > >>>>>>>>>>>>>>>>>> by
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> users, rather than a change they need to
> > > >> handle.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> I believe that performance using
> > > transactions
> > > >> > > > >> (aka.
> > > >> > > > >>>>>> RocksDB
> > > >> > > > >>>>>>>>>>>>>>>>>>>> WriteBatches)
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> should actually be *better* than the
> > > >> un-batched
> > > >> > > > >>>>>> write-path
> > > >> > > > >>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> currently used[1]. The only
> "performance"
> > > >> > > > >>>> consideration
> > > >> > > > >>>>>> will
> > > >> > > > >>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> increased memory usage that transactions
> > > >> require.
> > > >> > > > >>>> Given
> > > >> > > > >>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>> mitigations
> > > >> > > > >>>>>>>>>>>>>>>>>>>> for
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> this memory that we have in place, I
> would
> > > >> expect
> > > >> > > > >>>> that
> > > >> > > > >>>>>> this
> > > >> > > > >>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>> not a
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> problem for most users.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> If we're happy to do so, we can make
> ALOS
> > > also
> > > >> > > > >> use
> > > >> > > > >>>>>>>>>> transactions.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> Nick
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> Link 1:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>
> > > >> > > > >>>>
> > > >> > > > >>
> > > >> > >
> > > >>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno
> > > Cadonna <
> > > >> > > > >>>>>>>>>>>> cadonna@apache.org
> > > >> > > > >>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Thanks for the updates and sorry for
> the
> > > >> delay
> > > >> > > > >> on my
> > > >> > > > >>>>>> side!
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 1.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Making the default implementation for
> > > >> flush() a
> > > >> > > > >>>> no-op
> > > >> > > > >>>>>> sounds
> > > >> > > > >>>>>>>>>>>>>>>> good to
> > > >> > > > >>>>>>>>>>>>>>>>>>>> me.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 2.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I think what was bugging me here is
> that a
> > > >> > > > >>>> third-party
> > > >> > > > >>>>>> state
> > > >> > > > >>>>>>>>>>>>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>>>>>> needs
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> to implement the state store interface.
> > > That
> > > >> > > > >> means
> > > >> > > > >>>> they
> > > >> > > > >>>>>> need
> > > >> > > > >>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> implement a wrapper around the actual
> state
> > > >> > > > >> store
> > > >> > > > >>>> as we
> > > >> > > > >>>>>> do
> > > >> > > > >>>>>>>> for
> > > >> > > > >>>>>>>>>>>>>>>>>> RocksDB
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> with RocksDBStore. So, a third-party
> state
> > > >> > > > >> store can
> > > >> > > > >>>>>> always
> > > >> > > > >>>>>>>>>>>>>>>> estimate
> > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> uncommitted bytes, if it wants,
> because the
> > > >> > > > >> wrapper
> > > >> > > > >>>> can
> > > >> > > > >>>>>>>> record
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>> added
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> bytes.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> One case I can think of where
> returning -1
> > > >> makes
> > > >> > > > >>>> sense
> > > >> > > > >>>>>> is
> > > >> > > > >>>>>>>> when
> > > >> > > > >>>>>>>>>>>>>>>>>> Streams
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> does not need to estimate the size of
> the
> > > >> write
> > > >> > > > >>>> batch
> > > >> > > > >>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>> trigger
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> extraordinary commits, because the
> > > >> third-party
> > > >> > > > >> state
> > > >> > > > >>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>> takes care
> > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memory. But in that case the method
> could
> > > >> also
> > > >> > > > >> just
> > > >> > > > >>>>>> return
> > > >> > > > >>>>>>>> 0.
> > > >> > > > >>>>>>>>>>>>>>>> Even
> > > >> > > > >>>>>>>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> case would be better solved with a
> method
> > > >> that
> > > >> > > > >>>> returns
> > > >> > > > >>>>>>>> whether
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> store manages itself the memory used
> for
> > > >> > > > >> uncommitted
> > > >> > > > >>>>>> bytes
> > > >> > > > >>>>>>>> or
> > > >> > > > >>>>>>>>>>>>>>>> not.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Said that, I am fine with keeping the
> -1
> > > >> return
> > > >> > > > >>>> value,
> > > >> > > > >>>>>> I was
> > > >> > > > >>>>>>>>>>>>>>>> just
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> wondering when and if it will be used.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Regarding returning 0 for transactional
> > > state
> > > >> > > > >> stores
> > > >> > > > >>>>>> when
> > > >> > > > >>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>> batch
> > > >> > > > >>>>>>>>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> empty, I was just wondering because you
> > > >> > > > >> explicitly
> > > >> > > > >>>>>> stated
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> "or {@code 0} if this StateStore does
> not
> > > >> > > > >> support
> > > >> > > > >>>>>>>>>>>> transactions."
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> So it seemed to me returning 0 could
> only
> > > >> > > > >> happen for
> > > >> > > > >>>>>>>>>>>>>>>>>> non-transactional
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> state stores.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a) What do you think if we move the
> > > isolation
> > > >> > > > >> level
> > > >> > > > >>>> to
> > > >> > > > >>>>>> IQ
> > > >> > > > >>>>>>>> (v1
> > > >> > > > >>>>>>>>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>> v2)?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> In the end this is the only component
> that
> > > >> > > > >> really
> > > >> > > > >>>> needs
> > > >> > > > >>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> specify
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> isolation level. It is similar to the
> Kafka
> > > >> > > > >> consumer
> > > >> > > > >>>>>> that
> > > >> > > > >>>>>>>> can
> > > >> > > > >>>>>>>>>>>>>>>> choose
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> with what isolation level to read the
> input
> > > >> > > > >> topic.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> For IQv1 the isolation level should go
> into
> > > >> > > > >>>>>>>>>>>>>>>> StoreQueryParameters. For
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> IQv2, I would add it to the Query
> > > interface.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> b) Point a) raises the question what
> should
> > > >> > > > >> happen
> > > >> > > > >>>>>> during
> > > >> > > > >>>>>>>>>>>>>>>>>> at-least-once
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing when the state store does
> not
> > > use
> > > >> > > > >>>>>> transactions?
> > > >> > > > >>>>>>>>>> John
> > > >> > > > >>>>>>>>>>>>>>>> in
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> past proposed to also use transactions
> on
> > > >> state
> > > >> > > > >>>> stores
> > > >> > > > >>>>>> for
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> at-least-once. I like that idea,
> because it
> > > >> > > > >> avoids
> > > >> > > > >>>>>>>> aggregating
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>> same
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> records over and over again in the
> case of
> > > a
> > > >> > > > >>>> failure. We
> > > >> > > > >>>>>>>> had a
> > > >> > > > >>>>>>>>>>>>>>>> case
> > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the past where a Streams applications
> in
> > > >> > > > >>>> at-least-once
> > > >> > > > >>>>>> mode
> > > >> > > > >>>>>>>>>> was
> > > >> > > > >>>>>>>>>>>>>>>>>> failing
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> continuously for some reasons I do not
> > > >> remember
> > > >> > > > >>>> before
> > > >> > > > >>>>>>>>>>>>>>>> committing the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> offsets. After each failover, the app
> > > >> aggregated
> > > >> > > > >>>> again
> > > >> > > > >>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>> again the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> same records. Of course the aggregate
> > > >> increased
> > > >> > > > >> to
> > > >> > > > >>>> very
> > > >> > > > >>>>>>>> wrong
> > > >> > > > >>>>>>>>>>>>>>>> values
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> just because of the failover. With
> > > >> transactions
> > > >> > > > >> on
> > > >> > > > >>>> the
> > > >> > > > >>>>>> state
> > > >> > > > >>>>>>>>>>>>>>>> stores
> > > >> > > > >>>>>>>>>>>>>>>>>> we
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> could have avoided this. The app would
> have
> > > >> > > > >> output
> > > >> > > > >>>> the
> > > >> > > > >>>>>> same
> > > >> > > > >>>>>>>>>>>>>>>> aggregate
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> multiple times (i.e., after each
> failover)
> > > >> but
> > > >> > > > >> at
> > > >> > > > >>>> least
> > > >> > > > >>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>> value of
> > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> aggregate would not depend on the
> number of
> > > >> > > > >>>> failovers.
> > > >> > > > >>>>>>>>>>>>>>>> Outputting the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> same aggregate multiple times would be
> > > >> incorrect
> > > >> > > > >>>> under
> > > >> > > > >>>>>>>>>>>>>>>> exactly-once
> > > >> > > > >>>>>>>>>>>>>>>>>> but
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> it is OK for at-least-once.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> If it makes sense to add a config to
> turn
> > > on
> > > >> > > > >> and off
> > > >> > > > >>>>>>>>>>>>>>>> transactions on
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> state stores under at-least-once or
> just
> > > use
> > > >> > > > >>>>>> transactions in
> > > >> > > > >>>>>>>>>>>>>>>> any case
> > > >> > > > >>>>>>>>>>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a question we should also discuss in
> this
> > > >> KIP.
> > > >> > > > >> It
> > > >> > > > >>>>>> depends a
> > > >> > > > >>>>>>>>>> bit
> > > >> > > > >>>>>>>>>>>>>>>> on
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> performance trade-off. Maybe to be
> safe, I
> > > >> would
> > > >> > > > >>>> add a
> > > >> > > > >>>>>>>> config.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 4.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Your points are all valid. I tend to
> say to
> > > >> > > > >> keep the
> > > >> > > > >>>>>> metrics
> > > >> > > > >>>>>>>>>>>>>>>> around
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() until we remove flush()
> completely
> > > >> from
> > > >> > > > >> the
> > > >> > > > >>>>>>>> interface.
> > > >> > > > >>>>>>>>>>>>>>>> Calls
> > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() might still exist since
> existing
> > > >> > > > >> processors
> > > >> > > > >>>>>> might
> > > >> > > > >>>>>>>>>> still
> > > >> > > > >>>>>>>>>>>>>>>> call
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() explicitly as you mentioned in
> 1).
> > > >> For
> > > >> > > > >>>> sure, we
> > > >> > > > >>>>>> need
> > > >> > > > >>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>> document
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> how the metrics change due to the
> > > >> transactions
> > > >> > > > >> in
> > > >> > > > >>>> the
> > > >> > > > >>>>>>>> upgrade
> > > >> > > > >>>>>>>>>>>>>>>> notes.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> 5.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I see. Then you should describe how the
> > > >> > > > >> .position
> > > >> > > > >>>> files
> > > >> > > > >>>>>> are
> > > >> > > > >>>>>>>>>>>>>>>> handled
> > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a dedicated section of the KIP or
> > > incorporate
> > > >> > > > >> the
> > > >> > > > >>>>>>>> description
> > > >> > > > >>>>>>>>>>>>>>>> in the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> "Atomic Checkpointing" section instead
> of
> > > >> only
> > > >> > > > >>>>>> mentioning it
> > > >> > > > >>>>>>>>>> in
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> "Compatibility, 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:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>
> > > >> > > > >>>>
> > > >> > > > >>
> > > >> > >
> > > >> >
> > > >>
> > >
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> It is a bit brittle, but it works.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Best,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Bruno
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford
> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for taking the time to review
> the
> > > >> KIP.
> > > >> > > > >> I'm
> > > >> > > > >>>> back
> > > >> > > > >>>>>> from
> > > >> > > > >>>>>>>>>>>>>>>> leave
> > > >> > > > >>>>>>>>>>>>>>>>>> now
> > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> intend to move this forwards as
> quickly
> > > as I
> > > >> > > > >> can.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Addressing your points:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Because flush() is part of the
> StateStore
> > > >> API,
> > > >> > > > >> it's
> > > >> > > > >>>>>> exposed
> > > >> > > > >>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>> custom
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors, which might be making
> calls to
> > > >> > > > >> flush().
> > > >> > > > >>>>>> This
> > > >> > > > >>>>>>>> was
> > > >> > > > >>>>>>>>>>>>>>>>>> actually
> > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> case in a few integration tests.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> To maintain as much compatibility as
> > > >> possible,
> > > >> > > > >> I'd
> > > >> > > > >>>>>> prefer
> > > >> > > > >>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>> make
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> this
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> an UnsupportedOperationException, as
> it
> > > will
> > > >> > > > >> cause
> > > >> > > > >>>>>>>> previously
> > > >> > > > >>>>>>>>>>>>>>>>>> working
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors to start throwing
> exceptions at
> > > >> > > > >> runtime.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that it doesn't make sense
> for it
> > > to
> > > >> > > > >> proxy
> > > >> > > > >>>>>>>> commit(),
> > > >> > > > >>>>>>>>>>>>>>>> though,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> as
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> that would cause it to violate the
> > > >> "StateStores
> > > >> > > > >>>> commit
> > > >> > > > >>>>>> only
> > > >> > > > >>>>>>>>>>>>>>>> when the
> > > >> > > > >>>>>>>>>>>>>>>>>>>> Task
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commits" rule.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Instead, I think we should make this a
> > > >> no-op.
> > > >> > > > >> That
> > > >> > > > >>>> way,
> > > >> > > > >>>>>>>>>>>>>>>> existing
> > > >> > > > >>>>>>>>>>>>>>>>>> user
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors will continue to work
> > > as-before,
> > > >> > > > >> without
> > > >> > > > >>>>>>>> violation
> > > >> > > > >>>>>>>>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> consistency that would be caused by
> > > >> premature
> > > >> > > > >>>>>> flush/commit
> > > >> > > > >>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>>>>> StateStore
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> data to disk.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> What do you think?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> As stated in the JavaDoc, when a
> > > StateStore
> > > >> > > > >>>>>> implementation
> > > >> > > > >>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, but is unable to
> estimate
> > > the
> > > >> > > > >>>>>> uncommitted
> > > >> > > > >>>>>>>>>>>> memory
> > > >> > > > >>>>>>>>>>>>>>>>>> usage,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> method will return -1.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> The intention here is to permit
> > > third-party
> > > >> > > > >>>>>> implementations
> > > >> > > > >>>>>>>>>>>>>>>> that may
> > > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> able to estimate memory usage.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has
> been
> > > >> > > > >> written to
> > > >> > > > >>>> the
> > > >> > > > >>>>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>> yet. I
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> thought
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> that was implied by "This method will
> > > >> return an
> > > >> > > > >>>>>>>> approximation
> > > >> > > > >>>>>>>>>>>>>>>> of the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memory
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> would be freed by the next call to
> {@link
> > > >> > > > >>>>>> #commit(Map)}"
> > > >> > > > >>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>> "@return
> > > >> > > > >>>>>>>>>>>>>>>>>>>> The
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> approximate size of all records
> awaiting
> > > >> {@link
> > > >> > > > >>>>>>>>>>>> #commit(Map)}",
> > > >> > > > >>>>>>>>>>>>>>>>>>>> however,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> can add it explicitly to the JavaDoc
> if
> > > you
> > > >> > > > >> think
> > > >> > > > >>>> this
> > > >> > > > >>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>> unclear?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I realise this is probably the most
> > > >> contentious
> > > >> > > > >>>> point
> > > >> > > > >>>>>> in my
> > > >> > > > >>>>>>>>>>>>>>>> design,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> I'm
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> open to changing it if I'm unable to
> > > >> convince
> > > >> > > > >> you
> > > >> > > > >>>> of
> > > >> > > > >>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>> benefits.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are
> > > >> directly
> > > >> > > > >>>> provided
> > > >> > > > >>>>>>>>>>>>>>>> StateStores
> > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> query, and it may be important for
> users
> > > to
> > > >> > > > >>>>>>>> programmatically
> > > >> > > > >>>>>>>>>>>>>>>> know
> > > >> > > > >>>>>>>>>>>>>>>>>>>> which
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> mode the StateStore is operating
> under. If
> > > >> we
> > > >> > > > >>>> simply
> > > >> > > > >>>>>>>> provide
> > > >> > > > >>>>>>>>>>>> an
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "eosEnabled" boolean (as used
> throughout
> > > the
> > > >> > > > >>>> internal
> > > >> > > > >>>>>>>> streams
> > > >> > > > >>>>>>>>>>>>>>>>>>>> engine), or
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> similar, then users will need to
> > > understand
> > > >> the
> > > >> > > > >>>>>> operation
> > > >> > > > >>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> consequences
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> of each available processing mode and
> how
> > > it
> > > >> > > > >>>> pertains
> > > >> > > > >>>>>> to
> > > >> > > > >>>>>>>>>> their
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Interactive Query users aren't the
> only
> > > >> people
> > > >> > > > >> that
> > > >> > > > >>>>>> care
> > > >> > > > >>>>>>>>>> about
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> processing.mode/IsolationLevel of a
> > > >> StateStore:
> > > >> > > > >>>>>>>> implementers
> > > >> > > > >>>>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>>> custom
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStores also need to understand
> the
> > > >> > > > >> behaviour
> > > >> > > > >>>>>> expected
> > > >> > > > >>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>> their
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> implementation. KIP-892 introduces
> some
> > > >> > > > >> assumptions
> > > >> > > > >>>>>> into
> > > >> > > > >>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>> Streams
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> Engine
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> about how StateStores operate under
> each
> > > >> > > > >> processing
> > > >> > > > >>>>>> mode,
> > > >> > > > >>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>> it's
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> important that custom implementations
> > > >> adhere to
> > > >> > > > >>>> those
> > > >> > > > >>>>>>>>>>>>>>>> assumptions in
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> order
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> to maintain the consistency
> guarantees.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> IsolationLevels provide a high-level
> > > >> contract
> > > >> > > > >> on
> > > >> > > > >>>> the
> > > >> > > > >>>>>>>>>> behaviour
> > > >> > > > >>>>>>>>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStore: a user knows that under
> > > >> > > > >> READ_COMMITTED,
> > > >> > > > >>>>>> they
> > > >> > > > >>>>>>>> will
> > > >> > > > >>>>>>>>>>>>>>>> see
> > > >> > > > >>>>>>>>>>>>>>>>>>>> writes
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> only after the Task has committed, and
> > > under
> > > >> > > > >>>>>>>> READ_UNCOMMITTED
> > > >> > > > >>>>>>>>>>>>>>>> they
> > > >> > > > >>>>>>>>>>>>>>>>>>>> will
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> see
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> writes immediately. No understanding
> of
> > > the
> > > >> > > > >>>> details of
> > > >> > > > >>>>>> each
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing.mode
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> is required, either for IQ users or
> > > >> StateStore
> > > >> > > > >>>>>>>> implementers.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> An argument can be made that these
> > > >> contractual
> > > >> > > > >>>>>> guarantees
> > > >> > > > >>>>>>>> can
> > > >> > > > >>>>>>>>>>>>>>>> simply
> > > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> documented for the processing.mode
> (i.e.
> > > >> that
> > > >> > > > >>>>>> exactly-once
> > > >> > > > >>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> exactly-once-v2 behave like
> READ_COMMITTED
> > > >> and
> > > >> > > > >>>>>>>> at-least-once
> > > >> > > > >>>>>>>>>>>>>>>> behaves
> > > >> > > > >>>>>>>>>>>>>>>>>>>> like
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are
> several
> > > >> small
> > > >> > > > >>>> issues
> > > >> > > > >>>>>> with
> > > >> > > > >>>>>>>>>>>>>>>> this I'd
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> prefer
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> to avoid:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - Where would we document
> these
> > > >> > > > >> contracts,
> > > >> > > > >>>> in
> > > >> > > > >>>>>> a way
> > > >> > > > >>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>> difficult
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          for users/implementers to
> > > >> miss/ignore?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It's not clear to users
> that
> > > the
> > > >> > > > >>>> processing
> > > >> > > > >>>>>> mode
> > > >> > > > >>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>> communicating
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          an expectation of read
> isolation,
> > > >> > > > >> unless
> > > >> > > > >>>> they
> > > >> > > > >>>>>> read
> > > >> > > > >>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> documentation. Users
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          rarely consult documentation
> > > unless
> > > >> > > > >> they
> > > >> > > > >>>> feel
> > > >> > > > >>>>>> they
> > > >> > > > >>>>>>>>>> need
> > > >> > > > >>>>>>>>>>>>>>>> to, so
> > > >> > > > >>>>>>>>>>>>>>>>>>>> it's
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> likely
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          this detail would get missed
> by
> > > >> many
> > > >> > > > >> users.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It tightly couples
> processing
> > > >> modes
> > > >> > > > >> to
> > > >> > > > >>>> read
> > > >> > > > >>>>>>>>>> isolation.
> > > >> > > > >>>>>>>>>>>>>>>> Adding
> > > >> > > > >>>>>>>>>>>>>>>>>>>> new
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          processing modes, or
> changing the
> > > >> read
> > > >> > > > >>>>>> isolation of
> > > >> > > > >>>>>>>>>>>>>>>> existing
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          modes would be
> > > >> difficult/impossible.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Ultimately, the cost of introducing
> > > >> > > > >>>> IsolationLevels is
> > > >> > > > >>>>>>>> just a
> > > >> > > > >>>>>>>>>>>>>>>> single
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> method, since we re-use the existing
> > > >> > > > >> IsolationLevel
> > > >> > > > >>>>>> enum
> > > >> > > > >>>>>>>> from
> > > >> > > > >>>>>>>>>>>>>>>> Kafka.
> > > >> > > > >>>>>>>>>>>>>>>>>>>> This
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> gives us a clear place to document the
> > > >> > > > >> contractual
> > > >> > > > >>>>>>>> guarantees
> > > >> > > > >>>>>>>>>>>>>>>>>> expected
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> of/provided by StateStores, that is
> > > >> accessible
> > > >> > > > >>>> both by
> > > >> > > > >>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>> StateStore
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> itself, and by IQ users.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> (Writing this I've just realised that
> the
> > > >> > > > >>>> StateStore
> > > >> > > > >>>>>> and IQ
> > > >> > > > >>>>>>>>>>>>>>>> APIs
> > > >> > > > >>>>>>>>>>>>>>>>>>>> actually
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> don't provide access to
> StateStoreContext
> > > >> that
> > > >> > > > >> IQ
> > > >> > > > >>>> users
> > > >> > > > >>>>>>>> would
> > > >> > > > >>>>>>>>>>>>>>>> have
> > > >> > > > >>>>>>>>>>>>>>>>>>>> direct
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> access to... Perhaps StateStore should
> > > >> expose
> > > >> > > > >>>>>>>>>> isolationLevel()
> > > >> > > > >>>>>>>>>>>>>>>>>> itself
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> too?)
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 4.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the
> > > >> metrics
> > > >> > > > >>>> in-place
> > > >> > > > >>>>>>>>>>>>>>>> either, as
> > > >> > > > >>>>>>>>>>>>>>>>>>>> it's a
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> backwards incompatible change. My
> concern
> > > is
> > > >> > > > >> that,
> > > >> > > > >>>> if
> > > >> > > > >>>>>> we
> > > >> > > > >>>>>>>>>> leave
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> existing
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics in place, they will be
> > > >> > > > >> confusing to
> > > >> > > > >>>>>> users.
> > > >> > > > >>>>>>>>>>>>>>>> Right
> > > >> > > > >>>>>>>>>>>>>>>>>> now,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics record explicit
> flushes to
> > > >> > > > >> disk,
> > > >> > > > >>>> but
> > > >> > > > >>>>>> under
> > > >> > > > >>>>>>>>>>>>>>>> KIP-892,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> even
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> a
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() will not explicitly flush
> data to
> > > >> > > > >> disk -
> > > >> > > > >>>>>> RocksDB
> > > >> > > > >>>>>>>>>> will
> > > >> > > > >>>>>>>>>>>>>>>>>> decide
> > > >> > > > >>>>>>>>>>>>>>>>>>>> on
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> when to flush memtables to disk
> itself.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> If we keep the existing "flush"
> metrics,
> > > >> we'd
> > > >> > > > >> have
> > > >> > > > >>>> two
> > > >> > > > >>>>>>>>>>>> options,
> > > >> > > > >>>>>>>>>>>>>>>>>> which
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> both
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> seem pretty bad to me:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          1. Have them record calls to
> > > >> commit(),
> > > >> > > > >>>> which
> > > >> > > > >>>>>> would
> > > >> > > > >>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>>>> misleading, as
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          data is no longer explicitly
> > > >> "flushed"
> > > >> > > > >> to
> > > >> > > > >>>> disk
> > > >> > > > >>>>>> by
> > > >> > > > >>>>>>>> this
> > > >> > > > >>>>>>>>>>>>>>>> call.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          2. Have them record nothing
> at
> > > all,
> > > >> > > > >> which
> > > >> > > > >>>> is
> > > >> > > > >>>>>>>>>> equivalent
> > > >> > > > >>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>> removing
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          metrics, except that users
> will
> > > see
> > > >> > the
> > > >> > > > >>>> metric
> > > >> > > > >>>>>>>> still
> > > >> > > > >>>>>>>>>>>>>>>> exists and
> > > >> > > > >>>>>>>>>>>>>>>>>>>> so
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> assume
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          that the metric is correct,
> and
> > > >> that
> > > >> > > > >>>> there's a
> > > >> > > > >>>>>>>> problem
> > > >> > > > >>>>>>>>>>>>>>>> with
> > > >> > > > >>>>>>>>>>>>>>>>>> their
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> system
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          when there isn't.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that removing them is also a
> bad
> > > >> > > > >> solution,
> > > >> > > > >>>> and
> > > >> > > > >>>>>> I'd
> > > >> > > > >>>>>>>>>>>>>>>> like some
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> guidance on the best path forward
> here.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 5.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Position files are updated on every
> write
> > > >> to a
> > > >> > > > >>>>>> StateStore.
> > > >> > > > >>>>>>>>>>>>>>>> Since our
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> writes
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> are now buffered until commit(), we
> can't
> > > >> > > > >> update
> > > >> > > > >>>> the
> > > >> > > > >>>>>>>> Position
> > > >> > > > >>>>>>>>>>>>>>>> file
> > > >> > > > >>>>>>>>>>>>>>>>>>>> until
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() has been called, otherwise it
> > > >> would be
> > > >> > > > >>>>>>>> inconsistent
> > > >> > > > >>>>>>>>>>>>>>>> with
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> data
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> in the event of a rollback.
> Consequently,
> > > we
> > > >> > > > >> need
> > > >> > > > >>>> to
> > > >> > > > >>>>>> manage
> > > >> > > > >>>>>>>>>>>>>>>> these
> > > >> > > > >>>>>>>>>>>>>>>>>>>> offsets
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> the same way we manage the checkpoint
> > > >> offsets,
> > > >> > > > >> and
> > > >> > > > >>>>>> ensure
> > > >> > > > >>>>>>>>>>>>>>>> they're
> > > >> > > > >>>>>>>>>>>>>>>>>> only
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> written on commit().
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 6.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Agreed, although I'm not exactly sure
> yet
> > > >> what
> > > >> > > > >>>> tests to
> > > >> > > > >>>>>>>>>> write.
> > > >> > > > >>>>>>>>>>>>>>>> How
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> explicit
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> do we need to be here in the KIP?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is
> > > >> designed
> > > >> > > > >> to be
> > > >> > > > >>>>>>>> seamless,
> > > >> > > > >>>>>>>>>>>>>>>> and we
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> should
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> definitely add some tests around that.
> > > >> > > > >> Downgrade,
> > > >> > > > >>>> it
> > > >> > > > >>>>>>>>>>>>>>>> transpires,
> > > >> > > > >>>>>>>>>>>>>>>>>> isn't
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> currently possible, as the extra
> column
> > > >> family
> > > >> > > > >> for
> > > >> > > > >>>>>> offset
> > > >> > > > >>>>>>>>>>>>>>>> storage is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> incompatible with the pre-KIP-892
> > > >> > > > >> implementation:
> > > >> > > > >>>> when
> > > >> > > > >>>>>> you
> > > >> > > > >>>>>>>>>>>>>>>> open a
> > > >> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> database, you must open all available
> > > column
> > > >> > > > >>>> families
> > > >> > > > >>>>>> or
> > > >> > > > >>>>>>>>>>>>>>>> receive an
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> error.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> What currently happens on downgrade is
> > > that
> > > >> it
> > > >> > > > >>>>>> attempts to
> > > >> > > > >>>>>>>>>>>>>>>> open the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> store,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> throws an error about the offsets
> column
> > > >> > > > >> family not
> > > >> > > > >>>>>> being
> > > >> > > > >>>>>>>>>>>>>>>> opened,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> which
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> triggers a wipe and rebuild of the
> Task.
> > > >> Given
> > > >> > > > >> that
> > > >> > > > >>>>>>>>>> downgrades
> > > >> > > > >>>>>>>>>>>>>>>>>> should
> > > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> uncommon, I think this is acceptable
> > > >> > > > >> behaviour, as
> > > >> > > > >>>> the
> > > >> > > > >>>>>>>>>>>>>>>> end-state is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> consistent, even if it results in an
> > > >> > > > >> undesirable
> > > >> > > > >>>> state
> > > >> > > > >>>>>>>>>>>> restore.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Should I document the
> upgrade/downgrade
> > > >> > > > >> behaviour
> > > >> > > > >>>>>>>> explicitly
> > > >> > > > >>>>>>>>>>>>>>>> in the
> > > >> > > > >>>>>>>>>>>>>>>>>>>> KIP?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> --
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno
> > > Cadonna
> > > >> <
> > > >> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > > >> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Nick!
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates!
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Why does StateStore#flush() default
> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > StateStore#commit(Collections.emptyMap())?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Since calls to flush() will not exist
> > > >> anymore
> > > >> > > > >>>> after
> > > >> > > > >>>>>> this
> > > >> > > > >>>>>>>> KIP
> > > >> > > > >>>>>>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> released, I would rather throw an
> > > >> unsupported
> > > >> > > > >>>>>> operation
> > > >> > > > >>>>>>>>>>>>>>>> exception
> > > >> > > > >>>>>>>>>>>>>>>>>> by
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> default.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> When would a state store return -1
> from
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > StateStore#approximateNumUncommittedBytes()
> > > >> > > > >> while
> > > >> > > > >>>>>> being
> > > >> > > > >>>>>>>>>>>>>>>>>>>> transactional?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Wouldn't
> > > >> > > > >>>> StateStore#approximateNumUncommittedBytes()
> > > >> > > > >>>>>> also
> > > >> > > > >>>>>>>>>>>>>>>> return 0
> > > >> > > > >>>>>>>>>>>>>>>>>> if
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the state store is transactional but
> > > >> nothing
> > > >> > > > >> has
> > > >> > > > >>>> been
> > > >> > > > >>>>>>>>>> written
> > > >> > > > >>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> state store yet?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Sorry for bringing this up again.
> Does
> > > this
> > > >> > > > >> KIP
> > > >> > > > >>>> really
> > > >> > > > >>>>>>>> need
> > > >> > > > >>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>> introduce
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> > > >> > > > >>>> StateStoreContext
> > > >> > > > >>>>>> has
> > > >> > > > >>>>>>>>>>>>>>>> already
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> appConfigs() which basically exposes
> the
> > > >> same
> > > >> > > > >>>>>> information,
> > > >> > > > >>>>>>>>>>>>>>>> i.e., if
> > > >> > > > >>>>>>>>>>>>>>>>>>>> EOS
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> is enabled or not.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> In one of your previous e-mails you
> > > wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> "My idea was to try to keep the
> > > StateStore
> > > >> > > > >>>> interface
> > > >> > > > >>>>>> as
> > > >> > > > >>>>>>>>>>>>>>>> loosely
> > > >> > > > >>>>>>>>>>>>>>>>>>>> coupled
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible,
> to
> > > >> give
> > > >> > > > >>>>>> implementers
> > > >> > > > >>>>>>>>>>>> more
> > > >> > > > >>>>>>>>>>>>>>>>>>>> freedom,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and reduce the amount of internal
> > > knowledge
> > > >> > > > >>>> required."
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> While I understand the intent, I
> doubt
> > > >> that it
> > > >> > > > >>>>>> decreases
> > > >> > > > >>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>> coupling of
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> a StateStore interface and the
> Streams
> > > >> engine.
> > > >> > > > >>>>>>>>>> READ_COMMITTED
> > > >> > > > >>>>>>>>>>>>>>>> only
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> applies to IQ but not to reads by
> > > >> processors.
> > > >> > > > >>>> Thus,
> > > >> > > > >>>>>>>>>>>>>>>> implementers
> > > >> > > > >>>>>>>>>>>>>>>>>>>> need to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand how Streams accesses the
> state
> > > >> > > > >> stores.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> I would like to hear what others
> think
> > > >> about
> > > >> > > > >> this.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 4.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Great exposing new metrics for
> > > >> transactional
> > > >> > > > >> state
> > > >> > > > >>>>>> stores!
> > > >> > > > >>>>>>>>>>>>>>>>>> However, I
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> would prefer to add new metrics and
> > > >> deprecate
> > > >> > > > >> (in
> > > >> > > > >>>> the
> > > >> > > > >>>>>>>> docs)
> > > >> > > > >>>>>>>>>>>>>>>> the 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 also need to
> adapt/add
> > > >> > > > >> integration
> > > >> > > > >>>>>> tests
> > > >> > > > >>>>>>>>>>>>>>>> besides
> > > >> > > > >>>>>>>>>>>>>>>>>> unit
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> tests. Additionally, we probably need
> > > >> > > > >> integration
> > > >> > > > >>>> or
> > > >> > > > >>>>>>>> system
> > > >> > > > >>>>>>>>>>>>>>>> tests
> > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> verify that upgrades and downgrades
> > > between
> > > >> > > > >>>>>> transactional
> > > >> > > > >>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> non-transactional state stores work
> as
> > > >> > > > >> expected.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford
> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> One more thing: I noted John's
> > > suggestion
> > > >> in
> > > >> > > > >> the
> > > >> > > > >>>> KIP,
> > > >> > > > >>>>>>>> under
> > > >> > > > >>>>>>>>>>>>>>>>>>>> "Rejected
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> Alternatives". I still think it's an
> > > idea
> > > >> > > > >> worth
> > > >> > > > >>>>>> pursuing,
> > > >> > > > >>>>>>>>>>>>>>>> but I
> > > >> > > > >>>>>>>>>>>>>>>>>>>> believe
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> that it's out of the scope of this
> KIP,
> > > >> > > > >> because
> > > >> > > > >>>> it
> > > >> > > > >>>>>>>> solves a
> > > >> > > > >>>>>>>>>>>>>>>>>>>> different
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> set
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> of problems to this KIP, and the
> scope
> > > of
> > > >> > > > >> this
> > > >> > > > >>>> one
> > > >> > > > >>>>>> has
> > > >> > > > >>>>>>>>>>>>>>>> already
> > > >> > > > >>>>>>>>>>>>>>>>>> grown
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> quite
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> large!
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick
> > > >> Telford <
> > > >> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP (
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>
> > > >> > > > >>>>
> > > >> > > > >>
> > > >> > >
> > > >> >
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> )
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with the latest changes; mostly
> > > bringing
> > > >> > > > >> back
> > > >> > > > >>>>>> "Atomic
> > > >> > > > >>>>>>>>>>>>>>>>>>>> Checkpointing"
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> (for
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> what feels like the 10th time!). I
> > > think
> > > >> > > > >> the one
> > > >> > > > >>>>>> thing
> > > >> > > > >>>>>>>>>>>>>>>> missing is
> > > >> > > > >>>>>>>>>>>>>>>>>>>> some
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> changes to metrics (notably the
> store
> > > >> > > > >> "flush"
> > > >> > > > >>>>>> metrics
> > > >> > > > >>>>>>>> will
> > > >> > > > >>>>>>>>>>>>>>>> need
> > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> renamed to "commit").
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> The reason I brought back Atomic
> > > >> > > > >> Checkpointing
> > > >> > > > >>>> was
> > > >> > > > >>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> decouple
> > > >> > > > >>>>>>>>>>>>>>>>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flush
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from store commit. This is
> important,
> > > >> > > > >> because
> > > >> > > > >>>> with
> > > >> > > > >>>>>>>>>>>>>>>> Transactional
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> StateStores, we now need to call
> > > "flush"
> > > >> on
> > > >> > > > >>>> *every*
> > > >> > > > >>>>>> Task
> > > >> > > > >>>>>>>>>>>>>>>> commit,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> just when the StateStore is
> closing,
> > > >> > > > >> otherwise
> > > >> > > > >>>> our
> > > >> > > > >>>>>>>>>>>>>>>> transaction
> > > >> > > > >>>>>>>>>>>>>>>>>>>> buffer
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> will
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> never be written and persisted,
> instead
> > > >> > > > >> growing
> > > >> > > > >>>>>>>> unbounded!
> > > >> > > > >>>>>>>>>>>> I
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> experimented
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with some simple solutions, like
> > > forcing
> > > >> a
> > > >> > > > >> store
> > > >> > > > >>>>>> flush
> > > >> > > > >>>>>>>>>>>>>>>> whenever
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer was likely to
> exceed
> > > >> its
> > > >> > > > >>>>>> configured
> > > >> > > > >>>>>>>>>>>>>>>> size, but
> > > >> > > > >>>>>>>>>>>>>>>>>>>> this
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> brittle: it prevented the
> transaction
> > > >> buffer
> > > >> > > > >>>> from
> > > >> > > > >>>>>> being
> > > >> > > > >>>>>>>>>>>>>>>>>> configured
> > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> unbounded, and it still would have
> > > >> required
> > > >> > > > >>>> explicit
> > > >> > > > >>>>>>>>>>>>>>>> flushes of
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> RocksDB,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yielding sub-optimal performance
> and
> > > >> memory
> > > >> > > > >>>>>> utilization.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to
> be the
> > > >> > > > >> "right"
> > > >> > > > >>>> way
> > > >> > > > >>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> resolve
> > > >> > > > >>>>>>>>>>>>>>>>>> this
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> problem. By ensuring that the
> changelog
> > > >> > > > >> offsets
> > > >> > > > >>>> that
> > > >> > > > >>>>>>>>>>>>>>>> correspond
> > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> most
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> recently written records are always
> > > >> > > > >> atomically
> > > >> > > > >>>>>> written
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (by writing them to the same
> > > transaction
> > > >> > > > >>>> buffer),
> > > >> > > > >>>>>> we can
> > > >> > > > >>>>>>>>>>>>>>>> avoid
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> forcibly
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to
> disk,
> > > >> > > > >> letting
> > > >> > > > >>>>>> RocksDB
> > > >> > > > >>>>>>>>>>>>>>>> flush
> > > >> > > > >>>>>>>>>>>>>>>>>> them
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> only
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when necessary, without losing any
> of
> > > our
> > > >> > > > >>>>>> consistency
> > > >> > > > >>>>>>>>>>>>>>>> guarantees.
> > > >> > > > >>>>>>>>>>>>>>>>>>>> See
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> updated KIP for more info.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I have fully implemented these
> changes,
> > > >> > > > >>>> although I'm
> > > >> > > > >>>>>>>> still
> > > >> > > > >>>>>>>>>>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> entirely
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> happy with the implementation for
> > > >> segmented
> > > >> > > > >>>>>> StateStores,
> > > >> > > > >>>>>>>>>> so
> > > >> > > > >>>>>>>>>>>>>>>> I
> > > >> > > > >>>>>>>>>>>>>>>>>> plan
> > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> refactor that. Despite that, all
> tests
> > > >> > > > >> pass. If
> > > >> > > > >>>>>> you'd
> > > >> > > > >>>>>>>> like
> > > >> > > > >>>>>>>>>>>>>>>> to try
> > > >> > > > >>>>>>>>>>>>>>>>>>>> out
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> or
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> review this highly experimental and
> > > >> > > > >> incomplete
> > > >> > > > >>>>>> branch,
> > > >> > > > >>>>>>>>>> it's
> > > >> > > > >>>>>>>>>>>>>>>>>>>> available
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> here:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>
> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > > >> > > > >>>>>>>> .
> > > >> > > > >>>>>>>>>>>>>>>> Note:
> > > >> > > > >>>>>>>>>>>>>>>>>>>> it's
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> built
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a
> > > >> stable
> > > >> > > > >> base
> > > >> > > > >>>> to
> > > >> > > > >>>>>> build
> > > >> > > > >>>>>>>>>>>>>>>> and test
> > > >> > > > >>>>>>>>>>>>>>>>>>>> it
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> on,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples
> > > >> > > > >> comparisons
> > > >> > > > >>>> in a
> > > >> > > > >>>>>>>> live
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> environment. I
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> plan to rebase it against trunk
> once
> > > it's
> > > >> > > > >> nearer
> > > >> > > > >>>>>>>>>> completion
> > > >> > > > >>>>>>>>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>> has
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> been
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> proven on our main application.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I would really appreciate help in
> > > >> reviewing
> > > >> > > > >> and
> > > >> > > > >>>>>> testing:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and
> > > >> Window)
> > > >> > > > >>>> stores
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Global stores
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> As I do not currently use either of
> > > >> these,
> > > >> > > > >> so my
> > > >> > > > >>>>>> primary
> > > >> > > > >>>>>>>>>>>>>>>> test
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> environment
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> doesn't test these areas.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I'm going on Parental Leave
> starting
> > > next
> > > >> > > > >> week
> > > >> > > > >>>> for
> > > >> > > > >>>>>> a few
> > > >> > > > >>>>>>>>>>>>>>>> weeks,
> > > >> > > > >>>>>>>>>>>>>>>>>> so
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> will
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> not have time to move this forward
> > > until
> > > >> > > > >> late
> > > >> > > > >>>>>> August.
> > > >> > > > >>>>>>>> That
> > > >> > > > >>>>>>>>>>>>>>>> said,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> your
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> feedback is welcome and
> appreciated, I
> > > >> just
> > > >> > > > >>>> won't be
> > > >> > > > >>>>>>>> able
> > > >> > > > >>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>> respond
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> as
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> quickly as usual.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick
> > > >> Telford <
> > > >> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, that's correct, although the
> > > impact
> > > >> > > > >> on IQ
> > > >> > > > >>>> is
> > > >> > > > >>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>> something
> > > >> > > > >>>>>>>>>>>>>>>>>> I
> > > >> > > > >>>>>>>>>>>>>>>>>>>> had
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> considered.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the
> > > state
> > > >> > > > >> store
> > > >> > > > >>>>>> from the
> > > >> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and
> > > >> writing
> > > >> > > > >> the
> > > >> > > > >>>>>>>> checkpoint
> > > >> > > > >>>>>>>>>>>>>>>> (thus,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured
> amount
> > > >> of
> > > >> > > > >> data
> > > >> > > > >>>>>> and/or
> > > >> > > > >>>>>>>>>>>>>>>> number of
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are
> you
> > > >> > > > >> suggesting
> > > >> > > > >>>>>> that we
> > > >> > > > >>>>>>>>>>>>>>>> add an
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> additional
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> config for the max number of
> commit
> > > >> > > > >> intervals
> > > >> > > > >>>>>> between
> > > >> > > > >>>>>>>>>>>>>>>>>> checkpoints?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> That
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> way, we would checkpoint *either*
> when
> > > >> the
> > > >> > > > >>>>>> transaction
> > > >> > > > >>>>>>>>>>>>>>>> buffers
> > > >> > > > >>>>>>>>>>>>>>>>>> are
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> nearly
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain
> number
> > > of
> > > >> > > > >> commit
> > > >> > > > >>>>>>>> intervals
> > > >> > > > >>>>>>>>>>>>>>>> have
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> elapsed,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> whichever comes first?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> That certainly seems reasonable,
> > > >> although
> > > >> > > > >> this
> > > >> > > > >>>>>>>> re-ignites
> > > >> > > > >>>>>>>>>>>>>>>> an
> > > >> > > > >>>>>>>>>>>>>>>>>>>> earlier
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> debate about whether a config
> should
> > > be
> > > >> > > > >>>> measured in
> > > >> > > > >>>>>>>>>>>>>>>> "number of
> > > >> > > > >>>>>>>>>>>>>>>>>>>> commit
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> intervals", instead of just an
> > > absolute
> > > >> > > > >> time.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue
> is
> > > the
> > > >> > > > >> reason
> > > >> > > > >>>> I
> > > >> > > > >>>>>> was
> > > >> > > > >>>>>>>>>>>>>>>> pursuing
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Atomic
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples
> memtable
> > > >> > > > >> flush
> > > >> > > > >>>> from
> > > >> > > > >>>>>>>>>>>>>>>> checkpointing,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> which
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> enables us to just checkpoint on
> every
> > > >> > > > >> commit
> > > >> > > > >>>>>> without
> > > >> > > > >>>>>>>> any
> > > >> > > > >>>>>>>>>>>>>>>>>>>> performance
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is
> > > >> definitely
> > > >> > > > >> the
> > > >> > > > >>>>>> "best"
> > > >> > > > >>>>>>>>>>>>>>>> solution,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> but
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm not
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sure if this is enough to bring it
> > > back
> > > >> > > > >> into
> > > >> > > > >>>> this
> > > >> > > > >>>>>> KIP.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm currently working on moving
> all
> > > the
> > > >> > > > >>>>>> transactional
> > > >> > > > >>>>>>>>>>>> logic
> > > >> > > > >>>>>>>>>>>>>>>>>>>> directly
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> into
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does
> away
> > > >> with
> > > >> > > > >> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore#newTransaction
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> method, and reduces the number of
> new
> > > >> > > > >> classes
> > > >> > > > >>>>>>>> introduced,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> significantly
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reducing the complexity. If it
> works,
> > > >> and
> > > >> > > > >> the
> > > >> > > > >>>>>>>> complexity
> > > >> > > > >>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> drastically
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reduced, I may try bringing back
> > > Atomic
> > > >> > > > >>>> Checkpoints
> > > >> > > > >>>>>>>> into
> > > >> > > > >>>>>>>>>>>>>>>> this
> > > >> > > > >>>>>>>>>>>>>>>>>> KIP.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno
> > > >> Cadonna
> > > >> > > > >> <
> > > >> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very
> > > >> interesting!
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> As far as I understand, you want
> to
> > > >> > > > >> atomically
> > > >> > > > >>>>>> update
> > > >> > > > >>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>> state
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> from the transaction buffer,
> flush
> > > the
> > > >> > > > >>>> memtable
> > > >> > > > >>>>>> of a
> > > >> > > > >>>>>>>>>>>> state
> > > >> > > > >>>>>>>>>>>>>>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write the checkpoint not after
> the
> > > >> commit
> > > >> > > > >> time
> > > >> > > > >>>>>> elapsed
> > > >> > > > >>>>>>>>>>>> but
> > > >> > > > >>>>>>>>>>>>>>>>>> after
> > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reached a size
> > > that
> > > >> > > > >> would
> > > >> > > > >>>> lead
> > > >> > > > >>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> exceeding
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > statestore.transaction.buffer.max.bytes
> > > >> > > > >>>> before the
> > > >> > > > >>>>>>>> next
> > > >> > > > >>>>>>>>>>>>>>>> commit
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> interval
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> ends.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction
> > > would
> > > >> > > > >> commit
> > > >> > > > >>>>>> every
> > > >> > > > >>>>>>>>>>>>>>>> commit
> > > >> > > > >>>>>>>>>>>>>>>>>>>> interval
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store will only be
> > > atomically
> > > >> > > > >>>> updated
> > > >> > > > >>>>>>>> roughly
> > > >> > > > >>>>>>>>>>>>>>>> every
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> statestore.transaction.buffer.max.bytes of
> > > >> > > > >>>> data.
> > > >> > > > >>>>>> Also
> > > >> > > > >>>>>>>> IQ
> > > >> > > > >>>>>>>>>>>>>>>> would
> > > >> > > > >>>>>>>>>>>>>>>>>>>> then
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> only
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see new data roughly every
> > > >> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> After a failure the state store
> needs
> > > >> to
> > > >> > > > >>>> restore
> > > >> > > > >>>>>> up to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> statestore.transaction.buffer.max.bytes.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Is this correct?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating
> the
> > > >> state
> > > >> > > > >> store
> > > >> > > > >>>>>> from
> > > >> > > > >>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and
> > > >> writing
> > > >> > > > >> the
> > > >> > > > >>>>>>>> checkpoint
> > > >> > > > >>>>>>>>>>>>>>>> (thus,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured
> amount
> > > >> of
> > > >> > > > >> data
> > > >> > > > >>>>>> and/or
> > > >> > > > >>>>>>>>>>>>>>>> number of
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we
> would
> > > have
> > > >> > > > >> the
> > > >> > > > >>>> same
> > > >> > > > >>>>>> delay
> > > >> > > > >>>>>>>>>>>> for
> > > >> > > > >>>>>>>>>>>>>>>>>>>> records
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ
> > > >> because
> > > >> > > > >> both
> > > >> > > > >>>>>> would
> > > >> > > > >>>>>>>>>>>>>>>> appear
> > > >> > > > >>>>>>>>>>>>>>>>>> when
> > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka transaction is committed.
> > > >> However,
> > > >> > > > >>>> after a
> > > >> > > > >>>>>>>> failure
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > > >> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it might restore data that is
> already
> > > >> in
> > > >> > > > >> the
> > > >> > > > >>>> state
> > > >> > > > >>>>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>>>> because
> > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last
> > > stable
> > > >> > > > >> offset
> > > >> > > > >>>>>> (i.e.
> > > >> > > > >>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>> last
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> committed
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset) of the changelog topics.
> > > >> Restoring
> > > >> > > > >>>> data
> > > >> > > > >>>>>> that
> > > >> > > > >>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>> already
> > > >> > > > >>>>>>>>>>>>>>>>>>>> in
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos
> > > >> should
> > > >> > > > >> not
> > > >> > > > >>>>>> violated.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> This solution needs at least one
> new
> > > >> > > > >> config to
> > > >> > > > >>>>>> specify
> > > >> > > > >>>>>>>>>>>>>>>> when a
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> should be written.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> A small correction to your
> previous
> > > >> e-mail
> > > >> > > > >>>> that
> > > >> > > > >>>>>> does
> > > >> > > > >>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>> change
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> anything
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> you said: Under alos the default
> > > commit
> > > >> > > > >>>> interval
> > > >> > > > >>>>>> is 30
> > > >> > > > >>>>>>>>>>>>>>>> seconds,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> five
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> seconds.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford
> > > wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've begun performance testing
> my
> > > >> branch
> > > >> > > > >> on
> > > >> > > > >>>> our
> > > >> > > > >>>>>>>> staging
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> environment,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting it through its paces in
> our
> > > >> > > > >>>> non-trivial
> > > >> > > > >>>>>>>>>>>>>>>> application.
> > > >> > > > >>>>>>>>>>>>>>>>>> I'm
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing the same increased
> flush
> > > >> rate
> > > >> > > > >> that
> > > >> > > > >>>> we
> > > >> > > > >>>>>> saw
> > > >> > > > >>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>> last
> > > >> > > > >>>>>>>>>>>>>>>>>>>> time
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> attempted to use a version of
> this
> > > >> KIP,
> > > >> > > > >> but
> > > >> > > > >>>> this
> > > >> > > > >>>>>>>> time,
> > > >> > > > >>>>>>>>>> I
> > > >> > > > >>>>>>>>>>>>>>>>>> think I
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> know
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> why.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892,
> StreamTask#postCommit,
> > > >> > > > >> which is
> > > >> > > > >>>>>> called
> > > >> > > > >>>>>>>> at
> > > >> > > > >>>>>>>>>>>>>>>> the end
> > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Task
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit process, has the
> following
> > > >> > > > >> behaviour:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under ALOS:
> checkpoint
> > > >> the
> > > >> > > > >> state
> > > >> > > > >>>>>> stores.
> > > >> > > > >>>>>>>>>> This
> > > >> > > > >>>>>>>>>>>>>>>>>> includes
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            flushing memtables in
> > > >> RocksDB.
> > > >> > > > >>>> This is
> > > >> > > > >>>>>>>>>>>> acceptable
> > > >> > > > >>>>>>>>>>>>>>>>>>>> because the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> default
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit.interval.ms
> is 5
> > > >> > > > >> seconds,
> > > >> > > > >>>> so
> > > >> > > > >>>>>>>> forcibly
> > > >> > > > >>>>>>>>>>>>>>>> flushing
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> every 5
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            seconds is
> acceptable for
> > > >> most
> > > >> > > > >>>>>>>> applications.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under EOS:
> > > checkpointing
> > > >> is
> > > >> > > > >> not
> > > >> > > > >>>> done,
> > > >> > > > >>>>>>>>>> *unless*
> > > >> > > > >>>>>>>>>>>>>>>> it's
> > > >> > > > >>>>>>>>>>>>>>>>>>>> being
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> forced, due
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to e.g. the Task
> closing
> > > or
> > > >> > > > >> being
> > > >> > > > >>>>>> revoked.
> > > >> > > > >>>>>>>>>> This
> > > >> > > > >>>>>>>>>>>>>>>> means
> > > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> under
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> normal
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing
> conditions,
> > > the
> > > >> > > > >> state
> > > >> > > > >>>> stores
> > > >> > > > >>>>>>>> will
> > > >> > > > >>>>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpointed,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            not have memtables
> > > flushed
> > > >> at
> > > >> > > > >> all ,
> > > >> > > > >>>>>> unless
> > > >> > > > >>>>>>>>>>>> RocksDB
> > > >> > > > >>>>>>>>>>>>>>>>>>>> decides to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush them on
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            its own.
> Checkpointing
> > > >> stores
> > > >> > > > >> and
> > > >> > > > >>>>>>>>>> force-flushing
> > > >> > > > >>>>>>>>>>>>>>>> their
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            done when a Task is
> being
> > > >> > > > >> closed.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to
> > > checkpoint
> > > >> > > > >>>> stores on
> > > >> > > > >>>>>> at
> > > >> > > > >>>>>>>>>>>> least
> > > >> > > > >>>>>>>>>>>>>>>>>> *some*
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> normal
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task commits, in order to write
> the
> > > >> > > > >> RocksDB
> > > >> > > > >>>>>>>> transaction
> > > >> > > > >>>>>>>>>>>>>>>>>> buffers
> > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the
> > > >> offsets
> > > >> > > > >> are
> > > >> > > > >>>>>> synced to
> > > >> > > > >>>>>>>>>>>>>>>> disk to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> prevent
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> restores from getting out of
> hand.
> > > >> > > > >>>> Consequently,
> > > >> > > > >>>>>> my
> > > >> > > > >>>>>>>>>>>>>>>> current
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every*
> > > Task
> > > >> > > > >> commit,
> > > >> > > > >>>>>> which
> > > >> > > > >>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>> far too
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> frequent.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This causes checkpoints every
> 10,000
> > > >> > > > >> records,
> > > >> > > > >>>>>> which
> > > >> > > > >>>>>>>> is
> > > >> > > > >>>>>>>>>> a
> > > >> > > > >>>>>>>>>>>>>>>>>> change
> > > >> > > > >>>>>>>>>>>>>>>>>>>> in
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
> > > >> > > > >> performance
> > > >> > > > >>>>>> problems
> > > >> > > > >>>>>>>> for
> > > >> > > > >>>>>>>>>>>>>>>> some
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> applications.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm looking into possible
> solutions,
> > > >> and
> > > >> > > > >> I'm
> > > >> > > > >>>>>>>> currently
> > > >> > > > >>>>>>>>>>>>>>>> leaning
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> towards
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> using the
> > > >> > > > >>>> statestore.transaction.buffer.max.bytes
> > > >> > > > >>>>>>>>>>>>>>>>>> configuration
> > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are
> likely
> > > to
> > > >> > > > >>>> exceed it.
> > > >> > > > >>>>>>>> This
> > > >> > > > >>>>>>>>>>>>>>>> would
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> complement the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
> > > >> > > > >> functionality
> > > >> > > > >>>> that
> > > >> > > > >>>>>> this
> > > >> > > > >>>>>>>>>>>>>>>>>>>> configuration
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Currently, we use
> > > >> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> force an
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            early Task commit if
> > > >> > processing
> > > >> > > > >>>> more
> > > >> > > > >>>>>>>> records
> > > >> > > > >>>>>>>>>>>> would
> > > >> > > > >>>>>>>>>>>>>>>>>> cause
> > > >> > > > >>>>>>>>>>>>>>>>>>>> our
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> state
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            transactions to
> exceed
> > > the
> > > >> > > > >> memory
> > > >> > > > >>>>>> assigned
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> them.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - New functionality:
> > > when a
> > > >> > > > >> Task
> > > >> > > > >>>> *does*
> > > >> > > > >>>>>>>>>> commit,
> > > >> > > > >>>>>>>>>>>>>>>> we will
> > > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the stores (and hence
> > > flush
> > > >> > the
> > > >> > > > >>>>>> transaction
> > > >> > > > >>>>>>>>>>>>>>>> buffers)
> > > >> > > > >>>>>>>>>>>>>>>>>>>> unless
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> expect to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            cross the
> > > >> > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > >> > > > >>>>>>>>>>>>>>>>>>>> threshold
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the next
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06,
> Nick
> > > >> > > > >> Telford <
> > > >> > > > >>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> By "less predictable for
> users", I
> > > >> > > > >> meant in
> > > >> > > > >>>>>> terms of
> > > >> > > > >>>>>>>>>>>>>>>>>>>> understanding
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance profile under
> various
> > > >> > > > >>>>>> circumstances. The
> > > >> > > > >>>>>>>>>>>>>>>> more
> > > >> > > > >>>>>>>>>>>>>>>>>>>> complex
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it
> > > would
> > > >> > > > >> be for
> > > >> > > > >>>>>> users
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>> understand
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance they see. For
> example,
> > > >> > > > >> spilling
> > > >> > > > >>>>>> records
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> disk
> > > >> > > > >>>>>>>>>>>>>>>>>>>> when
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a
> > > >> threshold
> > > >> > > > >>>> would, I
> > > >> > > > >>>>>>>>>> expect,
> > > >> > > > >>>>>>>>>>>>>>>>>> reduce
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> write
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in
> write
> > > >> > > > >>>> throughput
> > > >> > > > >>>>>> could
> > > >> > > > >>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> unexpected,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> > > >> > > > >>>> diagnose/understand for
> > > >> > > > >>>>>>>>>> users.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the
> "early
> > > >> > > > >> commit"
> > > >> > > > >>>>>> concept is
> > > >> > > > >>>>>>>>>>>>>>>>>> relatively
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to
> > > >> document,
> > > >> > > > >> and
> > > >> > > > >>>>>>>>>>>> conceptually
> > > >> > > > >>>>>>>>>>>>>>>>>> fairly
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> obvious to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users. We could probably add a
> > > >> metric to
> > > >> > > > >>>> make it
> > > >> > > > >>>>>>>>>> easier
> > > >> > > > >>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it happens though.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is
> *essentially*
> > > >> an
> > > >> > > > >>>> indirect
> > > >> > > > >>>>>> way
> > > >> > > > >>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>>> telling
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether they should be
> > > transactional.
> > > >> > > > >>>>>> READ_COMMITTED
> > > >> > > > >>>>>>>>>>>>>>>>>>>> essentially
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> requires
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, because it
> dictates
> > > >> that
> > > >> > > > >> two
> > > >> > > > >>>>>> threads
> > > >> > > > >>>>>>>>>>>>>>>> calling
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not
> see
> > > >> writes
> > > >> > > > >>>> from
> > > >> > > > >>>>>> the
> > > >> > > > >>>>>>>>>> other
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> transaction
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> until
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> > > >> > > > >>>>>> READ_UNCOMMITTED, all
> > > >> > > > >>>>>>>>>>>>>>>> bets are
> > > >> > > > >>>>>>>>>>>>>>>>>>>> off,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to
> observe
> > > >> > > > >> written
> > > >> > > > >>>>>> records
> > > >> > > > >>>>>>>> at
> > > >> > > > >>>>>>>>>>>>>>>> any
> > > >> > > > >>>>>>>>>>>>>>>>>>>> time,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> which is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially "no transactions".
> That
> > > >> > > > >> said,
> > > >> > > > >>>>>>>> StateStores
> > > >> > > > >>>>>>>>>>>>>>>> are
> > > >> > > > >>>>>>>>>>>>>>>>>> free
> > > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implement
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these guarantees however they
> can,
> > > >> > > > >> which is
> > > >> > > > >>>> a
> > > >> > > > >>>>>> bit
> > > >> > > > >>>>>>>> more
> > > >> > > > >>>>>>>>>>>>>>>>>> relaxed
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> than
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dictating "you must use
> > > >> transactions".
> > > >> > > > >> For
> > > >> > > > >>>>>> example,
> > > >> > > > >>>>>>>>>>>> with
> > > >> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement these as
> READ_COMMITTED
> > > ==
> > > >> > > > >>>> WBWI-based
> > > >> > > > >>>>>>>>>>>>>>>>>> "transactions",
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct
> writes
> > > to
> > > >> the
> > > >> > > > >>>>>> database.
> > > >> > > > >>>>>>>> But
> > > >> > > > >>>>>>>>>>>>>>>> with
> > > >> > > > >>>>>>>>>>>>>>>>>>>> other
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> storage
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> engines, it might be
> preferable to
> > > >> > > > >> *always*
> > > >> > > > >>>> use
> > > >> > > > >>>>>>>>>>>>>>>> transactions,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> even
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be
> > > storage
> > > >> > > > >> engines
> > > >> > > > >>>>>> that
> > > >> > > > >>>>>>>>>> don't
> > > >> > > > >>>>>>>>>>>>>>>>>> provide
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation
> > > >> > > > >> guarantees
> > > >> > > > >>>> can
> > > >> > > > >>>>>> be
> > > >> > > > >>>>>>>> met
> > > >> > > > >>>>>>>>>>>>>>>> using a
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> different
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> technique.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the
> > > >> > > > >> StateStore
> > > >> > > > >>>>>> interface
> > > >> > > > >>>>>>>> as
> > > >> > > > >>>>>>>>>>>>>>>>>> loosely
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> coupled
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as
> > > possible,
> > > >> to
> > > >> > > > >> give
> > > >> > > > >>>>>>>>>>>>>>>> implementers
> > > >> > > > >>>>>>>>>>>>>>>>>> more
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> freedom, and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal
> > > >> knowledge
> > > >> > > > >>>>>> required.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> > > >> > > > >>>> "IsolationLevel"
> > > >> > > > >>>>>> might
> > > >> > > > >>>>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> right
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always
> make
> > > >> it
> > > >> > > > >> much
> > > >> > > > >>>> more
> > > >> > > > >>>>>>>>>>>>>>>> explicit if
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> required, e.g.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7-8.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I can make these changes either
> > > later
> > > >> > > > >> today
> > > >> > > > >>>> or
> > > >> > > > >>>>>>>>>>>> tomorrow.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Small update:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on
> trunk and
> > > >> > > > >> fixed a
> > > >> > > > >>>>>> bunch of
> > > >> > > > >>>>>>>>>>>>>>>> issues
> > > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the
> > > tests
> > > >> > > > >> pass,
> > > >> > > > >>>>>> which is
> > > >> > > > >>>>>>>>>>>>>>>>>> promising,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> but
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> it
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to undergo some
> performance
> > > >> > > > >> testing. I
> > > >> > > > >>>>>> haven't
> > > >> > > > >>>>>>>>>>>>>>>> (yet)
> > > >> > > > >>>>>>>>>>>>>>>>>>>> worked
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> on
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()`
> > > >> stuff,
> > > >> > > > >> but I
> > > >> > > > >>>>>> would
> > > >> > > > >>>>>>>>>>>>>>>> expect
> > > >> > > > >>>>>>>>>>>>>>>>>> that,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make
> no
> > > >> > > > >>>> difference. The
> > > >> > > > >>>>>>>>>> branch
> > > >> > > > >>>>>>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> available
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> at
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> > > >> > > > >>>>>>>>>> if
> > > >> > > > >>>>>>>>>>>>>>>>>> anyone
> > > >> > > > >>>>>>>>>>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interested in taking an early
> look.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59,
> Bruno
> > > >> > > > >> Cadonna
> > > >> > > > >>>> <
> > > >> > > > >>>>>>>>>>>>>>>>>>>> cadonna@apache.org>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That
> was
> > > >> > > > >> actually
> > > >> > > > >>>> also
> > > >> > > > >>>>>> my
> > > >> > > > >>>>>>>>>>>>>>>> point. I
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understood
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that John was proposing the
> > > >> ingestion
> > > >> > > > >> path
> > > >> > > > >>>> as
> > > >> > > > >>>>>> a way
> > > >> > > > >>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> avoid
> > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> early
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I
> > > misinterpreted
> > > >> the
> > > >> > > > >>>> intent.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that
> > > >> actually
> > > >> > > > >> it is
> > > >> > > > >>>>>> public
> > > >> > > > >>>>>>>>>>>>>>>> API. My
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> question
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects
> > > >> normal
> > > >> > > > >>>>>> processing.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that checking
> for
> > > the
> > > >> > > > >> size
> > > >> > > > >>>> of the
> > > >> > > > >>>>>>>>>>>>>>>> transaction
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> buffer
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early
> commit
> > > >> > > > >> affects
> > > >> > > > >>>> the
> > > >> > > > >>>>>> whole
> > > >> > > > >>>>>>>>>>>>>>>>>> processing
> > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality
> of a
> > > >> > > > >> state
> > > >> > > > >>>> store
> > > >> > > > >>>>>> is
> > > >> > > > >>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>>>>>> confined to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills
> > > over
> > > >> and
> > > >> > > > >>>>>> changes the
> > > >> > > > >>>>>>>>>>>>>>>> behavior
> > > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree
> with
> > > >> you
> > > >> > > > >> that
> > > >> > > > >>>> it
> > > >> > > > >>>>>> is a
> > > >> > > > >>>>>>>>>>>>>>>> decent
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compromise. I
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the
> > > downsides
> > > >> > > > >> and
> > > >> > > > >>>> list
> > > >> > > > >>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>> options to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> overcome
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you
> that
> > > all
> > > >> > > > >>>> options
> > > >> > > > >>>>>> seem
> > > >> > > > >>>>>>>>>>>> quite
> > > >> > > > >>>>>>>>>>>>>>>>>> heavy
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compared
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not
> understand
> > > >> > > > >> what you
> > > >> > > > >>>>>> mean
> > > >> > > > >>>>>>>> with
> > > >> > > > >>>>>>>>>>>>>>>> "less
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> predictable
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I found the discussions about
> the
> > > >> > > > >>>> alternatives
> > > >> > > > >>>>>>>> really
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> interesting.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> But I
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also think that your plan
> sounds
> > > >> good
> > > >> > > > >> and
> > > >> > > > >>>> we
> > > >> > > > >>>>>> should
> > > >> > > > >>>>>>>>>>>>>>>> continue
> > > >> > > > >>>>>>>>>>>>>>>>>>>> with
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> it!
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply
> to my
> > > >> > > > >> e-mail on
> > > >> > > > >>>>>> June
> > > >> > > > >>>>>>>>>> 20th:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the
> > > reasoning
> > > >> > > > >> behind
> > > >> > > > >>>>>> putting
> > > >> > > > >>>>>>>>>>>>>>>> isolation
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> level
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store context.
> Thanks!
> > > >> Should
> > > >> > > > >>>> that
> > > >> > > > >>>>>> also
> > > >> > > > >>>>>>>> be
> > > >> > > > >>>>>>>>>> a
> > > >> > > > >>>>>>>>>>>>>>>> way
> > > >> > > > >>>>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> give
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store the
> opportunity to
> > > >> > > > >> decide
> > > >> > > > >>>>>> whether
> > > >> > > > >>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> turn on
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more
> > > >> concerned
> > > >> > > > >> about
> > > >> > > > >>>>>> how do
> > > >> > > > >>>>>>>>>> you
> > > >> > > > >>>>>>>>>>>>>>>> know
> > > >> > > > >>>>>>>>>>>>>>>>>>>> if a
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be
> > > written
> > > >> > > > >> under
> > > >> > > > >>>> EOS,
> > > >> > > > >>>>>> if
> > > >> > > > >>>>>>>> you
> > > >> > > > >>>>>>>>>>>>>>>> do not
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> have a
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
> > > >> > > > >>>> transactional or
> > > >> > > > >>>>>> not.
> > > >> > > > >>>>>>>>>> If
> > > >> > > > >>>>>>>>>>>>>>>> a
> > > >> > > > >>>>>>>>>>>>>>>>>> state
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint
> file
> > > >> can
> > > >> > > > >> be
> > > >> > > > >>>>>> written
> > > >> > > > >>>>>>>>>>>>>>>> during
> > > >> > > > >>>>>>>>>>>>>>>>>>>> normal
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the
> state
> > > >> > > > >> store
> > > >> > > > >>>> is not
> > > >> > > > >>>>>>>>>>>>>>>>>> transactional,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be
> > > written
> > > >> > > > >> under
> > > >> > > > >>>> EOS.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My point was about not only
> > > >> > > > >> considering the
> > > >> > > > >>>>>> bytes
> > > >> > > > >>>>>>>> in
> > > >> > > > >>>>>>>>>>>>>>>> memory
> > > >> > > > >>>>>>>>>>>>>>>>>> in
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> config
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> statestore.uncommitted.max.bytes,
> > > >> but
> > > >> > > > >> also
> > > >> > > > >>>>>> bytes
> > > >> > > > >>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>> might
> > > >> > > > >>>>>>>>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> spilled
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was
> > > wondering
> > > >> > > > >>>> whether you
> > > >> > > > >>>>>>>>>> should
> > > >> > > > >>>>>>>>>>>>>>>>>> remove
> > > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of
> > > >> memory
> > > >> > > > >>>> bytes to
> > > >> > > > >>>>>> be
> > > >> > > > >>>>>>>>>> used
> > > >> > > > >>>>>>>>>>>>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store
> > > >> > > > >> records." My
> > > >> > > > >>>>>>>> thinking
> > > >> > > > >>>>>>>>>>>>>>>> was
> > > >> > > > >>>>>>>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> even
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted
> > > >> bytes to
> > > >> > > > >>>> disk,
> > > >> > > > >>>>>>>>>> limiting
> > > >> > > > >>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> overall
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> bytes
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking
> about
> > > it
> > > >> > > > >> again
> > > >> > > > >>>> and
> > > >> > > > >>>>>>>>>>>>>>>> considering
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> recent
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make
> too
> > > >> much
> > > >> > > > >>>> sense
> > > >> > > > >>>>>>>> anymore.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I like the name
> > > >> > > > >>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes that
> > > >> > > > >>>>>>>>>>>>>>>>>>>> you
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 8.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A high-level description
> (without
> > > >> > > > >>>>>> implementation
> > > >> > > > >>>>>>>>>>>>>>>> details) of
> > > >> > > > >>>>>>>>>>>>>>>>>>>> how
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the
> commit of
> > > >> > > > >> changelog
> > > >> > > > >>>>>>>>>>>>>>>> transactions,
> > > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing
> > > >> would be
> > > >> > > > >>>> great.
> > > >> > > > >>>>>>>> Would
> > > >> > > > >>>>>>>>>>>> be
> > > >> > > > >>>>>>>>>>>>>>>>>> great
> > > >> > > > >>>>>>>>>>>>>>>>>>>> if
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> you
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could also add some sentences
> > > about
> > > >> the
> > > >> > > > >>>>>> behavior in
> > > >> > > > >>>>>>>>>>>>>>>> case of
> > > >> > > > >>>>>>>>>>>>>>>>>> a
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> failure.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For instance how does a
> > > >> transactional
> > > >> > > > >> state
> > > >> > > > >>>>>> store
> > > >> > > > >>>>>>>>>>>>>>>> recover
> > > >> > > > >>>>>>>>>>>>>>>>>>>> after a
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure or what happens with
> the
> > > >> > > > >>>> transaction
> > > >> > > > >>>>>>>> buffer,
> > > >> > > > >>>>>>>>>>>>>>>> etc.
> > > >> > > > >>>>>>>>>>>>>>>>>>>> (that
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> what
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in
> point
> > > 9.)
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick
> Telford
> > > >> wrote:
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same
> issue
> > > >> that
> > > >> > > > >>>>>>>>>>>>>>>> WriteBatchWithIndex
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or
> > > likely
> > > >> to
> > > >> > > > >>>> exceed)
> > > >> > > > >>>>>>>>>>>>>>>> configured
> > > >> > > > >>>>>>>>>>>>>>>>>>>> memory
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needs to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big
> concerns.
> > > >> > > > >>>> Ultimately,
> > > >> > > > >>>>>> any
> > > >> > > > >>>>>>>>>>>>>>>> approach
> > > >> > > > >>>>>>>>>>>>>>>>>>>> based
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> on
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cracking
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and
> using
> > > >> it in
> > > >> > > > >>>> ways
> > > >> > > > >>>>>> it's
> > > >> > > > >>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>> really
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> designed
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen
> > > >> > > > >> performance
> > > >> > > > >>>> or
> > > >> > > > >>>>>>>>>>>>>>>> consistency
> > > >> > > > >>>>>>>>>>>>>>>>>>>> issues.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for
> > > removing
> > > >> > > > >> these
> > > >> > > > >>>>>> early
> > > >> > > > >>>>>>>>>>>>>>>> commits?
> > > >> > > > >>>>>>>>>>>>>>>>>>>> While
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> not
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent
> compromise
> > > >> to
> > > >> > > > >>>> ensure
> > > >> > > > >>>>>>>>>>>>>>>> consistency
> > > >> > > > >>>>>>>>>>>>>>>>>>>> whilst
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maintaining
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good and predictable
> performance.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas
> > > seem
> > > >> > > > >> *very*
> > > >> > > > >>>>>>>>>>>>>>>> complicated, and
> > > >> > > > >>>>>>>>>>>>>>>>>>>> might
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less
> predictable
> > > for
> > > >> > > > >> users
> > > >> > > > >>>> as a
> > > >> > > > >>>>>>>>>>>>>>>> consequence.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the
> > > scope
> > > >> of
> > > >> > > > >> this
> > > >> > > > >>>>>> KIP is
> > > >> > > > >>>>>>>>>>>>>>>> growing a
> > > >> > > > >>>>>>>>>>>>>>>>>>>> bit
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> out
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to
> > > discuss
> > > >> > > > >> ideas
> > > >> > > > >>>> for
> > > >> > > > >>>>>>>> future
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> improvements, I
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > >> > > > >>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the
> > > scope
> > > >> > > > >> down
> > > >> > > > >>>> to a
> > > >> > > > >>>>>>>> design
> > > >> > > > >>>>>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> achieves
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> most
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant
> > > sized
> > > >> > > > >>>>>> restorations
> > > >> > > > >>>>>>>>>>>>>>>> during
> > > >> > > > >>>>>>>>>>>>>>>>>> dirty
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any
> > > >> design
> > > >> > > > >> that
> > > >> > > > >>>>>> this KIP
> > > >> > > > >>>>>>>>>>>>>>>> produces
> > > >> > > > >>>>>>>>>>>>>>>>>>>> can
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ultimately
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future,
> > > >> especially
> > > >> > > > >> if
> > > >> > > > >>>> the
> > > >> > > > >>>>>> bulk
> > > >> > > > >>>>>>>> of
> > > >> > > > >>>>>>>>>>>>>>>> it is
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> internal
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time
> next
> > > >> week
> > > >> > > > >>>> trying
> > > >> > > > >>>>>> to
> > > >> > > > >>>>>>>>>>>>>>>> re-work
> > > >> > > > >>>>>>>>>>>>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to
> > > >> remove
> > > >> > > > >> the
> > > >> > > > >>>>>>>>>>>>>>>> newTransaction()
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>> method,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> such
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation
> > > detail
> > > >> of
> > > >> > > > >>>>>>>> RocksDBStore.
> > > >> > > > >>>>>>>>>>>>>>>> That
> > > >> > > > >>>>>>>>>>>>>>>>>>>> way, if
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something
> in
> > > the
> > > >> > > > >> future,
> > > >> > > > >>>>>> like
> > > >> > > > >>>>>>>> the
> > > >> > > > >>>>>>>>>>>>>>>> SST
> > > >> > > > >>>>>>>>>>>>>>>>>> file
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> management
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we
> can do
> > > so
> > > >> > > > >> with
> > > >> > > > >>>>>> little/no
> > > >> > > > >>>>>>>>>>>> API
> > > >> > > > >>>>>>>>>>>>>>>>>>>> changes.
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>>
> > > >> > > > >>>>>>>>>>>
> > > >> > > > >>>>>>>>>>
> > > >> > > > >>>>>>>>>
> > > >> > > > >>>>>>>>
> > > >> > > > >>>>>>
> > > >> > > > >>>>>>
> > > >> > > > >>>>
> > > >> > > > >>
> > > >> > > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > >
>

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

Posted by Lucas Brutschy <lb...@confluent.io.INVALID>.
Hi Nick,

the java clients do have some properties that are of the form
`enable....` (`enable.auto.comit` and `enable.idempotance`), so I
would go with this form. State updater and processing thread configs
use the `....enabled` form, but they are internal and could be changed
to the `enable...` form for consistency. But, not intending to start a
big discussion here. Both options are fine with me.

Thanks for the updates!


On Sun, Apr 14, 2024 at 10:58 PM Sophie Blee-Goldman
<so...@responsive.dev> wrote:
>
> Makes sense to me! Regarding the new config name -- doesn't seem like
> we have any *public *configs in StreamsConfig that set a standard yet, so
> imo we're free to pick whatever we think sounds good.
>
> However we do have a few internal configs that follow the pattern
> *feature.name.enabled* so if we want to play it safe and adhere to the
> established pattern, we should call it "transactional.statestores.enabled"
>
> I also personally think x.y.enabled sounds better than enable.x.y, but
> honestly I don't feel too strongly either way. I don't think it's worth
> prolonging
> the discussion any further over
>
> I'll take a look at KIP-892 as soon as I can since it's a prereq for this
> one.
> Thanks for the updates!
>
>
> On Thu, Apr 11, 2024 at 12:02 PM Nick Telford <ni...@gmail.com>
> wrote:
>
> > Hi everyone,
> >
> > After various off-list discussions, it's become clear there are still some
> > contentious points to this KIP.
> >
> > 1.
> > We agreed to change the "default.state.isolation.level" config property to
> > be a transactional state store feature toggle. What should we name this?
> > "enable.transactional.statestores"? What's the convention for this kind of
> > flag?
> >
> > 2.
> > The "atomic checkpointing" section has been broken out into KIP-1035 (with
> > one significant change, and otherwise just more explanation). This is now a
> > hard dependency of KIP-892.
> >
> > Regards,
> > Nick
> >
> > On Mon, 6 Nov 2023 at 17:00, Nick Telford <ni...@gmail.com> wrote:
> >
> > > Hi everyone,
> > >
> > > Sorry for the delay.
> > >
> > > I've updated the KIP based on our discussion. I've also added some
> > details
> > > on "Transactional support under READ_UNCOMMITTED" and  "Query-time
> > > Isolation Levels" in the "Rejected Alternatives" section, to highlight
> > the
> > > potential to improve things in a future KIP.
> > >
> > > If there are no more requested changes or objections, I'll kick off the
> > > vote thread tomorrow, approximately 24 hours from now.
> > >
> > > While I'd love for this KIP to make it into 3.7.0, I can't commit to it
> > > being code-complete by the December 20th deadline. While the branch I
> > have
> > > is broadly working, there are areas that will likely require improvement,
> > > and others that are untested:
> > >
> > > - Untested outside of unit tests: Windowed stores, Versioned stores,
> > > Global stores, IQv2. They should all work as expected, but the Position
> > > file logic might be a bit janky.
> > > - Requires improvement: Position data handling. Very clunky right now.
> > > - Requires implementation: The "feature flag", where we disable
> > > transactionality under EOS+READ_UNCOMMITTED. I've started this, but it's
> > > far from simple. Plus this will require fleshing out the test suite
> > quite a
> > > bit.
> > >
> > > For the curious, my active development branch has been rebased against
> > > 3.6.0: https://github.com/nicktelford/kafka/tree/KIP-892-3.6.0
> > >
> > > Regards,
> > > Nick
> > >
> > > On Mon, 30 Oct 2023 at 23:32, Sophie Blee-Goldman <sophie@responsive.dev
> > >
> > > wrote:
> > >
> > >> Hey Nick, sounds like things are moving along here. I think you're
> > already
> > >> aware, but just as a reminder to us all, the KIP freeze deadline for 3.7
> > >> is
> > >> approaching and is currently set for Nov 18. I think we can all agree
> > that
> > >> it would be great to have this KIP accepted by then, and it sounds like
> > >> much of the implementation is already completed, in which case this
> > might
> > >> be able to make it into 3.7.
> > >>
> > >> It sounds like this discussion is wrapping up so I just want to
> > summarize
> > >> my understanding of the current plan regarding configs since it doesn't
> > >> seem like the KIP has been updated to include this yet.
> > >>
> > >> Basically we're all in agreement to go with option #1 that you presented
> > >> earlier, right? Assuming so, I just want to clear up some details around
> > >> the default behavior. What happens if:
> > >> 1. User sets EOS and sets READ_UNCOMMITTED: txn state stores will be
> > >> disabled/feature-flagged
> > >> 2. User sets EOS and does not set anything for the isolation level:
> > >> although the default is READ_UNCOMMITTED, enabling EOS will change the
> > >> default to READ_COMMITTED and txn state stores will be used
> > >>
> > >> As for the future, it sounds like when READ_UNCOMMITTED mode is
> > >> implemented, we will basically just remove this "feature flag" and txn
> > >> state stores will always be used for all EOS cases. EOS users will be
> > able
> > >> to configure the isolation level independently, although it will still
> > >> default to READ_COMMITTED when EOS is enabled and it wasn't explicitly
> > >> configured.
> > >>
> > >> Is this understanding correct? I think this was the right choice
> > >> regardless, as it will give people a way to disable the txn stores in an
> > >> emergency -- as a project we went a long time with little pressure to
> > >> feature flag things, and our users paid the cost for that. Even if we
> > >> managed to ship something without bugs, it was often only after an
> > intense
> > >> period of successive blocker bugs that delayed the entire released for
> > >> weeks. Other times, major bugs slipped through and some versions became
> > >> unusable for certain use cases. So having some way to disable the txn
> > >> state
> > >> stores when EOS is used feels like a good strategy, since you just never
> > >> know what might get through all the testing we do.
> > >>
> > >> If this ends up slipping to 4.0 and you manage to implement the
> > >> READ_UNCOMMITTED mode within the same release, I think it's worth
> > >> considering to add in an additional feature flag, even if it's just a
> > >> backdoor internal config (eg as we did in KIP-441 with the internal task
> > >> assignor config). But for now let's assume the first release this KIP
> > >> appears in will have the behavior as described above, with
> > >> READ_UNCOMMITTED
> > >> mode acting as a feature flag
> > >>
> > >> Nick -- please let us know when you've updated the KIP to clarify the
> > >> config behavior, and are ready for a vote!
> > >>
> > >>
> > >> On Sun, Oct 29, 2023 at 12:02 PM Colt McNealy <co...@littlehorse.io>
> > >> wrote:
> > >>
> > >> > Guozhang—I agree, I am in favor of moving forward with the KIP now
> > that
> > >> the
> > >> > Transactional State Stores will be behind a feature flag.
> > >> >
> > >> > Nick—I just did a bit more light testing of your branch
> > `KIP-892-3.5.0`
> > >> > with your most recent changes. I couldn't detect a performance
> > >> difference
> > >> > versus trunk (in the past there was a slight degradation of
> > performance
> > >> on
> > >> > the restoration path, but that has been fixed). I don't believe that
> > >> your
> > >> > branch has the state updater thread enabled, so I didn't test that
> > path
> > >> too
> > >> > heavily.
> > >> >
> > >> > As expected, however, our internal correctness tests failed due to the
> > >> IQ
> > >> > read-your-own-writes issue we discussed previously. The community as a
> > >> > whole would vastly benefit from this KIP getting over the finish line
> > in
> > >> > 3.7.0, and so long as it is behind a feature flag so that we at
> > >> LittleHorse
> > >> > can still guarantee RYOW for our users, I think it's purely a win 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 <
> > >> guozhang.wang.us@gmail.com
> > >> > >
> > >> > wrote:
> > >> >
> > >> > > I'd agree with you guys that as long as we are in agreement about
> > the
> > >> > > configuration semantics, that would be a big win to move forward for
> > >> > > this KIP. As for the TaskCorruptedException handling like wiping
> > state
> > >> > > stores, we can discuss that in the PR rather than in the KIP.
> > >> > >
> > >> > > Just to clarify, I'm onboard with the latest proposal, and probably
> > we
> > >> > > can move on for voting on this KIP now?
> > >> > >
> > >> > > Guozhang
> > >> > >
> > >> > > On Thu, Oct 19, 2023 at 5:33 AM Bruno Cadonna <ca...@apache.org>
> > >> > wrote:
> > >> > > >
> > >> > > > Hi Nick,
> > >> > > >
> > >> > > > What you and Lucas wrote about the different configurations of
> > >> ALOS/EOS
> > >> > > > and READ_COMMITTED/READ_UNCOMMITTED make sense to me. My earlier
> > >> > > > concerns about changelogs diverging from the content of the local
> > >> state
> > >> > > > stores turned out to not apply. So I think, we can move on with
> > >> those
> > >> > > > configurations.
> > >> > > >
> > >> > > > Regarding the TaskCorruptedException and wiping out the state
> > stores
> > >> > > > under EOS, couldn't we abort the transaction on the state store
> > and
> > >> > > > close the task dirty? If the Kafka transaction was indeed
> > committed,
> > >> > the
> > >> > > > store would restore the missing part from the changelog topic. If
> > >> the
> > >> > > > Kafka transaction was not committed, changelog topic and state
> > store
> > >> > are
> > >> > > > in-sync.
> > >> > > >
> > >> > > > In any case, IMO those are implementation details that we do not
> > >> need
> > >> > to
> > >> > > > discuss and solve in the KIP discussion. We can solve them on the
> > >> PR.
> > >> > > > The important thing is that the processing guarantees hold.
> > >> > > >
> > >> > > > Best,
> > >> > > > Bruno
> > >> > > >
> > >> > > > On 10/18/23 3:56 PM, Nick Telford wrote:
> > >> > > > > Hi Lucas,
> > >> > > > >
> > >> > > > > TaskCorruptedException is how Streams signals that the Task
> > state
> > >> > > needs to
> > >> > > > > be wiped, so we can't retain that exception without also wiping
> > >> state
> > >> > > on
> > >> > > > > timeouts.
> > >> > > > >
> > >> > > > > Regards,
> > >> > > > > Nick
> > >> > > > >
> > >> > > > > On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <
> > >> lbrutschy@confluent.io
> > >> > > .invalid>
> > >> > > > > wrote:
> > >> > > > >
> > >> > > > >> Hi Nick,
> > >> > > > >>
> > >> > > > >> I think indeed the better behavior would be to retry
> > >> > commitTransaction
> > >> > > > >> until we risk running out of time to meet `
> > max.poll.interval.ms
> > >> `.
> > >> > > > >>
> > >> > > > >> However, if it's handled as a `TaskCorruptedException` at the
> > >> > moment,
> > >> > > > >> I would do the same in this KIP, and leave exception handling
> > >> > > > >> improvements to future work. 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 <
> > >> > nick.telford@gmail.com>
> > >> > > > >> wrote:
> > >> > > > >>>
> > >> > > > >>> Hi Lucas,
> > >> > > > >>>
> > >> > > > >>> Yeah, this is pretty much the direction I'm thinking of going
> > in
> > >> > > now. You
> > >> > > > >>> make an interesting point about committing on-error under
> > >> > > > >>> ALOS/READ_COMMITTED, although I haven't had a chance to think
> > >> > > through the
> > >> > > > >>> implications yet.
> > >> > > > >>>
> > >> > > > >>> Something that I ran into earlier this week is an issue with
> > the
> > >> > new
> > >> > > > >>> handling of TimeoutException. Without TX stores,
> > >> TimeoutException
> > >> > > under
> > >> > > > >> EOS
> > >> > > > >>> throws a TaskCorruptedException, which wipes the stores.
> > >> However,
> > >> > > with TX
> > >> > > > >>> stores, TimeoutException is now just bubbled up and dealt with
> > >> as
> > >> > it
> > >> > > is
> > >> > > > >>> under ALOS. The problem arises when the
> > >> Producer#commitTransaction
> > >> > > call
> > >> > > > >>> times out: Streams attempts to ignore the error and continue
> > >> > > producing,
> > >> > > > >>> which causes the next call to Producer#send to throw
> > >> > > > >>> "IllegalStateException: Cannot attempt operation `send`
> > because
> > >> the
> > >> > > > >>> previous call to `commitTransaction` timed out and must be
> > >> > retried".
> > >> > > > >>>
> > >> > > > >>> I'm not sure what we should do here: retrying the
> > >> commitTransaction
> > >> > > seems
> > >> > > > >>> logical, but what if it times out again? Where do we draw the
> > >> line
> > >> > > and
> > >> > > > >>> shutdown the instance?
> > >> > > > >>>
> > >> > > > >>> Regards,
> > >> > > > >>> Nick
> > >> > > > >>>
> > >> > > > >>> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <
> > >> > lbrutschy@confluent.io
> > >> > > > >> .invalid>
> > >> > > > >>> wrote:
> > >> > > > >>>
> > >> > > > >>>> Hi all,
> > >> > > > >>>>
> > >> > > > >>>> I think I liked your suggestion of allowing EOS with
> > >> > > READ_UNCOMMITTED,
> > >> > > > >>>> but keep wiping the state on error, and I'd vote for this
> > >> solution
> > >> > > > >>>> when introducing `default.state.isolation.level`. This way,
> > >> we'd
> > >> > > have
> > >> > > > >>>> the most low-risk roll-out of this feature (no behavior
> > change
> > >> > > without
> > >> > > > >>>> reconfiguration), with the possibility of switching to the
> > most
> > >> > > sane /
> > >> > > > >>>> battle-tested default settings in 4.0. Essentially, we'd
> > have a
> > >> > > > >>>> feature flag but call it `default.state.isolation.level` and
> > >> don't
> > >> > > > >>>> have to deprecate it later.
> > >> > > > >>>>
> > >> > > > >>>> So the possible configurations would then be this:
> > >> > > > >>>>
> > >> > > > >>>> 1. ALOS/READ_UNCOMMITTED (default) = processing uses
> > >> direct-to-DB,
> > >> > > IQ
> > >> > > > >>>> reads from DB.
> > >> > > > >>>> 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads
> > >> from
> > >> > > > >>>> WriteBatch/DB. Flush on error (see note below).
> > >> > > > >>>> 3. EOS/READ_UNCOMMITTED (default) = processing uses
> > >> direct-to-DB,
> > >> > IQ
> > >> > > > >>>> reads from DB. Wipe state on error.
> > >> > > > >>>> 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads
> > >> from
> > >> > > > >>>> WriteBatch/DB.
> > >> > > > >>>>
> > >> > > > >>>> I believe the feature is important enough that we will see
> > good
> > >> > > > >>>> adoption even without changing the default. In 4.0, when we
> > >> have
> > >> > > seen
> > >> > > > >>>> this being adopted and is battle-tested, we make
> > READ_COMMITTED
> > >> > the
> > >> > > > >>>> default for EOS, or even READ_COMITTED always the default,
> > >> > depending
> > >> > > > >>>> on our experiences. And we could add a clever implementation
> > of
> > >> > > > >>>> READ_UNCOMITTED with WriteBatches later.
> > >> > > > >>>>
> > >> > > > >>>> The only smell here is that `default.state.isolation.level`
> > >> > wouldn't
> > >> > > > >>>> be purely an IQ setting, but it would also (slightly) change
> > >> the
> > >> > > > >>>> behavior of the processing, but that seems unavoidable as
> > long
> > >> as
> > >> > we
> > >> > > > >>>> haven't solve READ_UNCOMITTED IQ with WriteBatches.
> > >> > > > >>>>
> > >> > > > >>>> Minor: As for Bruno's point 4, I think if we are concerned
> > >> about
> > >> > > this
> > >> > > > >>>> 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).
> > >> > > > >>>>
> > >> > > > >>>> Cheers,
> > >> > > > >>>> Lucas
> > >> > > > >>>>
> > >> > > > >>>> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <
> > >> > > nick.telford@gmail.com>
> > >> > > > >>>> wrote:
> > >> > > > >>>>>
> > >> > > > >>>>> Hi Guozhang,
> > >> > > > >>>>>
> > >> > > > >>>>> The KIP as it stands introduces a new configuration,
> > >> > > > >>>>> default.state.isolation.level, which is independent of
> > >> > > > >> processing.mode.
> > >> > > > >>>>> It's intended that this new configuration be used to
> > >> configure a
> > >> > > > >> global
> > >> > > > >>>> IQ
> > >> > > > >>>>> isolation level in the short term, with a future KIP
> > >> introducing
> > >> > > the
> > >> > > > >>>>> capability to change the isolation level on a per-query
> > basis,
> > >> > > > >> falling
> > >> > > > >>>> back
> > >> > > > >>>>> to the "default" defined by this config. That's why I called
> > >> it
> > >> > > > >>>> "default",
> > >> > > > >>>>> for future-proofing.
> > >> > > > >>>>>
> > >> > > > >>>>> However, it currently includes the caveat that
> > >> READ_UNCOMMITTED
> > >> > is
> > >> > > > >> not
> > >> > > > >>>>> available under EOS. I think this is the coupling you are
> > >> > alluding
> > >> > > > >> to?
> > >> > > > >>>>>
> > >> > > > >>>>> This isn't intended to be a restriction of the API, but is
> > >> > > currently
> > >> > > > >> a
> > >> > > > >>>>> technical limitation. However, after discussing with some
> > >> users
> > >> > > about
> > >> > > > >>>>> use-cases that would require READ_UNCOMMITTED under EOS, I'm
> > >> > > > >> inclined to
> > >> > > > >>>>> remove that clause and put in the necessary work to make
> > that
> > >> > > > >> combination
> > >> > > > >>>>> possible now.
> > >> > > > >>>>>
> > >> > > > >>>>> I currently see two possible approaches:
> > >> > > > >>>>>
> > >> > > > >>>>>     1. Disable TX StateStores internally when the
> > >> IsolationLevel
> > >> > is
> > >> > > > >>>>>     READ_UNCOMMITTED and the processing.mode is EOS. This is
> > >> more
> > >> > > > >>>> difficult
> > >> > > > >>>>>     than it sounds, as there are many assumptions being made
> > >> > > > >> throughout
> > >> > > > >>>> the
> > >> > > > >>>>>     internals about the guarantees StateStores provide. It
> > >> would
> > >> > > > >>>> definitely add
> > >> > > > >>>>>     a lot of extra "if (read_uncommitted && eos)" branches,
> > >> > > > >> complicating
> > >> > > > >>>>>     maintenance and testing.
> > >> > > > >>>>>     2. Invest the time *now* to make READ_UNCOMMITTED of EOS
> > >> > > > >> StateStores
> > >> > > > >>>>>     possible. I have some ideas on how this could be
> > achieved,
> > >> > but
> > >> > > > >> they
> > >> > > > >>>> would
> > >> > > > >>>>>     need testing and could introduce some additional issues.
> > >> The
> > >> > > > >> benefit
> > >> > > > >>>> of
> > >> > > > >>>>>     this approach is that it would make query-time
> > >> > IsolationLevels
> > >> > > > >> much
> > >> > > > >>>> simpler
> > >> > > > >>>>>     to implement in the future.
> > >> > > > >>>>>
> > >> > > > >>>>> Unfortunately, both will require considerable work that will
> > >> > > further
> > >> > > > >>>> delay
> > >> > > > >>>>> this KIP, which was the reason I placed the restriction in
> > the
> > >> > KIP
> > >> > > > >> in the
> > >> > > > >>>>> first place.
> > >> > > > >>>>>
> > >> > > > >>>>> Regards,
> > >> > > > >>>>> Nick
> > >> > > > >>>>>
> > >> > > > >>>>> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
> > >> > > > >> guozhang.wang.us@gmail.com>
> > >> > > > >>>>> wrote:
> > >> > > > >>>>>
> > >> > > > >>>>>> Hello Nick,
> > >> > > > >>>>>>
> > >> > > > >>>>>> First of all, thanks a lot for the great effort you've put
> > in
> > >> > > > >> driving
> > >> > > > >>>>>> this KIP! I really like it coming through finally, as many
> > >> > people
> > >> > > > >> in
> > >> > > > >>>>>> the community have raised this. At the same time I honestly
> > >> > feel a
> > >> > > > >> bit
> > >> > > > >>>>>> ashamed for not putting enough of my time supporting it and
> > >> > > > >> pushing it
> > >> > > > >>>>>> through the finish line (you raised this KIP almost a year
> > >> ago).
> > >> > > > >>>>>>
> > >> > > > >>>>>> I briefly passed through the DISCUSS thread so far, not
> > sure
> > >> > I've
> > >> > > > >> 100
> > >> > > > >>>>>> percent digested all the bullet points. But with the goal
> > of
> > >> > > > >> trying to
> > >> > > > >>>>>> help take it through the finish line in mind, I'd want to
> > >> throw
> > >> > > > >>>>>> thoughts on top of my head only on the point #4 above
> > which I
> > >> > felt
> > >> > > > >> may
> > >> > > > >>>>>> be the main hurdle for the current KIP to drive to a
> > >> consensus
> > >> > > now.
> > >> > > > >>>>>>
> > >> > > > >>>>>> The general question I asked myself is, whether we want to
> > >> > couple
> > >> > > > >> "IQ
> > >> > > > >>>>>> reading mode" with "processing mode". While technically I
> > >> tend
> > >> > to
> > >> > > > >>>>>> agree with you that, it's feels like a bug if some single
> > >> user
> > >> > > > >> chose
> > >> > > > >>>>>> "EOS" for processing mode while choosing "read uncommitted"
> > >> for
> > >> > IQ
> > >> > > > >>>>>> reading mode, at the same time, I'm thinking if it's
> > possible
> > >> > that
> > >> > > > >>>>>> there could be two different persons (or even two teams)
> > that
> > >> > > > >> would be
> > >> > > > >>>>>> using the stream API to build the app, and the IQ API to
> > >> query
> > >> > the
> > >> > > > >>>>>> running state of the app. I know this is less of a
> > technical
> > >> > thing
> > >> > > > >> but
> > >> > > > >>>>>> rather a more design stuff, but if it could be ever the
> > case,
> > >> > I'm
> > >> > > > >>>>>> wondering if the personale using the IQ API knows about the
> > >> > risks
> > >> > > > >> of
> > >> > > > >>>>>> using read uncommitted but still chose so for the favor of
> > >> > > > >>>>>> performance, no matter if the underlying stream processing
> > >> mode
> > >> > > > >>>>>> configured by another personale is EOS or not. In that
> > >> regard,
> > >> > I'm
> > >> > > > >>>>>> leaning towards a "leaving the door open, and close it
> > later
> > >> if
> > >> > we
> > >> > > > >>>>>> found it's a bad idea" aspect with a configuration 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 can still
> > >> enforce
> > >> > > > >>>>>> coupling them.
> > >> > > > >>>>>>
> > >> > > > >>>>>> Again, just my 2c here. Thanks again for the great patience
> > >> and
> > >> > > > >>>>>> diligence on this KIP.
> > >> > > > >>>>>>
> > >> > > > >>>>>>
> > >> > > > >>>>>> Guozhang
> > >> > > > >>>>>>
> > >> > > > >>>>>>
> > >> > > > >>>>>>
> > >> > > > >>>>>> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
> > >> > > > >> nick.telford@gmail.com>
> > >> > > > >>>>>> wrote:
> > >> > > > >>>>>>>
> > >> > > > >>>>>>> Hi Bruno,
> > >> > > > >>>>>>>
> > >> > > > >>>>>>> 4.
> > >> > > > >>>>>>> I'll hold off on making that change until we have a
> > >> consensus
> > >> > as
> > >> > > > >> to
> > >> > > > >>>> what
> > >> > > > >>>>>>> configuration to use to control all of this, as it'll be
> > >> > > > >> affected by
> > >> > > > >>>> the
> > >> > > > >>>>>>> decision on EOS isolation levels.
> > >> > > > >>>>>>>
> > >> > > > >>>>>>> 5.
> > >> > > > >>>>>>> Done. I've chosen "committedOffsets".
> > >> > > > >>>>>>>
> > >> > > > >>>>>>> Regards,
> > >> > > > >>>>>>> Nick
> > >> > > > >>>>>>>
> > >> > > > >>>>>>> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <
> > >> > cadonna@apache.org>
> > >> > > > >>>> wrote:
> > >> > > > >>>>>>>
> > >> > > > >>>>>>>> Hi Nick,
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>>> 1.
> > >> > > > >>>>>>>> Yeah, you are probably right that it does not make too
> > much
> > >> > > > >> sense.
> > >> > > > >>>>>>>> Thanks for the clarification!
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>>> 4.
> > >> > > > >>>>>>>> Yes, sorry for the back and forth, but I think for the
> > >> sake of
> > >> > > > >> the
> > >> > > > >>>> KIP
> > >> > > > >>>>>>>> it is better to let the ALOS behavior as it is for now
> > due
> > >> to
> > >> > > > >> the
> > >> > > > >>>>>>>> possible issues you would run into. Maybe we can find a
> > >> > > > >> solution
> > >> > > > >>>> in the
> > >> > > > >>>>>>>> future. Now the question returns to whether we really
> > need
> > >> > > > >>>>>>>> default.state.isolation.level. Maybe the config could be
> > >> the
> > >> > > > >>>> feature
> > >> > > > >>>>>>>> flag Sophie requested.
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>>> 5.
> > >> > > > >>>>>>>> There 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
> > >> committedOffsetFor().
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>>> Best,
> > >> > > > >>>>>>>> Bruno
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>>> On 10/13/23 3:21 PM, Nick Telford wrote:
> > >> > > > >>>>>>>>> Hi Bruno,
> > >> > > > >>>>>>>>>
> > >> > > > >>>>>>>>> Thanks for getting back to me.
> > >> > > > >>>>>>>>>
> > >> > > > >>>>>>>>> 1.
> > >> > > > >>>>>>>>> I think this should be possible. Are you thinking of the
> > >> > > > >>>> situation
> > >> > > > >>>>>> where
> > >> > > > >>>>>>>> a
> > >> > > > >>>>>>>>> user may downgrade to a previous version of Kafka
> > >> Streams? In
> > >> > > > >>>> that
> > >> > > > >>>>>> case,
> > >> > > > >>>>>>>>> sadly, the RocksDBStore would get wiped by the older
> > >> version
> > >> > > > >> of
> > >> > > > >>>> Kafka
> > >> > > > >>>>>>>>> Streams anyway, because that version wouldn't understand
> > >> the
> > >> > > > >>>> extra
> > >> > > > >>>>>> column
> > >> > > > >>>>>>>>> family (that holds offsets), so the missing Position
> > file
> > >> > > > >> would
> > >> > > > >>>>>>>>> automatically get rebuilt when the store is rebuilt from
> > >> the
> > >> > > > >>>>>> changelog.
> > >> > > > >>>>>>>>> Are there other situations than downgrade where a
> > >> > > > >> transactional
> > >> > > > >>>> store
> > >> > > > >>>>>>>> could
> > >> > > > >>>>>>>>> be replaced by a non-transactional one? I can't think of
> > >> any.
> > >> > > > >>>>>>>>>
> > >> > > > >>>>>>>>> 2.
> > >> > > > >>>>>>>>> Ahh yes, the Test Plan - my Kryptonite! This section
> > >> > > > >> definitely
> > >> > > > >>>>>> needs to
> > >> > > > >>>>>>>> be
> > >> > > > >>>>>>>>> fleshed out. I'll work on that. How much detail do you
> > >> need?
> > >> > > > >>>>>>>>>
> > >> > > > >>>>>>>>> 3.
> > >> > > > >>>>>>>>> See my previous email discussing this.
> > >> > > > >>>>>>>>>
> > >> > > > >>>>>>>>> 4.
> > >> > > > >>>>>>>>> Hmm, this is an interesting point. Are you suggesting
> > that
> > >> > > > >> under
> > >> > > > >>>> ALOS
> > >> > > > >>>>>>>>> READ_COMMITTED should not be supported?
> > >> > > > >>>>>>>>>
> > >> > > > >>>>>>>>> Regards,
> > >> > > > >>>>>>>>> Nick
> > >> > > > >>>>>>>>>
> > >> > > > >>>>>>>>> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
> > >> > > > >> cadonna@apache.org>
> > >> > > > >>>>>> wrote:
> > >> > > > >>>>>>>>>
> > >> > > > >>>>>>>>>> Hi Nick,
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>> I think the KIP is converging!
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>> 1.
> > >> > > > >>>>>>>>>> I am wondering whether it makes sense to write the
> > >> position
> > >> > > > >> file
> > >> > > > >>>>>> during
> > >> > > > >>>>>>>>>> close as we do for the checkpoint file, so that in case
> > >> the
> > >> > > > >>>> state
> > >> > > > >>>>>> store
> > >> > > > >>>>>>>>>> is replaced with a non-transactional state store the
> > >> > > > >>>>>> non-transactional
> > >> > > > >>>>>>>>>> state store finds the position file. I think, this is
> > not
> > >> > > > >>>> strictly
> > >> > > > >>>>>>>>>> needed, but would be a nice behavior instead of just
> > >> > > > >> deleting
> > >> > > > >>>> the
> > >> > > > >>>>>>>>>> position file.
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>> 2.
> > >> > > > >>>>>>>>>> The test plan does not mention integration tests. Do
> > you
> > >> not
> > >> > > > >>>> need to
> > >> > > > >>>>>>>>>> extend existing ones and add new ones. Also for
> > upgrading
> > >> > > > >> and
> > >> > > > >>>>>>>>>> downgrading you might need integration and/or system
> > >> tests.
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>> 3.
> > >> > > > >>>>>>>>>> I think Sophie made a point. Although, IQ reading from
> > >> > > > >>>> uncommitted
> > >> > > > >>>>>> data
> > >> > > > >>>>>>>>>> under EOS might be considered a bug by some people.
> > Thus,
> > >> > > > >> your
> > >> > > > >>>> KIP
> > >> > > > >>>>>> would
> > >> > > > >>>>>>>>>> fix a bug rather than changing the intended behavior.
> > >> > > > >> However, I
> > >> > > > >>>>>> also
> > >> > > > >>>>>>>>>> see that a feature flag would help users that rely on
> > >> this
> > >> > > > >> buggy
> > >> > > > >>>>>>>>>> behavior (at least until AK 4.0).
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>> 4.
> > >> > > > >>>>>>>>>> This is related to the previous point. I assume that
> > the
> > >> > > > >>>> difference
> > >> > > > >>>>>>>>>> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is
> > >> > > > >> that in
> > >> > > > >>>> the
> > >> > > > >>>>>>>>>> former you enable transactions on the state store and
> > in
> > >> the
> > >> > > > >>>> latter
> > >> > > > >>>>>> you
> > >> > > > >>>>>>>>>> disable them. If my assumption is correct, I think that
> > >> is
> > >> > > > >> an
> > >> > > > >>>> issue.
> > >> > > > >>>>>>>>>> Let's assume under ALOS Streams fails over a couple of
> > >> times
> > >> > > > >>>> more or
> > >> > > > >>>>>>>>>> less at the same step in processing after value 3 is
> > >> added
> > >> > > > >> to an
> > >> > > > >>>>>>>>>> aggregation but the offset of the corresponding input
> > >> record
> > >> > > > >>>> was not
> > >> > > > >>>>>>>>>> committed. Without transactions disabled, the
> > aggregation
> > >> > > > >> value
> > >> > > > >>>>>> would
> > >> > > > >>>>>>>>>> increase by 3 for each failover. With transactions
> > >> enabled,
> > >> > > > >>>> value 3
> > >> > > > >>>>>>>>>> would only be added to the aggregation once when the
> > >> offset
> > >> > > > >> of
> > >> > > > >>>> the
> > >> > > > >>>>>> input
> > >> > > > >>>>>>>>>> record is committed and the transaction finally
> > >> completes.
> > >> > > > >> So
> > >> > > > >>>> the
> > >> > > > >>>>>>>>>> content of the state store would change depending on
> > the
> > >> > > > >>>>>> configuration
> > >> > > > >>>>>>>>>> for IQ. IMO, the content of the state store should be
> > >> > > > >>>> independent
> > >> > > > >>>>>> from
> > >> > > > >>>>>>>>>> IQ. Given this issue, I propose to not use transactions
> > >> with
> > >> > > > >>>> ALOS at
> > >> > > > >>>>>>>>>> all. I was a big proponent of using transactions with
> > >> ALOS,
> > >> > > > >> but
> > >> > > > >>>> I
> > >> > > > >>>>>>>>>> realized that transactions with ALOS is not as easy as
> > >> > > > >> enabling
> > >> > > > >>>>>>>>>> transactions on state stores. Another aspect that is
> > >> > > > >>>> problematic is
> > >> > > > >>>>>> that
> > >> > > > >>>>>>>>>> the changelog topic which actually replicates the state
> > >> > > > >> store
> > >> > > > >>>> is not
> > >> > > > >>>>>>>>>> transactional under ALOS. Thus, it might happen that
> > the
> > >> > > > >> state
> > >> > > > >>>>>> store and
> > >> > > > >>>>>>>>>> the changelog differ in their content. All of this is
> > >> maybe
> > >> > > > >>>> solvable
> > >> > > > >>>>>>>>>> somehow, but for the sake of this KIP, I would leave it
> > >> for
> > >> > > > >> the
> > >> > > > >>>>>> future.
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>> Best,
> > >> > > > >>>>>>>>>> Bruno
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > >> > > > >>>>>>>>>>> Hey Nick! First of all thanks for taking up this
> > awesome
> > >> > > > >>>> feature,
> > >> > > > >>>>>> I'm
> > >> > > > >>>>>>>>>> sure
> > >> > > > >>>>>>>>>>> every single
> > >> > > > >>>>>>>>>>> Kafka Streams user and dev would agree that it is
> > sorely
> > >> > > > >>>> needed.
> > >> > > > >>>>>>>>>>>
> > >> > > > >>>>>>>>>>> I've just been catching up on the KIP and surrounding
> > >> > > > >>>> discussion,
> > >> > > > >>>>>> so
> > >> > > > >>>>>>>>>> please
> > >> > > > >>>>>>>>>>> forgive me
> > >> > > > >>>>>>>>>>> for any misunderstandings or misinterpretations of the
> > >> > > > >> current
> > >> > > > >>>>>> plan and
> > >> > > > >>>>>>>>>>> don't hesitate to
> > >> > > > >>>>>>>>>>> correct me.
> > >> > > > >>>>>>>>>>>
> > >> > > > >>>>>>>>>>> Before I jump in, I just want to say that having seen
> > >> this
> > >> > > > >>>> drag on
> > >> > > > >>>>>> for
> > >> > > > >>>>>>>> so
> > >> > > > >>>>>>>>>>> long, my singular
> > >> > > > >>>>>>>>>>> goal in responding is to help this KIP past a
> > perceived
> > >> > > > >>>> impasse so
> > >> > > > >>>>>> we
> > >> > > > >>>>>>>> can
> > >> > > > >>>>>>>>>>> finally move on
> > >> > > > >>>>>>>>>>> to voting and implementing it. Long discussions are to
> > >> be
> > >> > > > >>>> expected
> > >> > > > >>>>>> for
> > >> > > > >>>>>>>>>>> major features like
> > >> > > > >>>>>>>>>>> this but it's completely on us as the Streams devs to
> > >> make
> > >> > > > >> sure
> > >> > > > >>>>>> there
> > >> > > > >>>>>>>> is
> > >> > > > >>>>>>>>>> an
> > >> > > > >>>>>>>>>>> end in sight
> > >> > > > >>>>>>>>>>> for any ongoing discussion.
> > >> > > > >>>>>>>>>>>
> > >> > > > >>>>>>>>>>> With that said, it's my understanding that the KIP as
> > >> > > > >> currently
> > >> > > > >>>>>>>> proposed
> > >> > > > >>>>>>>>>> is
> > >> > > > >>>>>>>>>>> just not tenable
> > >> > > > >>>>>>>>>>> for Kafka Streams, and would prevent some EOS users
> > from
> > >> > > > >>>> upgrading
> > >> > > > >>>>>> to
> > >> > > > >>>>>>>> the
> > >> > > > >>>>>>>>>>> version it
> > >> > > > >>>>>>>>>>> first appears in. Given that we can't predict or
> > >> guarantee
> > >> > > > >>>> whether
> > >> > > > >>>>>> any
> > >> > > > >>>>>>>> of
> > >> > > > >>>>>>>>>>> the followup KIPs
> > >> > > > >>>>>>>>>>> would be completed in the same release cycle as this
> > >> one,
> > >> > > > >> we
> > >> > > > >>>> need
> > >> > > > >>>>>> to
> > >> > > > >>>>>>>> make
> > >> > > > >>>>>>>>>>> sure that the
> > >> > > > >>>>>>>>>>> feature is either compatible with all current users or
> > >> else
> > >> > > > >>>>>>>>>> feature-flagged
> > >> > > > >>>>>>>>>>> so that they may
> > >> > > > >>>>>>>>>>> opt in/out.
> > >> > > > >>>>>>>>>>>
> > >> > > > >>>>>>>>>>> Therefore, IIUC we need to have either (or both) of
> > >> these
> > >> > > > >> as
> > >> > > > >>>>>>>>>>> fully-implemented config options:
> > >> > > > >>>>>>>>>>> 1. default.state.isolation.level
> > >> > > > >>>>>>>>>>> 2. enable.transactional.state.stores
> > >> > > > >>>>>>>>>>>
> > >> > > > >>>>>>>>>>> This way EOS users for whom read_committed semantics
> > are
> > >> > > > >> not
> > >> > > > >>>>>> viable can
> > >> > > > >>>>>>>>>>> still upgrade,
> > >> > > > >>>>>>>>>>> and either use the isolation.level config to leverage
> > >> the
> > >> > > > >> new
> > >> > > > >>>> txn
> > >> > > > >>>>>> state
> > >> > > > >>>>>>>>>>> stores without sacrificing
> > >> > > > >>>>>>>>>>> their application semantics, or else simply keep the
> > >> > > > >>>> transactional
> > >> > > > >>>>>>>> state
> > >> > > > >>>>>>>>>>> stores disabled until we
> > >> > > > >>>>>>>>>>> are able to fully implement the isolation level
> > >> > > > >> configuration
> > >> > > > >>>> at
> > >> > > > >>>>>> either
> > >> > > > >>>>>>>>>> an
> > >> > > > >>>>>>>>>>> application or query level.
> > >> > > > >>>>>>>>>>>
> > >> > > > >>>>>>>>>>> Frankly you are the expert here and know much more
> > about
> > >> > > > >> the
> > >> > > > >>>>>> tradeoffs
> > >> > > > >>>>>>>> in
> > >> > > > >>>>>>>>>>> both semantics and
> > >> > > > >>>>>>>>>>> effort level of implementing one of these configs vs
> > the
> > >> > > > >>>> other. In
> > >> > > > >>>>>> my
> > >> > > > >>>>>>>>>>> opinion, either option would
> > >> > > > >>>>>>>>>>> be fine and I would leave the decision of which one to
> > >> > > > >> include
> > >> > > > >>>> in
> > >> > > > >>>>>> this
> > >> > > > >>>>>>>>>> KIP
> > >> > > > >>>>>>>>>>> completely up to you.
> > >> > > > >>>>>>>>>>> I just don't see a way for the KIP to proceed without
> > >> some
> > >> > > > >>>>>> variation of
> > >> > > > >>>>>>>>>> the
> > >> > > > >>>>>>>>>>> above that would allow
> > >> > > > >>>>>>>>>>> EOS users to opt-out of read_committed.
> > >> > > > >>>>>>>>>>>
> > >> > > > >>>>>>>>>>> (If it's all the same to you, I would recommend 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 that gets released.
> > >> > > > >>>>>>>>>>> So personally, my choice would be to add the feature
> > >> flag
> > >> > > > >> and
> > >> > > > >>>>>> leave it
> > >> > > > >>>>>>>>>> off
> > >> > > > >>>>>>>>>>> by default. If all 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
> > >> > > > >>>>>>>>>>>
> > >> > > > >>>>>>>>>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > >> > > > >>>>>> nick.telford@gmail.com>
> > >> > > > >>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>
> > >> > > > >>>>>>>>>>>> Hi Bruno,
> > >> > > > >>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>> Agreed, I can live with that for now.
> > >> > > > >>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>> In an effort to keep the scope of this KIP from
> > >> > > > >> expanding, I'm
> > >> > > > >>>>>> leaning
> > >> > > > >>>>>>>>>>>> towards just providing a configurable
> > >> > > > >>>>>> default.state.isolation.level
> > >> > > > >>>>>>>> and
> > >> > > > >>>>>>>>>>>> removing IsolationLevel from the StateStoreContext.
> > >> This
> > >> > > > >>>> would be
> > >> > > > >>>>>>>>>>>> compatible with adding support for query-time
> > >> > > > >> IsolationLevels
> > >> > > > >>>> in
> > >> > > > >>>>>> the
> > >> > > > >>>>>>>>>>>> future, whilst providing a way for users to select an
> > >> > > > >>>> isolation
> > >> > > > >>>>>> level
> > >> > > > >>>>>>>>>> now.
> > >> > > > >>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>> The big problem with this, however, is that if a user
> > >> > > > >> selects
> > >> > > > >>>>>>>>>>>> processing.mode
> > >> > > > >>>>>>>>>>>> = "exactly-once(-v2|-beta)", and
> > >> > > > >>>> default.state.isolation.level =
> > >> > > > >>>>>>>>>>>> "READ_UNCOMMITTED", we need to guarantee that the
> > data
> > >> > > > >> isn't
> > >> > > > >>>>>> written
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>> disk until commit() is called, but we also need to
> > >> permit
> > >> > > > >> IQ
> > >> > > > >>>>>> threads
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>> read from the ongoing transaction.
> > >> > > > >>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>> A simple solution would be to (temporarily) forbid
> > this
> > >> > > > >>>>>> combination of
> > >> > > > >>>>>>>>>>>> configuration, and have default.state.isolation.level
> > >> > > > >>>>>> automatically
> > >> > > > >>>>>>>>>> switch
> > >> > > > >>>>>>>>>>>> to READ_COMMITTED when processing.mode is anything
> > >> other
> > >> > > > >> than
> > >> > > > >>>>>>>>>>>> at-least-once. Do you think this would be acceptable?
> > >> > > > >>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>> In a later KIP, we can add support for query-time
> > >> > > > >> isolation
> > >> > > > >>>>>> levels and
> > >> > > > >>>>>>>>>>>> solve this particular problem there, which would
> > relax
> > >> > > > >> this
> > >> > > > >>>>>>>> restriction.
> > >> > > > >>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>> Regards,
> > >> > > > >>>>>>>>>>>> Nick
> > >> > > > >>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> > >> > > > >>>> cadonna@apache.org>
> > >> > > > >>>>>>>>>> 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,
> > >> > > > >>>>>>>>>>>>> Bruno
> > >> > > > >>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > >> > > > >>>>>>>>>>>>>> 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 <
> > >> > > > >>>>>> nick.telford@gmail.com
> > >> > > > >>>>>>>>>
> > >> > > > >>>>>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> Hi everyone,
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> I agree that having IsolationLevel be determined
> > at
> > >> > > > >>>> query-time
> > >> > > > >>>>>> is
> > >> > > > >>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>> ideal design, but there are a few sticking points:
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> 1.
> > >> > > > >>>>>>>>>>>>>>> There needs to be some way to communicate the
> > >> > > > >>>> IsolationLevel
> > >> > > > >>>>>> down
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>> RocksDBStore itself, so that the query can respect
> > >> it.
> > >> > > > >>>> Since
> > >> > > > >>>>>> stores
> > >> > > > >>>>>>>>>>>> are
> > >> > > > >>>>>>>>>>>>>>> "layered" in functionality (i.e.
> > ChangeLoggingStore,
> > >> > > > >>>>>> MeteredStore,
> > >> > > > >>>>>>>>>>>>> etc.),
> > >> > > > >>>>>>>>>>>>>>> we need some way to deliver that information to
> > the
> > >> > > > >> bottom
> > >> > > > >>>>>> layer.
> > >> > > > >>>>>>>> For
> > >> > > > >>>>>>>>>>>>> IQv2,
> > >> > > > >>>>>>>>>>>>>>> we can use the existing State#query() method, but
> > >> IQv1
> > >> > > > >> has
> > >> > > > >>>> no
> > >> > > > >>>>>> way
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>> do
> > >> > > > >>>>>>>>>>>>>>> this.
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> A simple approach, which would potentially open up
> > >> > > > >> other
> > >> > > > >>>>>> options,
> > >> > > > >>>>>>>>>>>> would
> > >> > > > >>>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> > >> > > > >>>>>>>>>>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > >> > > > >>>>>>>> ReadOnlyKeyValueStore
> > >> > > > >>>>>>>>>>>>> (and
> > >> > > > >>>>>>>>>>>>>>> similar to ReadOnlyWindowStore,
> > >> ReadOnlySessionStore,
> > >> > > > >>>> etc.).
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> 2.
> > >> > > > >>>>>>>>>>>>>>> As mentioned above, RocksDB WriteBatches are not
> > >> > > > >>>> thread-safe,
> > >> > > > >>>>>> which
> > >> > > > >>>>>>>>>>>>> causes
> > >> > > > >>>>>>>>>>>>>>> a problem if we want to provide READ_UNCOMMITTED
> > >> > > > >>>> Iterators. I
> > >> > > > >>>>>> also
> > >> > > > >>>>>>>>>>>> had a
> > >> > > > >>>>>>>>>>>>>>> look at RocksDB Transactions[1], but they solve a
> > >> very
> > >> > > > >>>>>> different
> > >> > > > >>>>>>>>>>>>> problem,
> > >> > > > >>>>>>>>>>>>>>> and have the same thread-safety issue.
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> One possible approach that I mentioned is chaining
> > >> > > > >>>>>> WriteBatches:
> > >> > > > >>>>>>>>>> every
> > >> > > > >>>>>>>>>>>>>>> time a new Interactive Query is received (i.e.
> > >> > > > >>>> readOnlyView,
> > >> > > > >>>>>> see
> > >> > > > >>>>>>>>>>>> above,
> > >> > > > >>>>>>>>>>>>>>> is called) we "freeze" the existing WriteBatch,
> > and
> > >> > > > >> start a
> > >> > > > >>>>>> new one
> > >> > > > >>>>>>>>>>>> for
> > >> > > > >>>>>>>>>>>>> new
> > >> > > > >>>>>>>>>>>>>>> writes. The Interactive Query queries the "chain"
> > of
> > >> > > > >>>> previous
> > >> > > > >>>>>>>>>>>>> WriteBatches
> > >> > > > >>>>>>>>>>>>>>> + the underlying database; while the StreamThread
> > >> > > > >> starts
> > >> > > > >>>>>> writing to
> > >> > > > >>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>> *new* WriteBatch. On-commit, the StreamThread
> > would
> > >> > > > >> write
> > >> > > > >>>> *all*
> > >> > > > >>>>>>>>>>>>>>> WriteBatches in the chain to the database (that
> > have
> > >> > > > >> not
> > >> > > > >>>> yet
> > >> > > > >>>>>> been
> > >> > > > >>>>>>>>>>>>> written).
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> WriteBatches would be closed/freed only when they
> > >> have
> > >> > > > >> been
> > >> > > > >>>>>> both
> > >> > > > >>>>>>>>>>>>>>> committed, and all open Interactive Queries on
> > them
> > >> > > > >> have
> > >> > > > >>>> been
> > >> > > > >>>>>>>> closed.
> > >> > > > >>>>>>>>>>>>> This
> > >> > > > >>>>>>>>>>>>>>> would require some reference counting.
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> Obviously a drawback of this approach is the
> > >> potential
> > >> > > > >> for
> > >> > > > >>>>>>>> increased
> > >> > > > >>>>>>>>>>>>>>> memory usage: if an Interactive Query is
> > long-lived,
> > >> > > > >> for
> > >> > > > >>>>>> example by
> > >> > > > >>>>>>>>>>>>> doing a
> > >> > > > >>>>>>>>>>>>>>> full scan over a large database, or even just
> > >> pausing
> > >> > > > >> in
> > >> > > > >>>> the
> > >> > > > >>>>>> middle
> > >> > > > >>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>> an
> > >> > > > >>>>>>>>>>>>>>> iteration, then the existing chain of WriteBatches
> > >> > > > >> could be
> > >> > > > >>>>>> kept
> > >> > > > >>>>>>>>>>>> around
> > >> > > > >>>>>>>>>>>>> for
> > >> > > > >>>>>>>>>>>>>>> a long time, potentially forever.
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> --
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> A.
> > >> > > > >>>>>>>>>>>>>>> Going off on a tangent, it looks like in addition
> > to
> > >> > > > >>>> supporting
> > >> > > > >>>>>>>>>>>>>>> READ_COMMITTED queries, we could go further and
> > >> support
> > >> > > > >>>>>>>>>>>> REPEATABLE_READ
> > >> > > > >>>>>>>>>>>>>>> queries (i.e. where subsequent reads to the same
> > key
> > >> > > > >> in the
> > >> > > > >>>>>> same
> > >> > > > >>>>>>>>>>>>>>> Interactive Query are guaranteed to yield the same
> > >> > > > >> value)
> > >> > > > >>>> by
> > >> > > > >>>>>> making
> > >> > > > >>>>>>>>>>>> use
> > >> > > > >>>>>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>> RocksDB Snapshots[2]. These are fairly
> > lightweight,
> > >> so
> > >> > > > >> the
> > >> > > > >>>>>>>>>> performance
> > >> > > > >>>>>>>>>>>>>>> impact is likely to be negligible, but they do
> > >> require
> > >> > > > >>>> that the
> > >> > > > >>>>>>>>>>>>> Interactive
> > >> > > > >>>>>>>>>>>>>>> Query session can be explicitly closed.
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> This could be achieved if we made the above
> > >> > > > >> readOnlyView
> > >> > > > >>>>>> interface
> > >> > > > >>>>>>>>>>>> look
> > >> > > > >>>>>>>>>>>>>>> more like:
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> > >> > > > >>>>>>>>>>>> ReadOnlyKeyValueStore<K,
> > >> > > > >>>>>>>>>>>>>>> V>, AutoCloseable {}
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > >> > > > >>>>>>>>>>>>>>>         ...
> > >> > > > >>>>>>>>>>>>>>>         ReadOnlyKeyValueView<K, V>
> > >> > > > >>>> readOnlyView(IsolationLevel
> > >> > > > >>>>>>>>>>>>> isolationLevel);
> > >> > > > >>>>>>>>>>>>>>> }
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> But this would be a breaking change, as existing
> > >> IQv1
> > >> > > > >>>> queries
> > >> > > > >>>>>> are
> > >> > > > >>>>>>>>>>>>>>> guaranteed to never call store.close(), and
> > >> therefore
> > >> > > > >> these
> > >> > > > >>>>>> would
> > >> > > > >>>>>>>>>> leak
> > >> > > > >>>>>>>>>>>>>>> memory under REPEATABLE_READ.
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> B.
> > >> > > > >>>>>>>>>>>>>>> One thing that's notable: MyRocks states that they
> > >> > > > >> support
> > >> > > > >>>>>>>>>>>>> READ_COMMITTED
> > >> > > > >>>>>>>>>>>>>>> and REPEATABLE_READ, but they make no mention of
> > >> > > > >>>>>>>>>>>> READ_UNCOMMITTED[3][4].
> > >> > > > >>>>>>>>>>>>>>> This could be because doing so is technically
> > >> > > > >>>>>> difficult/impossible
> > >> > > > >>>>>>>>>>>> using
> > >> > > > >>>>>>>>>>>>>>> the primitives available in RocksDB.
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> --
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> Lucas, to address your points:
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> U1.
> > >> > > > >>>>>>>>>>>>>>> It's only "SHOULD" to permit alternative (i.e.
> > >> > > > >> non-RocksDB)
> > >> > > > >>>>>>>>>>>>>>> implementations of StateStore that do not support
> > >> > > > >> atomic
> > >> > > > >>>>>> writes.
> > >> > > > >>>>>>>>>>>>> Obviously
> > >> > > > >>>>>>>>>>>>>>> in those cases, the guarantees Kafka Streams
> > >> > > > >>>> provides/expects
> > >> > > > >>>>>> would
> > >> > > > >>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>> relaxed. Do you think we should require all
> > >> > > > >>>> implementations to
> > >> > > > >>>>>>>>>> support
> > >> > > > >>>>>>>>>>>>>>> atomic writes?
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> U2.
> > >> > > > >>>>>>>>>>>>>>> Stores can support multiple IsolationLevels. As
> > >> we've
> > >> > > > >>>> discussed
> > >> > > > >>>>>>>>>> above,
> > >> > > > >>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>> ideal scenario would be to specify the
> > >> IsolationLevel
> > >> > > > >> at
> > >> > > > >>>>>>>> query-time.
> > >> > > > >>>>>>>>>>>>>>> Failing that, I think the second-best approach is
> > to
> > >> > > > >>>> define the
> > >> > > > >>>>>>>>>>>>>>> IsolationLevel for *all* queries based on the
> > >> > > > >>>> processing.mode,
> > >> > > > >>>>>>>> which
> > >> > > > >>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>> what the default
> > StateStoreContext#isolationLevel()
> > >> > > > >>>> achieves.
> > >> > > > >>>>>> Would
> > >> > > > >>>>>>>>>>>> you
> > >> > > > >>>>>>>>>>>>>>> prefer an alternative?
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> While the existing implementation is equivalent to
> > >> > > > >>>>>>>> READ_UNCOMMITTED,
> > >> > > > >>>>>>>>>>>>> this
> > >> > > > >>>>>>>>>>>>>>> can yield unexpected results/errors under EOS, if
> > a
> > >> > > > >>>>>> transaction is
> > >> > > > >>>>>>>>>>>>> rolled
> > >> > > > >>>>>>>>>>>>>>> back. While this would be a change in behaviour
> > for
> > >> > > > >> users,
> > >> > > > >>>> it
> > >> > > > >>>>>> would
> > >> > > > >>>>>>>>>>>> look
> > >> > > > >>>>>>>>>>>>>>> more like a bug fix than a breaking change. That
> > >> said,
> > >> > > > >> we
> > >> > > > >>>>>> *could*
> > >> > > > >>>>>>>>>> make
> > >> > > > >>>>>>>>>>>>> it
> > >> > > > >>>>>>>>>>>>>>> configurable, and default to the existing
> > behaviour
> > >> > > > >>>>>>>>>> (READ_UNCOMMITTED)
> > >> > > > >>>>>>>>>>>>>>> instead of inferring it from the processing.mode?
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> N1, N2.
> > >> > > > >>>>>>>>>>>>>>> These were only primitives to avoid boxing costs,
> > >> but
> > >> > > > >> since
> > >> > > > >>>>>> this is
> > >> > > > >>>>>>>>>>>> not
> > >> > > > >>>>>>>>>>>>> a
> > >> > > > >>>>>>>>>>>>>>> performance sensitive area, it should be fine to
> > >> > > > >> change if
> > >> > > > >>>>>> that's
> > >> > > > >>>>>>>>>>>>> desirable.
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> N3.
> > >> > > > >>>>>>>>>>>>>>> It's because the store "manages its own offsets",
> > >> which
> > >> > > > >>>>>> includes
> > >> > > > >>>>>>>> both
> > >> > > > >>>>>>>>>>>>>>> committing the offset, *and providing it* via
> > >> > > > >>>>>> getCommittedOffset().
> > >> > > > >>>>>>>>>>>>>>> Personally, I think "managesOffsets" conveys this
> > >> best,
> > >> > > > >>>> but I
> > >> > > > >>>>>> don't
> > >> > > > >>>>>>>>>>>> mind
> > >> > > > >>>>>>>>>>>>>>> changing it if the nomenclature is unclear.
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> Sorry for the massive emails/essays!
> > >> > > > >>>>>>>>>>>>>>> --
> > >> > > > >>>>>>>>>>>>>>> Nick
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> 1:
> > >> > > > >> https://github.com/facebook/rocksdb/wiki/Transactions
> > >> > > > >>>>>>>>>>>>>>> 2:
> > >> https://github.com/facebook/rocksdb/wiki/Snapshot
> > >> > > > >>>>>>>>>>>>>>> 3:
> > >> > > > >>>>>>>>
> > >> > > > >>
> > https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > >> > > > >>>>>>>>>>>>>>> 4:
> > >> > > > >>>> https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > >> > > > >>>>>>>>>>>>>>> <lb...@confluent.io.invalid> wrote:
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>> Hi Nick,
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>> since I last read it in April, the KIP has become
> > >> much
> > >> > > > >>>>>> cleaner and
> > >> > > > >>>>>>>>>>>>>>>> easier to read. Great work!
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>> It feels to me the last big open point is whether
> > >> we
> > >> > > > >> can
> > >> > > > >>>>>> implement
> > >> > > > >>>>>>>>>>>>>>>> isolation level as a query parameter. I
> > understand
> > >> > > > >> that
> > >> > > > >>>> there
> > >> > > > >>>>>> are
> > >> > > > >>>>>>>>>>>>>>>> implementation concerns, but as Colt says, it
> > would
> > >> > > > >> be a
> > >> > > > >>>> great
> > >> > > > >>>>>>>>>>>>>>>> addition, and would also simplify the migration
> > >> path
> > >> > > > >> for
> > >> > > > >>>> this
> > >> > > > >>>>>>>>>> change.
> > >> > > > >>>>>>>>>>>>>>>> Is the implementation problem you mentioned
> > caused
> > >> by
> > >> > > > >> the
> > >> > > > >>>>>>>> WriteBatch
> > >> > > > >>>>>>>>>>>>>>>> not having a notion of a snapshot, as the
> > >> underlying
> > >> > > > >> DB
> > >> > > > >>>>>> iterator
> > >> > > > >>>>>>>>>>>> does?
> > >> > > > >>>>>>>>>>>>>>>> In that case, I am not sure a chain of
> > WriteBatches
> > >> > > > >> as you
> > >> > > > >>>>>> propose
> > >> > > > >>>>>>>>>>>>>>>> would fully solve the problem, but maybe I didn't
> > >> dig
> > >> > > > >>>> enough
> > >> > > > >>>>>> into
> > >> > > > >>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>> details to fully understand it.
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>> If it's not possible to implement it now, would
> > it
> > >> be
> > >> > > > >> an
> > >> > > > >>>>>> option to
> > >> > > > >>>>>>>>>>>>>>>> make sure in this KIP that we do not fully close
> > >> the
> > >> > > > >> door
> > >> > > > >>>> on
> > >> > > > >>>>>>>>>>>> per-query
> > >> > > > >>>>>>>>>>>>>>>> isolation levels in the interface, as it may be
> > >> > > > >> possible
> > >> > > > >>>> to
> > >> > > > >>>>>>>>>> implement
> > >> > > > >>>>>>>>>>>>>>>> the missing primitives in RocksDB or Speedb in
> > the
> > >> > > > >> future.
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>> Understanding:
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>> * U1) Why is it only "SHOULD" for
> > changelogOffsets
> > >> to
> > >> > > > >> be
> > >> > > > >>>>>> persisted
> > >> > > > >>>>>>>>>>>>>>>> atomically with the records?
> > >> > > > >>>>>>>>>>>>>>>> * U2) Don't understand the default implementation
> > >> of
> > >> > > > >>>>>>>>>>>> `isolationLevel`.
> > >> > > > >>>>>>>>>>>>>>>> The isolation level should be a property of the
> > >> > > > >> underlying
> > >> > > > >>>>>> store,
> > >> > > > >>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>> not be defined by the default config? Existing
> > >> stores
> > >> > > > >>>> probably
> > >> > > > >>>>>>>> don't
> > >> > > > >>>>>>>>>>>>>>>> guarantee READ_COMMITTED, so the default should
> > be
> > >> to
> > >> > > > >>>> return
> > >> > > > >>>>>>>>>>>>>>>> READ_UNCOMMITTED.
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>> Nits:
> > >> > > > >>>>>>>>>>>>>>>> * N1) Could `getComittedOffset` use an
> > >> `OptionalLong`
> > >> > > > >>>> return
> > >> > > > >>>>>> type,
> > >> > > > >>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> avoid the `null`?
> > >> > > > >>>>>>>>>>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use
> > an
> > >> > > > >>>>>> `OptionalLong`
> > >> > > > >>>>>>>>>>>>>>>> return type, to avoid the `-1`?
> > >> > > > >>>>>>>>>>>>>>>> * N3) I don't understand why `managesOffsets`
> > uses
> > >> the
> > >> > > > >>>>>> 'manage'
> > >> > > > >>>>>>>>>> verb,
> > >> > > > >>>>>>>>>>>>>>>> whereas all other methods use the "commits" verb.
> > >> I'd
> > >> > > > >>>> suggest
> > >> > > > >>>>>>>>>>>>>>>> `commitsOffsets`.
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>> Either way, it feels this KIP is very close to
> > the
> > >> > > > >> finish
> > >> > > > >>>>>> line,
> > >> > > > >>>>>>>> I'm
> > >> > > > >>>>>>>>>>>>>>>> looking forward to seeing this in production!
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>> Cheers,
> > >> > > > >>>>>>>>>>>>>>>> Lucas
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> > >> > > > >>>>>> colt@littlehorse.io
> > >> > > > >>>>>>>>>
> > >> > > > >>>>>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>> Making IsolationLevel a query-time constraint,
> > >> > > > >> rather
> > >> > > > >>>> than
> > >> > > > >>>>>>>> linking
> > >> > > > >>>>>>>>>>>> it
> > >> > > > >>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>> the processing.guarantee.
> > >> > > > >>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>> As I understand it, would this allow even a user
> > >> of
> > >> > > > >> EOS
> > >> > > > >>>> to
> > >> > > > >>>>>>>> control
> > >> > > > >>>>>>>>>>>>>>>> whether
> > >> > > > >>>>>>>>>>>>>>>>> reading committed or uncommitted records? If so,
> > >> I am
> > >> > > > >>>> highly
> > >> > > > >>>>>> in
> > >> > > > >>>>>>>>>>>> favor
> > >> > > > >>>>>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>> this.
> > >> > > > >>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>> I know that I was one of the early people to
> > point
> > >> > > > >> out
> > >> > > > >>>> the
> > >> > > > >>>>>>>> current
> > >> > > > >>>>>>>>>>>>>>>>> shortcoming that IQ reads uncommitted records,
> > but
> > >> > > > >> just
> > >> > > > >>>> this
> > >> > > > >>>>>>>>>>>> morning I
> > >> > > > >>>>>>>>>>>>>>>>> realized a pattern we use which means that (for
> > >> > > > >> certain
> > >> > > > >>>>>> queries)
> > >> > > > >>>>>>>>>> our
> > >> > > > >>>>>>>>>>>>>>>> system
> > >> > > > >>>>>>>>>>>>>>>>> needs to be able to read uncommitted records,
> > >> which
> > >> > > > >> is
> > >> > > > >>>> the
> > >> > > > >>>>>>>> current
> > >> > > > >>>>>>>>>>>>>>>> behavior
> > >> > > > >>>>>>>>>>>>>>>>> of Kafka Streams in EOS.***
> > >> > > > >>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>> If IsolationLevel being a query-time decision
> > >> allows
> > >> > > > >> for
> > >> > > > >>>>>> this,
> > >> > > > >>>>>>>> then
> > >> > > > >>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>> would be amazing. I would also vote that the
> > >> default
> > >> > > > >>>> behavior
> > >> > > > >>>>>>>>>> should
> > >> > > > >>>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>> for
> > >> > > > >>>>>>>>>>>>>>>>> reading uncommitted records, because it is
> > totally
> > >> > > > >>>> possible
> > >> > > > >>>>>> for a
> > >> > > > >>>>>>>>>>>>> valid
> > >> > > > >>>>>>>>>>>>>>>>> application to depend on that behavior, and
> > >> breaking
> > >> > > > >> it
> > >> > > > >>>> in a
> > >> > > > >>>>>>>> minor
> > >> > > > >>>>>>>>>>>>>>>> release
> > >> > > > >>>>>>>>>>>>>>>>> might be a bit strong.
> > >> > > > >>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>> *** (Note, for the curious reader....) Our
> > >> > > > >> use-case/query
> > >> > > > >>>>>> pattern
> > >> > > > >>>>>>>>>>>> is a
> > >> > > > >>>>>>>>>>>>>>>> bit
> > >> > > > >>>>>>>>>>>>>>>>> complex, but reading "uncommitted" records is
> > >> > > > >> actually
> > >> > > > >>>> safe
> > >> > > > >>>>>> in
> > >> > > > >>>>>>>> our
> > >> > > > >>>>>>>>>>>>> case
> > >> > > > >>>>>>>>>>>>>>>>> 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 soon as the processor
> > >> processes
> > >> > > > >> the
> > >> > > > >>>>>> result.
> > >> > > > >>>>>>>> If
> > >> > > > >>>>>>>>>>>>> they
> > >> > > > >>>>>>>>>>>>>>>>> can't read uncommitted records, they may
> > receive a
> > >> > > > >> "201 -
> > >> > > > >>>>>>>> Created"
> > >> > > > >>>>>>>>>>>>>>>>> response, immediately followed by a "404 - 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 <
> > >> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> > >> > > > >>>>>>>>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>> Addendum:
> > >> > > > >>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>> I think we would also face the same problem
> > with
> > >> the
> > >> > > > >>>>>> approach
> > >> > > > >>>>>>>> John
> > >> > > > >>>>>>>>>>>>>>>> outlined
> > >> > > > >>>>>>>>>>>>>>>>>> earlier (using the record cache as a
> > transaction
> > >> > > > >> buffer
> > >> > > > >>>> and
> > >> > > > >>>>>>>>>>>> flushing
> > >> > > > >>>>>>>>>>>>>>>> it
> > >> > > > >>>>>>>>>>>>>>>>>> straight to SST files). This is because the
> > >> record
> > >> > > > >> cache
> > >> > > > >>>>>> (the
> > >> > > > >>>>>>>>>>>>>>>> ThreadCache
> > >> > > > >>>>>>>>>>>>>>>>>> class) is not thread-safe, so every commit
> > would
> > >> > > > >>>> invalidate
> > >> > > > >>>>>> open
> > >> > > > >>>>>>>>>> IQ
> > >> > > > >>>>>>>>>>>>>>>>>> Iterators in the same way that RocksDB
> > >> WriteBatches
> > >> > > > >> do.
> > >> > > > >>>>>>>>>>>>>>>>>> --
> > >> > > > >>>>>>>>>>>>>>>>>> Nick
> > >> > > > >>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > >> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> > >> > > > >>>>>>>>>>>>>>>>>> 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.
> > >> > > > >>>>>>>>>>>>>>>>>>> 2. Making IsolationLevel a query-time
> > >> constraint,
> > >> > > > >>>> rather
> > >> > > > >>>>>> than
> > >> > > > >>>>>>>>>>>>>>>> linking it
> > >> > > > >>>>>>>>>>>>>>>>>>> to the processing.guarantee.
> > >> > > > >>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>> There's a wrinkle that makes this a challenge:
> > >> > > > >>>> Interactive
> > >> > > > >>>>>>>>>> Queries
> > >> > > > >>>>>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>>> open an Iterator, when using transactions and
> > >> > > > >>>>>> READ_UNCOMMITTED.
> > >> > > > >>>>>>>>>>>>>>>>>>> The problem is that under READ_UNCOMMITTED,
> > >> queries
> > >> > > > >>>> need
> > >> > > > >>>>>> to be
> > >> > > > >>>>>>>>>>>> able
> > >> > > > >>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>> read records from the currently uncommitted
> > >> > > > >> transaction
> > >> > > > >>>>>> buffer
> > >> > > > >>>>>>>>>>>>>>>>>>> (WriteBatch). This includes for Iterators,
> > which
> > >> > > > >> should
> > >> > > > >>>>>> iterate
> > >> > > > >>>>>>>>>>>>>>>> both the
> > >> > > > >>>>>>>>>>>>>>>>>>> transaction buffer and underlying database
> > >> (using
> > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > >> > > > >>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>> The issue is that when the StreamThread
> > >> commits, it
> > >> > > > >>>> writes
> > >> > > > >>>>>> the
> > >> > > > >>>>>>>>>>>>>>>> current
> > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch to RocksDB *and then clears the
> > >> > > > >> WriteBatch*.
> > >> > > > >>>>>>>> Clearing
> > >> > > > >>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch while an Interactive Query holds an
> > >> open
> > >> > > > >>>>>> Iterator on
> > >> > > > >>>>>>>>>> it
> > >> > > > >>>>>>>>>>>>>>>> will
> > >> > > > >>>>>>>>>>>>>>>>>>> invalidate the Iterator. Worse, it turns out
> > >> that
> > >> > > > >>>> Iterators
> > >> > > > >>>>>>>> over
> > >> > > > >>>>>>>>>> a
> > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch become invalidated not just when
> > the
> > >> > > > >>>> WriteBatch
> > >> > > > >>>>>> is
> > >> > > > >>>>>>>>>>>>>>>> cleared,
> > >> > > > >>>>>>>>>>>>>>>>>> but
> > >> > > > >>>>>>>>>>>>>>>>>>> also when the Iterators' current key receives
> > a
> > >> new
> > >> > > > >>>> write.
> > >> > > > >>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>> Now that I'm writing this, I remember that
> > this
> > >> is
> > >> > > > >> the
> > >> > > > >>>>>> major
> > >> > > > >>>>>>>>>>>> reason
> > >> > > > >>>>>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>> I
> > >> > > > >>>>>>>>>>>>>>>>>>> switched the original design from having a
> > >> > > > >> query-time
> > >> > > > >>>>>>>>>>>>>>>> IsolationLevel to
> > >> > > > >>>>>>>>>>>>>>>>>>> having the IsolationLevel linked to the
> > >> > > > >>>> transactionality
> > >> > > > >>>>>> of the
> > >> > > > >>>>>>>>>>>>>>>> stores
> > >> > > > >>>>>>>>>>>>>>>>>>> themselves.
> > >> > > > >>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>> It *might* be possible to resolve this, by
> > >> having a
> > >> > > > >>>>>> "chain" of
> > >> > > > >>>>>>>>>>>>>>>>>>> WriteBatches, with the StreamThread switching
> > >> to a
> > >> > > > >> new
> > >> > > > >>>>>>>> WriteBatch
> > >> > > > >>>>>>>>>>>>>>>>>> whenever
> > >> > > > >>>>>>>>>>>>>>>>>>> a new Interactive Query attempts to read from
> > >> the
> > >> > > > >>>>>> database, but
> > >> > > > >>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>> could
> > >> > > > >>>>>>>>>>>>>>>>>>> cause some performance problems/memory
> > pressure
> > >> > > > >> when
> > >> > > > >>>>>> subjected
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>> a
> > >> > > > >>>>>>>>>>>>>>>> high
> > >> > > > >>>>>>>>>>>>>>>>>>> Interactive Query load. It would also reduce
> > the
> > >> > > > >>>>>> efficiency of
> > >> > > > >>>>>>>>>>>>>>>>>> WriteBatches
> > >> > > > >>>>>>>>>>>>>>>>>>> on-commit, as we'd have to write N
> > WriteBatches,
> > >> > > > >> where
> > >> > > > >>>> N
> > >> > > > >>>>>> is the
> > >> > > > >>>>>>>>>>>>>>>> number of
> > >> > > > >>>>>>>>>>>>>>>>>>> Interactive Queries since the last commit.
> > >> > > > >>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>> I realise this is getting into the weeds of
> > the
> > >> > > > >>>>>> implementation,
> > >> > > > >>>>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>> you'd
> > >> > > > >>>>>>>>>>>>>>>>>>> rather we focus on the API for now, but I
> > think
> > >> > > > >> it's
> > >> > > > >>>>>> important
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>> consider
> > >> > > > >>>>>>>>>>>>>>>>>>> how to implement the desired API, in case we
> > >> come
> > >> > > > >> up
> > >> > > > >>>> with
> > >> > > > >>>>>> an
> > >> > > > >>>>>>>> API
> > >> > > > >>>>>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>>> cannot be implemented efficiently, or even at
> > >> all!
> > >> > > > >>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>> Thoughts?
> > >> > > > >>>>>>>>>>>>>>>>>>> --
> > >> > > > >>>>>>>>>>>>>>>>>>> Nick
> > >> > > > >>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> > >> > > > >>>>>>>> cadonna@apache.org
> > >> > > > >>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > >> > > > >>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>> 6.
> > >> > > > >>>>>>>>>>>>>>>>>>>> Of course, you are right! My bad!
> > >> > > > >>>>>>>>>>>>>>>>>>>> Wiping out the state in the downgrading case
> > is
> > >> > > > >> fine.
> > >> > > > >>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>> 3a.
> > >> > > > >>>>>>>>>>>>>>>>>>>> Focus on the public facing changes for the
> > >> KIP. We
> > >> > > > >>>> will
> > >> > > > >>>>>> manage
> > >> > > > >>>>>>>>>> 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.
> > >> > > > >>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>> Best,
> > >> > > > >>>>>>>>>>>>>>>>>>>> Bruno
> > >> > > > >>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> Thanks for getting back to me!
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> 2.
> > >> > > > >>>>>>>>>>>>>>>>>>>>> The fact that implementations can always
> > track
> > >> > > > >>>> estimated
> > >> > > > >>>>>>>> memory
> > >> > > > >>>>>>>>>>>>>>>> usage
> > >> > > > >>>>>>>>>>>>>>>>>> in
> > >> > > > >>>>>>>>>>>>>>>>>>>>> the wrapper is a good point. I can remove -1
> > >> as
> > >> > > > >> an
> > >> > > > >>>>>> option,
> > >> > > > >>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>> I'll
> > >> > > > >>>>>>>>>>>>>>>>>>>> clarify
> > >> > > > >>>>>>>>>>>>>>>>>>>>> the JavaDoc that 0 is not just for
> > >> > > > >> non-transactional
> > >> > > > >>>>>> stores,
> > >> > > > >>>>>>>>>>>>>>>> which is
> > >> > > > >>>>>>>>>>>>>>>>>>>>> currently misleading.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> 6.
> > >> > > > >>>>>>>>>>>>>>>>>>>>> The problem with catching the exception in
> > the
> > >> > > > >>>> downgrade
> > >> > > > >>>>>>>>>> process
> > >> > > > >>>>>>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>>>>> would require new code in the Kafka version
> > >> being
> > >> > > > >>>>>> downgraded
> > >> > > > >>>>>>>>>> to.
> > >> > > > >>>>>>>>>>>>>>>> Since
> > >> > > > >>>>>>>>>>>>>>>>>>>>> users could conceivably downgrade to almost
> > >> *any*
> > >> > > > >>>> older
> > >> > > > >>>>>>>> version
> > >> > > > >>>>>>>>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>>> Kafka
> > >> > > > >>>>>>>>>>>>>>>>>>>>> Streams, I'm not sure how we could add that
> > >> code?
> > >> > > > >>>>>>>>>>>>>>>>>>>>> The only way I can think of doing it would
> > be
> > >> to
> > >> > > > >>>> provide
> > >> > > > >>>>>> a
> > >> > > > >>>>>>>>>>>>>>>> dedicated
> > >> > > > >>>>>>>>>>>>>>>>>>>>> downgrade tool, that goes through every
> > local
> > >> > > > >> store
> > >> > > > >>>> and
> > >> > > > >>>>>>>> removes
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>> offsets column families. But that seems like
> > >> an
> > >> > > > >>>>>> unnecessary
> > >> > > > >>>>>>>>>>>>>>>> amount of
> > >> > > > >>>>>>>>>>>>>>>>>>>> extra
> > >> > > > >>>>>>>>>>>>>>>>>>>>> code to maintain just to handle a somewhat
> > >> niche
> > >> > > > >>>>>> situation,
> > >> > > > >>>>>>>>>> when
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>> alternative (automatically wipe and restore
> > >> > > > >> stores)
> > >> > > > >>>>>> should be
> > >> > > > >>>>>>>>>>>>>>>>>>>> acceptable.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes
> > you've
> > >> > > > >>>> requested.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> 3a.
> > >> > > > >>>>>>>>>>>>>>>>>>>>> I agree that IsolationLevel makes more sense
> > >> at
> > >> > > > >>>>>> query-time,
> > >> > > > >>>>>>>> and
> > >> > > > >>>>>>>>>>>> I
> > >> > > > >>>>>>>>>>>>>>>>>>>> actually
> > >> > > > >>>>>>>>>>>>>>>>>>>>> initially attempted to place the
> > >> IsolationLevel
> > >> > > > >> at
> > >> > > > >>>>>>>> query-time,
> > >> > > > >>>>>>>>>>>>>>>> but I
> > >> > > > >>>>>>>>>>>>>>>>>> ran
> > >> > > > >>>>>>>>>>>>>>>>>>>>> into some problems:
> > >> > > > >>>>>>>>>>>>>>>>>>>>> - The key issue is that, under ALOS we're
> > not
> > >> > > > >> staging
> > >> > > > >>>>>> writes
> > >> > > > >>>>>>>> in
> > >> > > > >>>>>>>>>>>>>>>>>>>>> transactions, so can't perform writes at the
> > >> > > > >>>>>> READ_COMMITTED
> > >> > > > >>>>>>>>>>>>>>>> isolation
> > >> > > > >>>>>>>>>>>>>>>>>>>>> level. However, this may be addressed if we
> > >> > > > >> decide to
> > >> > > > >>>>>>>> *always*
> > >> > > > >>>>>>>>>>>>>>>> use
> > >> > > > >>>>>>>>>>>>>>>>>>>>> transactions as discussed under 3b.
> > >> > > > >>>>>>>>>>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> > >> > > > >>>> implementations. I
> > >> > > > >>>>>>>>>> remember
> > >> > > > >>>>>>>>>>>>>>>>>> having
> > >> > > > >>>>>>>>>>>>>>>>>>>>> some difficulty understanding the IQv1
> > >> internals,
> > >> > > > >>>> which
> > >> > > > >>>>>> made
> > >> > > > >>>>>>>> it
> > >> > > > >>>>>>>>>>>>>>>>>>>> difficult
> > >> > > > >>>>>>>>>>>>>>>>>>>>> to determine what needed to be changed.
> > >> However,
> > >> > > > >> I
> > >> > > > >>>>>> *think*
> > >> > > > >>>>>>>> this
> > >> > > > >>>>>>>>>>>>>>>> can be
> > >> > > > >>>>>>>>>>>>>>>>>>>>> addressed for both implementations by
> > wrapping
> > >> > > > >> the
> > >> > > > >>>>>>>> RocksDBStore
> > >> > > > >>>>>>>>>>>>>>>> in an
> > >> > > > >>>>>>>>>>>>>>>>>>>>> IsolationLevel-dependent wrapper, that
> > >> overrides
> > >> > > > >> read
> > >> > > > >>>>>> methods
> > >> > > > >>>>>>>>>>>>>>>> (get,
> > >> > > > >>>>>>>>>>>>>>>>>>>> etc.)
> > >> > > > >>>>>>>>>>>>>>>>>>>>> to either read directly from the database or
> > >> > > > >> from the
> > >> > > > >>>>>> ongoing
> > >> > > > >>>>>>>>>>>>>>>>>>>> transaction.
> > >> > > > >>>>>>>>>>>>>>>>>>>>> But IQv1 might still be difficult.
> > >> > > > >>>>>>>>>>>>>>>>>>>>> - If IsolationLevel becomes a query
> > >> constraint,
> > >> > > > >> then
> > >> > > > >>>> all
> > >> > > > >>>>>>>> other
> > >> > > > >>>>>>>>>>>>>>>>>>>> StateStores
> > >> > > > >>>>>>>>>>>>>>>>>>>>> will need to respect it, including the
> > >> in-memory
> > >> > > > >>>> stores.
> > >> > > > >>>>>> This
> > >> > > > >>>>>>>>>>>>>>>> would
> > >> > > > >>>>>>>>>>>>>>>>>>>> require
> > >> > > > >>>>>>>>>>>>>>>>>>>>> us to adapt in-memory stores to stage their
> > >> > > > >> writes so
> > >> > > > >>>>>> they
> > >> > > > >>>>>>>> can
> > >> > > > >>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>>>> isolated
> > >> > > > >>>>>>>>>>>>>>>>>>>>> from READ_COMMITTTED queries. It would also
> > >> > > > >> become an
> > >> > > > >>>>>>>> important
> > >> > > > >>>>>>>>>>>>>>>>>>>>> consideration for third-party stores on
> > >> upgrade,
> > >> > > > >> as
> > >> > > > >>>>>> without
> > >> > > > >>>>>>>>>>>>>>>> changes,
> > >> > > > >>>>>>>>>>>>>>>>>>>> they
> > >> > > > >>>>>>>>>>>>>>>>>>>>> would not support READ_COMMITTED queries
> > >> > > > >> correctly.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> Ultimately, I may need some help making the
> > >> > > > >> necessary
> > >> > > > >>>>>> change
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> IQv1
> > >> > > > >>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>> support this, but I don't think it's
> > >> > > > >> fundamentally
> > >> > > > >>>>>>>> impossible,
> > >> > > > >>>>>>>>>>>>>>>> if we
> > >> > > > >>>>>>>>>>>>>>>>>>>> want
> > >> > > > >>>>>>>>>>>>>>>>>>>>> to pursue this route.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> 3b.
> > >> > > > >>>>>>>>>>>>>>>>>>>>> The main reason I chose to keep ALOS
> > >> > > > >> un-transactional
> > >> > > > >>>>>> was to
> > >> > > > >>>>>>>>>>>>>>>> minimize
> > >> > > > >>>>>>>>>>>>>>>>>>>>> behavioural change for most users (I believe
> > >> most
> > >> > > > >>>> Streams
> > >> > > > >>>>>>>> users
> > >> > > > >>>>>>>>>>>>>>>> use
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>> default configuration, which is ALOS). That
> > >> said,
> > >> > > > >>>> it's
> > >> > > > >>>>>> clear
> > >> > > > >>>>>>>>>>>>>>>> that if
> > >> > > > >>>>>>>>>>>>>>>>>>>> ALOS
> > >> > > > >>>>>>>>>>>>>>>>>>>>> also used transactional stores, the only
> > >> change
> > >> > > > >> in
> > >> > > > >>>>>> behaviour
> > >> > > > >>>>>>>>>>>>>>>> would be
> > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>>>>> it would become *more correct*, which could
> > be
> > >> > > > >>>>>> considered a
> > >> > > > >>>>>>>>>> "bug
> > >> > > > >>>>>>>>>>>>>>>> fix"
> > >> > > > >>>>>>>>>>>>>>>>>> by
> > >> > > > >>>>>>>>>>>>>>>>>>>>> users, rather than a change they need to
> > >> handle.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> I believe that performance using
> > transactions
> > >> > > > >> (aka.
> > >> > > > >>>>>> RocksDB
> > >> > > > >>>>>>>>>>>>>>>>>>>> WriteBatches)
> > >> > > > >>>>>>>>>>>>>>>>>>>>> should actually be *better* than the
> > >> un-batched
> > >> > > > >>>>>> write-path
> > >> > > > >>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>>> currently used[1]. The only "performance"
> > >> > > > >>>> consideration
> > >> > > > >>>>>> will
> > >> > > > >>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>> increased memory usage that transactions
> > >> require.
> > >> > > > >>>> Given
> > >> > > > >>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>> mitigations
> > >> > > > >>>>>>>>>>>>>>>>>>>> for
> > >> > > > >>>>>>>>>>>>>>>>>>>>> this memory that we have in place, I would
> > >> expect
> > >> > > > >>>> that
> > >> > > > >>>>>> this
> > >> > > > >>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>> not a
> > >> > > > >>>>>>>>>>>>>>>>>>>>> problem for most users.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> If we're happy to do so, we can make ALOS
> > also
> > >> > > > >> use
> > >> > > > >>>>>>>>>> transactions.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > >> > > > >>>>>>>>>>>>>>>>>>>>> Nick
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> Link 1:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>
> > >> > > > >>>>
> > >> > > > >>
> > >> > >
> > >> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno
> > Cadonna <
> > >> > > > >>>>>>>>>>>> cadonna@apache.org
> > >> > > > >>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> Thanks for the updates and sorry for the
> > >> delay
> > >> > > > >> on my
> > >> > > > >>>>>> side!
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> 1.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> Making the default implementation for
> > >> flush() a
> > >> > > > >>>> no-op
> > >> > > > >>>>>> sounds
> > >> > > > >>>>>>>>>>>>>>>> good to
> > >> > > > >>>>>>>>>>>>>>>>>>>> me.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> 2.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> I think what was bugging me here is that a
> > >> > > > >>>> third-party
> > >> > > > >>>>>> state
> > >> > > > >>>>>>>>>>>>>>>> store
> > >> > > > >>>>>>>>>>>>>>>>>>>> needs
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> to implement the state store interface.
> > That
> > >> > > > >> means
> > >> > > > >>>> they
> > >> > > > >>>>>> need
> > >> > > > >>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> implement a wrapper around the actual state
> > >> > > > >> store
> > >> > > > >>>> as we
> > >> > > > >>>>>> do
> > >> > > > >>>>>>>> for
> > >> > > > >>>>>>>>>>>>>>>>>> RocksDB
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> with RocksDBStore. So, a third-party state
> > >> > > > >> store can
> > >> > > > >>>>>> always
> > >> > > > >>>>>>>>>>>>>>>> estimate
> > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> uncommitted bytes, if it wants, because the
> > >> > > > >> wrapper
> > >> > > > >>>> can
> > >> > > > >>>>>>>> record
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>> added
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> bytes.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> One case I can think of where returning -1
> > >> makes
> > >> > > > >>>> sense
> > >> > > > >>>>>> is
> > >> > > > >>>>>>>> when
> > >> > > > >>>>>>>>>>>>>>>>>> Streams
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> does not need to estimate the size of the
> > >> write
> > >> > > > >>>> batch
> > >> > > > >>>>>> and
> > >> > > > >>>>>>>>>>>>>>>> trigger
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> extraordinary commits, because the
> > >> third-party
> > >> > > > >> state
> > >> > > > >>>>>> store
> > >> > > > >>>>>>>>>>>>>>>> takes care
> > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> memory. But in that case the method could
> > >> also
> > >> > > > >> just
> > >> > > > >>>>>> return
> > >> > > > >>>>>>>> 0.
> > >> > > > >>>>>>>>>>>>>>>> Even
> > >> > > > >>>>>>>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> case would be better solved with a method
> > >> that
> > >> > > > >>>> returns
> > >> > > > >>>>>>>> whether
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> store manages itself the memory used for
> > >> > > > >> uncommitted
> > >> > > > >>>>>> bytes
> > >> > > > >>>>>>>> or
> > >> > > > >>>>>>>>>>>>>>>> not.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> Said that, I am fine with keeping the -1
> > >> return
> > >> > > > >>>> value,
> > >> > > > >>>>>> I was
> > >> > > > >>>>>>>>>>>>>>>> just
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> wondering when and if it will be used.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> Regarding returning 0 for transactional
> > state
> > >> > > > >> stores
> > >> > > > >>>>>> when
> > >> > > > >>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>> batch
> > >> > > > >>>>>>>>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> empty, I was just wondering because you
> > >> > > > >> explicitly
> > >> > > > >>>>>> stated
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> "or {@code 0} if this StateStore does not
> > >> > > > >> support
> > >> > > > >>>>>>>>>>>> transactions."
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> So it seemed to me returning 0 could only
> > >> > > > >> happen for
> > >> > > > >>>>>>>>>>>>>>>>>> non-transactional
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> state stores.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> a) What do you think if we move the
> > isolation
> > >> > > > >> level
> > >> > > > >>>> to
> > >> > > > >>>>>> IQ
> > >> > > > >>>>>>>> (v1
> > >> > > > >>>>>>>>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>> v2)?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> In the end this is the only component that
> > >> > > > >> really
> > >> > > > >>>> needs
> > >> > > > >>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> specify
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> isolation level. It is similar to the Kafka
> > >> > > > >> consumer
> > >> > > > >>>>>> that
> > >> > > > >>>>>>>> can
> > >> > > > >>>>>>>>>>>>>>>> choose
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> with what isolation level to read the input
> > >> > > > >> topic.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> For IQv1 the isolation level should go into
> > >> > > > >>>>>>>>>>>>>>>> StoreQueryParameters. For
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> IQv2, I would add it to the Query
> > interface.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> b) Point a) raises the question what should
> > >> > > > >> happen
> > >> > > > >>>>>> during
> > >> > > > >>>>>>>>>>>>>>>>>> at-least-once
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing when the state store does not
> > use
> > >> > > > >>>>>> transactions?
> > >> > > > >>>>>>>>>> John
> > >> > > > >>>>>>>>>>>>>>>> in
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> past proposed to also use transactions on
> > >> state
> > >> > > > >>>> stores
> > >> > > > >>>>>> for
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> at-least-once. I like that idea, because it
> > >> > > > >> avoids
> > >> > > > >>>>>>>> aggregating
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>> same
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> records over and over again in the case of
> > a
> > >> > > > >>>> failure. We
> > >> > > > >>>>>>>> had a
> > >> > > > >>>>>>>>>>>>>>>> case
> > >> > > > >>>>>>>>>>>>>>>>>> in
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> the past where a Streams applications in
> > >> > > > >>>> at-least-once
> > >> > > > >>>>>> mode
> > >> > > > >>>>>>>>>> was
> > >> > > > >>>>>>>>>>>>>>>>>> failing
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> continuously for some reasons I do not
> > >> remember
> > >> > > > >>>> before
> > >> > > > >>>>>>>>>>>>>>>> committing the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> offsets. After each failover, the app
> > >> aggregated
> > >> > > > >>>> again
> > >> > > > >>>>>> and
> > >> > > > >>>>>>>>>>>>>>>> again the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> same records. Of course the aggregate
> > >> increased
> > >> > > > >> to
> > >> > > > >>>> very
> > >> > > > >>>>>>>> wrong
> > >> > > > >>>>>>>>>>>>>>>> values
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> just because of the failover. With
> > >> transactions
> > >> > > > >> on
> > >> > > > >>>> the
> > >> > > > >>>>>> state
> > >> > > > >>>>>>>>>>>>>>>> stores
> > >> > > > >>>>>>>>>>>>>>>>>> we
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> could have avoided this. The app would have
> > >> > > > >> output
> > >> > > > >>>> the
> > >> > > > >>>>>> same
> > >> > > > >>>>>>>>>>>>>>>> aggregate
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> multiple times (i.e., after each failover)
> > >> but
> > >> > > > >> at
> > >> > > > >>>> least
> > >> > > > >>>>>> the
> > >> > > > >>>>>>>>>>>>>>>> value of
> > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> aggregate would not depend on the number of
> > >> > > > >>>> failovers.
> > >> > > > >>>>>>>>>>>>>>>> Outputting the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> same aggregate multiple times would be
> > >> incorrect
> > >> > > > >>>> under
> > >> > > > >>>>>>>>>>>>>>>> exactly-once
> > >> > > > >>>>>>>>>>>>>>>>>> but
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> it is OK for at-least-once.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> If it makes sense to add a config to turn
> > on
> > >> > > > >> and off
> > >> > > > >>>>>>>>>>>>>>>> transactions on
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> state stores under at-least-once or just
> > use
> > >> > > > >>>>>> transactions in
> > >> > > > >>>>>>>>>>>>>>>> any case
> > >> > > > >>>>>>>>>>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> a question we should also discuss in this
> > >> KIP.
> > >> > > > >> It
> > >> > > > >>>>>> depends a
> > >> > > > >>>>>>>>>> bit
> > >> > > > >>>>>>>>>>>>>>>> on
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I
> > >> would
> > >> > > > >>>> add a
> > >> > > > >>>>>>>> config.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> 4.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> Your points are all valid. I tend to say to
> > >> > > > >> keep the
> > >> > > > >>>>>> metrics
> > >> > > > >>>>>>>>>>>>>>>> around
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() until we remove flush() completely
> > >> from
> > >> > > > >> the
> > >> > > > >>>>>>>> interface.
> > >> > > > >>>>>>>>>>>>>>>> Calls
> > >> > > > >>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() might still exist since existing
> > >> > > > >> processors
> > >> > > > >>>>>> might
> > >> > > > >>>>>>>>>> still
> > >> > > > >>>>>>>>>>>>>>>> call
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() explicitly as you mentioned in 1).
> > >> For
> > >> > > > >>>> sure, we
> > >> > > > >>>>>> need
> > >> > > > >>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>> document
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> how the metrics change due to the
> > >> transactions
> > >> > > > >> in
> > >> > > > >>>> the
> > >> > > > >>>>>>>> upgrade
> > >> > > > >>>>>>>>>>>>>>>> notes.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> 5.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> I see. Then you should describe how the
> > >> > > > >> .position
> > >> > > > >>>> files
> > >> > > > >>>>>> are
> > >> > > > >>>>>>>>>>>>>>>> handled
> > >> > > > >>>>>>>>>>>>>>>>>> in
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> a dedicated section of the KIP or
> > incorporate
> > >> > > > >> the
> > >> > > > >>>>>>>> description
> > >> > > > >>>>>>>>>>>>>>>> in the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> "Atomic Checkpointing" section instead of
> > >> only
> > >> > > > >>>>>> mentioning it
> > >> > > > >>>>>>>>>> in
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> "Compatibility, 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:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>
> > >> > > > >>>>
> > >> > > > >>
> > >> > >
> > >> >
> > >>
> > https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> It is a bit brittle, but it works.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> Best,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> Bruno
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for taking the time to review the
> > >> KIP.
> > >> > > > >> I'm
> > >> > > > >>>> back
> > >> > > > >>>>>> from
> > >> > > > >>>>>>>>>>>>>>>> leave
> > >> > > > >>>>>>>>>>>>>>>>>> now
> > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> intend to move this forwards as quickly
> > as I
> > >> > > > >> can.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Addressing your points:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Because flush() is part of the StateStore
> > >> API,
> > >> > > > >> it's
> > >> > > > >>>>>> exposed
> > >> > > > >>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>> custom
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors, which might be making calls to
> > >> > > > >> flush().
> > >> > > > >>>>>> This
> > >> > > > >>>>>>>> was
> > >> > > > >>>>>>>>>>>>>>>>>> actually
> > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> case in a few integration tests.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> To maintain as much compatibility as
> > >> possible,
> > >> > > > >> I'd
> > >> > > > >>>>>> prefer
> > >> > > > >>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>> make
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> this
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> an UnsupportedOperationException, as it
> > will
> > >> > > > >> cause
> > >> > > > >>>>>>>> previously
> > >> > > > >>>>>>>>>>>>>>>>>> working
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors to start throwing exceptions at
> > >> > > > >> runtime.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that it doesn't make sense for it
> > to
> > >> > > > >> proxy
> > >> > > > >>>>>>>> commit(),
> > >> > > > >>>>>>>>>>>>>>>> though,
> > >> > > > >>>>>>>>>>>>>>>>>>>> as
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> that would cause it to violate the
> > >> "StateStores
> > >> > > > >>>> commit
> > >> > > > >>>>>> only
> > >> > > > >>>>>>>>>>>>>>>> when the
> > >> > > > >>>>>>>>>>>>>>>>>>>> Task
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commits" rule.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Instead, I think we should make this a
> > >> no-op.
> > >> > > > >> That
> > >> > > > >>>> way,
> > >> > > > >>>>>>>>>>>>>>>> existing
> > >> > > > >>>>>>>>>>>>>>>>>> user
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors will continue to work
> > as-before,
> > >> > > > >> without
> > >> > > > >>>>>>>> violation
> > >> > > > >>>>>>>>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>>> store
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> consistency that would be caused by
> > >> premature
> > >> > > > >>>>>> flush/commit
> > >> > > > >>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>>>>> StateStore
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> data to disk.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> What do you think?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> As stated in the JavaDoc, when a
> > StateStore
> > >> > > > >>>>>> implementation
> > >> > > > >>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, but is unable to estimate
> > the
> > >> > > > >>>>>> uncommitted
> > >> > > > >>>>>>>>>>>> memory
> > >> > > > >>>>>>>>>>>>>>>>>> usage,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> method will return -1.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> The intention here is to permit
> > third-party
> > >> > > > >>>>>> implementations
> > >> > > > >>>>>>>>>>>>>>>> that may
> > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> able to estimate memory usage.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been
> > >> > > > >> written to
> > >> > > > >>>> the
> > >> > > > >>>>>>>> store
> > >> > > > >>>>>>>>>>>>>>>> yet. I
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> thought
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> that was implied by "This method will
> > >> return an
> > >> > > > >>>>>>>> approximation
> > >> > > > >>>>>>>>>>>>>>>> of the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> memory
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> would be freed by the next call to {@link
> > >> > > > >>>>>> #commit(Map)}"
> > >> > > > >>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>> "@return
> > >> > > > >>>>>>>>>>>>>>>>>>>> The
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> approximate size of all records awaiting
> > >> {@link
> > >> > > > >>>>>>>>>>>> #commit(Map)}",
> > >> > > > >>>>>>>>>>>>>>>>>>>> however,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> I
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if
> > you
> > >> > > > >> think
> > >> > > > >>>> this
> > >> > > > >>>>>> is
> > >> > > > >>>>>>>>>>>>>>>> unclear?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I realise this is probably the most
> > >> contentious
> > >> > > > >>>> point
> > >> > > > >>>>>> in my
> > >> > > > >>>>>>>>>>>>>>>> design,
> > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> I'm
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> open to changing it if I'm unable to
> > >> convince
> > >> > > > >> you
> > >> > > > >>>> of
> > >> > > > >>>>>> the
> > >> > > > >>>>>>>>>>>>>>>> benefits.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are
> > >> directly
> > >> > > > >>>> provided
> > >> > > > >>>>>>>>>>>>>>>> StateStores
> > >> > > > >>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> query, and it may be important for users
> > to
> > >> > > > >>>>>>>> programmatically
> > >> > > > >>>>>>>>>>>>>>>> know
> > >> > > > >>>>>>>>>>>>>>>>>>>> which
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> mode the StateStore is operating under. If
> > >> we
> > >> > > > >>>> simply
> > >> > > > >>>>>>>> provide
> > >> > > > >>>>>>>>>>>> an
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout
> > the
> > >> > > > >>>> internal
> > >> > > > >>>>>>>> streams
> > >> > > > >>>>>>>>>>>>>>>>>>>> engine), or
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> similar, then users will need to
> > understand
> > >> the
> > >> > > > >>>>>> operation
> > >> > > > >>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> consequences
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> of each available processing mode and how
> > it
> > >> > > > >>>> pertains
> > >> > > > >>>>>> to
> > >> > > > >>>>>>>>>> their
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Interactive Query users aren't the only
> > >> people
> > >> > > > >> that
> > >> > > > >>>>>> care
> > >> > > > >>>>>>>>>> about
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> processing.mode/IsolationLevel of a
> > >> StateStore:
> > >> > > > >>>>>>>> implementers
> > >> > > > >>>>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>>> custom
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStores also need to understand the
> > >> > > > >> behaviour
> > >> > > > >>>>>> expected
> > >> > > > >>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>> their
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> implementation. KIP-892 introduces some
> > >> > > > >> assumptions
> > >> > > > >>>>>> into
> > >> > > > >>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>> Streams
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> Engine
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> about how StateStores operate under each
> > >> > > > >> processing
> > >> > > > >>>>>> mode,
> > >> > > > >>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>> it's
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> important that custom implementations
> > >> adhere to
> > >> > > > >>>> those
> > >> > > > >>>>>>>>>>>>>>>> assumptions in
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> order
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> to maintain the consistency guarantees.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> IsolationLevels provide a high-level
> > >> contract
> > >> > > > >> on
> > >> > > > >>>> the
> > >> > > > >>>>>>>>>> behaviour
> > >> > > > >>>>>>>>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStore: a user knows that under
> > >> > > > >> READ_COMMITTED,
> > >> > > > >>>>>> they
> > >> > > > >>>>>>>> will
> > >> > > > >>>>>>>>>>>>>>>> see
> > >> > > > >>>>>>>>>>>>>>>>>>>> writes
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> only after the Task has committed, and
> > under
> > >> > > > >>>>>>>> READ_UNCOMMITTED
> > >> > > > >>>>>>>>>>>>>>>> they
> > >> > > > >>>>>>>>>>>>>>>>>>>> will
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> see
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> writes immediately. No understanding of
> > the
> > >> > > > >>>> details of
> > >> > > > >>>>>> each
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing.mode
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> is required, either for IQ users or
> > >> StateStore
> > >> > > > >>>>>>>> implementers.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> An argument can be made that these
> > >> contractual
> > >> > > > >>>>>> guarantees
> > >> > > > >>>>>>>> can
> > >> > > > >>>>>>>>>>>>>>>> simply
> > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> documented for the processing.mode (i.e.
> > >> that
> > >> > > > >>>>>> exactly-once
> > >> > > > >>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED
> > >> and
> > >> > > > >>>>>>>> at-least-once
> > >> > > > >>>>>>>>>>>>>>>> behaves
> > >> > > > >>>>>>>>>>>>>>>>>>>> like
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several
> > >> small
> > >> > > > >>>> issues
> > >> > > > >>>>>> with
> > >> > > > >>>>>>>>>>>>>>>> this I'd
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> prefer
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> to avoid:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - Where would we document these
> > >> > > > >> contracts,
> > >> > > > >>>> in
> > >> > > > >>>>>> a way
> > >> > > > >>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>> difficult
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          for users/implementers to
> > >> miss/ignore?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It's not clear to users that
> > the
> > >> > > > >>>> processing
> > >> > > > >>>>>> mode
> > >> > > > >>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>> communicating
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          an expectation of read isolation,
> > >> > > > >> unless
> > >> > > > >>>> they
> > >> > > > >>>>>> read
> > >> > > > >>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> documentation. Users
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          rarely consult documentation
> > unless
> > >> > > > >> they
> > >> > > > >>>> feel
> > >> > > > >>>>>> they
> > >> > > > >>>>>>>>>> need
> > >> > > > >>>>>>>>>>>>>>>> to, so
> > >> > > > >>>>>>>>>>>>>>>>>>>> it's
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> likely
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          this detail would get missed by
> > >> many
> > >> > > > >> users.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It tightly couples processing
> > >> modes
> > >> > > > >> to
> > >> > > > >>>> read
> > >> > > > >>>>>>>>>> isolation.
> > >> > > > >>>>>>>>>>>>>>>> Adding
> > >> > > > >>>>>>>>>>>>>>>>>>>> new
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          processing modes, or changing the
> > >> read
> > >> > > > >>>>>> isolation of
> > >> > > > >>>>>>>>>>>>>>>> existing
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> processing
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          modes would be
> > >> difficult/impossible.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Ultimately, the cost of introducing
> > >> > > > >>>> IsolationLevels is
> > >> > > > >>>>>>>> just a
> > >> > > > >>>>>>>>>>>>>>>> single
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> method, since we re-use the existing
> > >> > > > >> IsolationLevel
> > >> > > > >>>>>> enum
> > >> > > > >>>>>>>> from
> > >> > > > >>>>>>>>>>>>>>>> Kafka.
> > >> > > > >>>>>>>>>>>>>>>>>>>> This
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> gives us a clear place to document the
> > >> > > > >> contractual
> > >> > > > >>>>>>>> guarantees
> > >> > > > >>>>>>>>>>>>>>>>>> expected
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> of/provided by StateStores, that is
> > >> accessible
> > >> > > > >>>> both by
> > >> > > > >>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>> StateStore
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> itself, and by IQ users.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> (Writing this I've just realised that the
> > >> > > > >>>> StateStore
> > >> > > > >>>>>> and IQ
> > >> > > > >>>>>>>>>>>>>>>> APIs
> > >> > > > >>>>>>>>>>>>>>>>>>>> actually
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> don't provide access to StateStoreContext
> > >> that
> > >> > > > >> IQ
> > >> > > > >>>> users
> > >> > > > >>>>>>>> would
> > >> > > > >>>>>>>>>>>>>>>> have
> > >> > > > >>>>>>>>>>>>>>>>>>>> direct
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> access to... Perhaps StateStore should
> > >> expose
> > >> > > > >>>>>>>>>> isolationLevel()
> > >> > > > >>>>>>>>>>>>>>>>>> itself
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> too?)
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 4.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the
> > >> metrics
> > >> > > > >>>> in-place
> > >> > > > >>>>>>>>>>>>>>>> either, as
> > >> > > > >>>>>>>>>>>>>>>>>>>> it's a
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> backwards incompatible change. My concern
> > is
> > >> > > > >> that,
> > >> > > > >>>> if
> > >> > > > >>>>>> we
> > >> > > > >>>>>>>>>> leave
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> existing
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics in place, they will be
> > >> > > > >> confusing to
> > >> > > > >>>>>> users.
> > >> > > > >>>>>>>>>>>>>>>> Right
> > >> > > > >>>>>>>>>>>>>>>>>> now,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics record explicit flushes to
> > >> > > > >> disk,
> > >> > > > >>>> but
> > >> > > > >>>>>> under
> > >> > > > >>>>>>>>>>>>>>>> KIP-892,
> > >> > > > >>>>>>>>>>>>>>>>>>>> even
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> a
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() will not explicitly flush data to
> > >> > > > >> disk -
> > >> > > > >>>>>> RocksDB
> > >> > > > >>>>>>>>>> will
> > >> > > > >>>>>>>>>>>>>>>>>> decide
> > >> > > > >>>>>>>>>>>>>>>>>>>> on
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> when to flush memtables to disk itself.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> If we keep the existing "flush" metrics,
> > >> we'd
> > >> > > > >> have
> > >> > > > >>>> two
> > >> > > > >>>>>>>>>>>> options,
> > >> > > > >>>>>>>>>>>>>>>>>> which
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> both
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> seem pretty bad to me:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          1. Have them record calls to
> > >> commit(),
> > >> > > > >>>> which
> > >> > > > >>>>>> would
> > >> > > > >>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>>>> misleading, as
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          data is no longer explicitly
> > >> "flushed"
> > >> > > > >> to
> > >> > > > >>>> disk
> > >> > > > >>>>>> by
> > >> > > > >>>>>>>> this
> > >> > > > >>>>>>>>>>>>>>>> call.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          2. Have them record nothing at
> > all,
> > >> > > > >> which
> > >> > > > >>>> is
> > >> > > > >>>>>>>>>> equivalent
> > >> > > > >>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>> removing
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          metrics, except that users will
> > see
> > >> > the
> > >> > > > >>>> metric
> > >> > > > >>>>>>>> still
> > >> > > > >>>>>>>>>>>>>>>> exists and
> > >> > > > >>>>>>>>>>>>>>>>>>>> so
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> assume
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          that the metric is correct, and
> > >> that
> > >> > > > >>>> there's a
> > >> > > > >>>>>>>> problem
> > >> > > > >>>>>>>>>>>>>>>> with
> > >> > > > >>>>>>>>>>>>>>>>>> their
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> system
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>          when there isn't.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that removing them is also a bad
> > >> > > > >> solution,
> > >> > > > >>>> and
> > >> > > > >>>>>> I'd
> > >> > > > >>>>>>>>>>>>>>>> like some
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> guidance on the best path forward here.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 5.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Position files are updated on every write
> > >> to a
> > >> > > > >>>>>> StateStore.
> > >> > > > >>>>>>>>>>>>>>>> Since our
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> writes
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> are now buffered until commit(), we can't
> > >> > > > >> update
> > >> > > > >>>> the
> > >> > > > >>>>>>>> Position
> > >> > > > >>>>>>>>>>>>>>>> file
> > >> > > > >>>>>>>>>>>>>>>>>>>> until
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() has been called, otherwise it
> > >> would be
> > >> > > > >>>>>>>> inconsistent
> > >> > > > >>>>>>>>>>>>>>>> with
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> data
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> in the event of a rollback. Consequently,
> > we
> > >> > > > >> need
> > >> > > > >>>> to
> > >> > > > >>>>>> manage
> > >> > > > >>>>>>>>>>>>>>>> these
> > >> > > > >>>>>>>>>>>>>>>>>>>> offsets
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> the same way we manage the checkpoint
> > >> offsets,
> > >> > > > >> and
> > >> > > > >>>>>> ensure
> > >> > > > >>>>>>>>>>>>>>>> they're
> > >> > > > >>>>>>>>>>>>>>>>>> only
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> written on commit().
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> 6.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet
> > >> what
> > >> > > > >>>> tests to
> > >> > > > >>>>>>>>>> write.
> > >> > > > >>>>>>>>>>>>>>>> How
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> explicit
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> do we need to be here in the KIP?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is
> > >> designed
> > >> > > > >> to be
> > >> > > > >>>>>>>> seamless,
> > >> > > > >>>>>>>>>>>>>>>> and we
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> should
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> definitely add some tests around that.
> > >> > > > >> Downgrade,
> > >> > > > >>>> it
> > >> > > > >>>>>>>>>>>>>>>> transpires,
> > >> > > > >>>>>>>>>>>>>>>>>> isn't
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> currently possible, as the extra column
> > >> family
> > >> > > > >> for
> > >> > > > >>>>>> offset
> > >> > > > >>>>>>>>>>>>>>>> storage is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> incompatible with the pre-KIP-892
> > >> > > > >> implementation:
> > >> > > > >>>> when
> > >> > > > >>>>>> you
> > >> > > > >>>>>>>>>>>>>>>> open a
> > >> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> database, you must open all available
> > column
> > >> > > > >>>> families
> > >> > > > >>>>>> or
> > >> > > > >>>>>>>>>>>>>>>> receive an
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> error.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> What currently happens on downgrade is
> > that
> > >> it
> > >> > > > >>>>>> attempts to
> > >> > > > >>>>>>>>>>>>>>>> open the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> store,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> throws an error about the offsets column
> > >> > > > >> family not
> > >> > > > >>>>>> being
> > >> > > > >>>>>>>>>>>>>>>> opened,
> > >> > > > >>>>>>>>>>>>>>>>>>>> which
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task.
> > >> Given
> > >> > > > >> that
> > >> > > > >>>>>>>>>> downgrades
> > >> > > > >>>>>>>>>>>>>>>>>> should
> > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> uncommon, I think this is acceptable
> > >> > > > >> behaviour, as
> > >> > > > >>>> the
> > >> > > > >>>>>>>>>>>>>>>> end-state is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> consistent, even if it results in an
> > >> > > > >> undesirable
> > >> > > > >>>> state
> > >> > > > >>>>>>>>>>>> restore.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Should I document the upgrade/downgrade
> > >> > > > >> behaviour
> > >> > > > >>>>>>>> explicitly
> > >> > > > >>>>>>>>>>>>>>>> in the
> > >> > > > >>>>>>>>>>>>>>>>>>>> KIP?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> --
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> Nick
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno
> > Cadonna
> > >> <
> > >> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > >> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Nick!
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates!
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > StateStore#commit(Collections.emptyMap())?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Since calls to flush() will not exist
> > >> anymore
> > >> > > > >>>> after
> > >> > > > >>>>>> this
> > >> > > > >>>>>>>> KIP
> > >> > > > >>>>>>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> released, I would rather throw an
> > >> unsupported
> > >> > > > >>>>>> operation
> > >> > > > >>>>>>>>>>>>>>>> exception
> > >> > > > >>>>>>>>>>>>>>>>>> by
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> default.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> When would a state store return -1 from
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > StateStore#approximateNumUncommittedBytes()
> > >> > > > >> while
> > >> > > > >>>>>> being
> > >> > > > >>>>>>>>>>>>>>>>>>>> transactional?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Wouldn't
> > >> > > > >>>> StateStore#approximateNumUncommittedBytes()
> > >> > > > >>>>>> also
> > >> > > > >>>>>>>>>>>>>>>> return 0
> > >> > > > >>>>>>>>>>>>>>>>>> if
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the state store is transactional but
> > >> nothing
> > >> > > > >> has
> > >> > > > >>>> been
> > >> > > > >>>>>>>>>> written
> > >> > > > >>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> state store yet?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Sorry for bringing this up again. Does
> > this
> > >> > > > >> KIP
> > >> > > > >>>> really
> > >> > > > >>>>>>>> need
> > >> > > > >>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>> introduce
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> > >> > > > >>>> StateStoreContext
> > >> > > > >>>>>> has
> > >> > > > >>>>>>>>>>>>>>>> already
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> appConfigs() which basically exposes the
> > >> same
> > >> > > > >>>>>> information,
> > >> > > > >>>>>>>>>>>>>>>> i.e., if
> > >> > > > >>>>>>>>>>>>>>>>>>>> EOS
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> is enabled or not.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> In one of your previous e-mails you
> > wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> "My idea was to try to keep the
> > StateStore
> > >> > > > >>>> interface
> > >> > > > >>>>>> as
> > >> > > > >>>>>>>>>>>>>>>> loosely
> > >> > > > >>>>>>>>>>>>>>>>>>>> coupled
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to
> > >> give
> > >> > > > >>>>>> implementers
> > >> > > > >>>>>>>>>>>> more
> > >> > > > >>>>>>>>>>>>>>>>>>>> freedom,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and reduce the amount of internal
> > knowledge
> > >> > > > >>>> required."
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> While I understand the intent, I doubt
> > >> that it
> > >> > > > >>>>>> decreases
> > >> > > > >>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>> coupling of
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> a StateStore interface and the Streams
> > >> engine.
> > >> > > > >>>>>>>>>> READ_COMMITTED
> > >> > > > >>>>>>>>>>>>>>>> only
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> applies to IQ but not to reads by
> > >> processors.
> > >> > > > >>>> Thus,
> > >> > > > >>>>>>>>>>>>>>>> implementers
> > >> > > > >>>>>>>>>>>>>>>>>>>> need to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand how Streams accesses the state
> > >> > > > >> stores.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> I would like to hear what others think
> > >> about
> > >> > > > >> this.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 4.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Great exposing new metrics for
> > >> transactional
> > >> > > > >> state
> > >> > > > >>>>>> stores!
> > >> > > > >>>>>>>>>>>>>>>>>> However, I
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> would prefer to add new metrics and
> > >> deprecate
> > >> > > > >> (in
> > >> > > > >>>> the
> > >> > > > >>>>>>>> docs)
> > >> > > > >>>>>>>>>>>>>>>> the 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 also need to adapt/add
> > >> > > > >> integration
> > >> > > > >>>>>> tests
> > >> > > > >>>>>>>>>>>>>>>> besides
> > >> > > > >>>>>>>>>>>>>>>>>> unit
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> tests. Additionally, we probably need
> > >> > > > >> integration
> > >> > > > >>>> or
> > >> > > > >>>>>>>> system
> > >> > > > >>>>>>>>>>>>>>>> tests
> > >> > > > >>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> verify that upgrades and downgrades
> > between
> > >> > > > >>>>>> transactional
> > >> > > > >>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> non-transactional state stores work as
> > >> > > > >> expected.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Best,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> One more thing: I noted John's
> > suggestion
> > >> in
> > >> > > > >> the
> > >> > > > >>>> KIP,
> > >> > > > >>>>>>>> under
> > >> > > > >>>>>>>>>>>>>>>>>>>> "Rejected
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> Alternatives". I still think it's an
> > idea
> > >> > > > >> worth
> > >> > > > >>>>>> pursuing,
> > >> > > > >>>>>>>>>>>>>>>> but I
> > >> > > > >>>>>>>>>>>>>>>>>>>> believe
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> that it's out of the scope of this KIP,
> > >> > > > >> because
> > >> > > > >>>> it
> > >> > > > >>>>>>>> solves a
> > >> > > > >>>>>>>>>>>>>>>>>>>> different
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> set
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> of problems to this KIP, and the scope
> > of
> > >> > > > >> this
> > >> > > > >>>> one
> > >> > > > >>>>>> has
> > >> > > > >>>>>>>>>>>>>>>> already
> > >> > > > >>>>>>>>>>>>>>>>>> grown
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> quite
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> large!
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick
> > >> Telford <
> > >> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP (
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>
> > >> > > > >>>>
> > >> > > > >>
> > >> > >
> > >> >
> > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> )
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with the latest changes; mostly
> > bringing
> > >> > > > >> back
> > >> > > > >>>>>> "Atomic
> > >> > > > >>>>>>>>>>>>>>>>>>>> Checkpointing"
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> (for
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> what feels like the 10th time!). I
> > think
> > >> > > > >> the one
> > >> > > > >>>>>> thing
> > >> > > > >>>>>>>>>>>>>>>> missing is
> > >> > > > >>>>>>>>>>>>>>>>>>>> some
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> changes to metrics (notably the store
> > >> > > > >> "flush"
> > >> > > > >>>>>> metrics
> > >> > > > >>>>>>>> will
> > >> > > > >>>>>>>>>>>>>>>> need
> > >> > > > >>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> renamed to "commit").
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> The reason I brought back Atomic
> > >> > > > >> Checkpointing
> > >> > > > >>>> was
> > >> > > > >>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> decouple
> > >> > > > >>>>>>>>>>>>>>>>>>>> store
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flush
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from store commit. This is important,
> > >> > > > >> because
> > >> > > > >>>> with
> > >> > > > >>>>>>>>>>>>>>>> Transactional
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> StateStores, we now need to call
> > "flush"
> > >> on
> > >> > > > >>>> *every*
> > >> > > > >>>>>> Task
> > >> > > > >>>>>>>>>>>>>>>> commit,
> > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> just when the StateStore is closing,
> > >> > > > >> otherwise
> > >> > > > >>>> our
> > >> > > > >>>>>>>>>>>>>>>> transaction
> > >> > > > >>>>>>>>>>>>>>>>>>>> buffer
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> will
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> never be written and persisted, instead
> > >> > > > >> growing
> > >> > > > >>>>>>>> unbounded!
> > >> > > > >>>>>>>>>>>> I
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> experimented
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with some simple solutions, like
> > forcing
> > >> a
> > >> > > > >> store
> > >> > > > >>>>>> flush
> > >> > > > >>>>>>>>>>>>>>>> whenever
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed
> > >> its
> > >> > > > >>>>>> configured
> > >> > > > >>>>>>>>>>>>>>>> size, but
> > >> > > > >>>>>>>>>>>>>>>>>>>> this
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> brittle: it prevented the transaction
> > >> buffer
> > >> > > > >>>> from
> > >> > > > >>>>>> being
> > >> > > > >>>>>>>>>>>>>>>>>> configured
> > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> unbounded, and it still would have
> > >> required
> > >> > > > >>>> explicit
> > >> > > > >>>>>>>>>>>>>>>> flushes of
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> RocksDB,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yielding sub-optimal performance and
> > >> memory
> > >> > > > >>>>>> utilization.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the
> > >> > > > >> "right"
> > >> > > > >>>> way
> > >> > > > >>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> resolve
> > >> > > > >>>>>>>>>>>>>>>>>> this
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog
> > >> > > > >> offsets
> > >> > > > >>>> that
> > >> > > > >>>>>>>>>>>>>>>> correspond
> > >> > > > >>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> most
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> recently written records are always
> > >> > > > >> atomically
> > >> > > > >>>>>> written
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (by writing them to the same
> > transaction
> > >> > > > >>>> buffer),
> > >> > > > >>>>>> we can
> > >> > > > >>>>>>>>>>>>>>>> avoid
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> forcibly
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk,
> > >> > > > >> letting
> > >> > > > >>>>>> RocksDB
> > >> > > > >>>>>>>>>>>>>>>> flush
> > >> > > > >>>>>>>>>>>>>>>>>> them
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> only
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when necessary, without losing any of
> > our
> > >> > > > >>>>>> consistency
> > >> > > > >>>>>>>>>>>>>>>> guarantees.
> > >> > > > >>>>>>>>>>>>>>>>>>>> See
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> updated KIP for more info.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I have fully implemented these changes,
> > >> > > > >>>> although I'm
> > >> > > > >>>>>>>> still
> > >> > > > >>>>>>>>>>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> entirely
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> happy with the implementation for
> > >> segmented
> > >> > > > >>>>>> StateStores,
> > >> > > > >>>>>>>>>> so
> > >> > > > >>>>>>>>>>>>>>>> I
> > >> > > > >>>>>>>>>>>>>>>>>> plan
> > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests
> > >> > > > >> pass. If
> > >> > > > >>>>>> you'd
> > >> > > > >>>>>>>> like
> > >> > > > >>>>>>>>>>>>>>>> to try
> > >> > > > >>>>>>>>>>>>>>>>>>>> out
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> or
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> review this highly experimental and
> > >> > > > >> incomplete
> > >> > > > >>>>>> branch,
> > >> > > > >>>>>>>>>> it's
> > >> > > > >>>>>>>>>>>>>>>>>>>> available
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> here:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > >> > > > >>>>>>>> .
> > >> > > > >>>>>>>>>>>>>>>> Note:
> > >> > > > >>>>>>>>>>>>>>>>>>>> it's
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> built
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a
> > >> stable
> > >> > > > >> base
> > >> > > > >>>> to
> > >> > > > >>>>>> build
> > >> > > > >>>>>>>>>>>>>>>> and test
> > >> > > > >>>>>>>>>>>>>>>>>>>> it
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> on,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples
> > >> > > > >> comparisons
> > >> > > > >>>> in a
> > >> > > > >>>>>>>> live
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> environment. I
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> plan to rebase it against trunk once
> > it's
> > >> > > > >> nearer
> > >> > > > >>>>>>>>>> completion
> > >> > > > >>>>>>>>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>> has
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> been
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> proven on our main application.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I would really appreciate help in
> > >> reviewing
> > >> > > > >> and
> > >> > > > >>>>>> testing:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and
> > >> Window)
> > >> > > > >>>> stores
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Global stores
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> As I do not currently use either of
> > >> these,
> > >> > > > >> so my
> > >> > > > >>>>>> primary
> > >> > > > >>>>>>>>>>>>>>>> test
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> environment
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> doesn't test these areas.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting
> > next
> > >> > > > >> week
> > >> > > > >>>> for
> > >> > > > >>>>>> a few
> > >> > > > >>>>>>>>>>>>>>>> weeks,
> > >> > > > >>>>>>>>>>>>>>>>>> so
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> will
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> not have time to move this forward
> > until
> > >> > > > >> late
> > >> > > > >>>>>> August.
> > >> > > > >>>>>>>> That
> > >> > > > >>>>>>>>>>>>>>>> said,
> > >> > > > >>>>>>>>>>>>>>>>>>>> your
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I
> > >> just
> > >> > > > >>>> won't be
> > >> > > > >>>>>>>> able
> > >> > > > >>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>> respond
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> as
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> quickly as usual.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick
> > >> Telford <
> > >> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, that's correct, although the
> > impact
> > >> > > > >> on IQ
> > >> > > > >>>> is
> > >> > > > >>>>>> not
> > >> > > > >>>>>>>>>>>>>>>> something
> > >> > > > >>>>>>>>>>>>>>>>>> I
> > >> > > > >>>>>>>>>>>>>>>>>>>> had
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> considered.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the
> > state
> > >> > > > >> store
> > >> > > > >>>>>> from the
> > >> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and
> > >> writing
> > >> > > > >> the
> > >> > > > >>>>>>>> checkpoint
> > >> > > > >>>>>>>>>>>>>>>> (thus,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount
> > >> of
> > >> > > > >> data
> > >> > > > >>>>>> and/or
> > >> > > > >>>>>>>>>>>>>>>> number of
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you
> > >> > > > >> suggesting
> > >> > > > >>>>>> that we
> > >> > > > >>>>>>>>>>>>>>>> add an
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> additional
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> config for the max number of commit
> > >> > > > >> intervals
> > >> > > > >>>>>> between
> > >> > > > >>>>>>>>>>>>>>>>>> checkpoints?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> That
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when
> > >> the
> > >> > > > >>>>>> transaction
> > >> > > > >>>>>>>>>>>>>>>> buffers
> > >> > > > >>>>>>>>>>>>>>>>>> are
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> nearly
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number
> > of
> > >> > > > >> commit
> > >> > > > >>>>>>>> intervals
> > >> > > > >>>>>>>>>>>>>>>> have
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> elapsed,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> whichever comes first?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> That certainly seems reasonable,
> > >> although
> > >> > > > >> this
> > >> > > > >>>>>>>> re-ignites
> > >> > > > >>>>>>>>>>>>>>>> an
> > >> > > > >>>>>>>>>>>>>>>>>>>> earlier
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> debate about whether a config should
> > be
> > >> > > > >>>> measured in
> > >> > > > >>>>>>>>>>>>>>>> "number of
> > >> > > > >>>>>>>>>>>>>>>>>>>> commit
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> intervals", instead of just an
> > absolute
> > >> > > > >> time.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is
> > the
> > >> > > > >> reason
> > >> > > > >>>> I
> > >> > > > >>>>>> was
> > >> > > > >>>>>>>>>>>>>>>> pursuing
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Atomic
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable
> > >> > > > >> flush
> > >> > > > >>>> from
> > >> > > > >>>>>>>>>>>>>>>> checkpointing,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> which
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every
> > >> > > > >> commit
> > >> > > > >>>>>> without
> > >> > > > >>>>>>>> any
> > >> > > > >>>>>>>>>>>>>>>>>>>> performance
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is
> > >> definitely
> > >> > > > >> the
> > >> > > > >>>>>> "best"
> > >> > > > >>>>>>>>>>>>>>>> solution,
> > >> > > > >>>>>>>>>>>>>>>>>>>> but
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm not
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sure if this is enough to bring it
> > back
> > >> > > > >> into
> > >> > > > >>>> this
> > >> > > > >>>>>> KIP.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm currently working on moving all
> > the
> > >> > > > >>>>>> transactional
> > >> > > > >>>>>>>>>>>> logic
> > >> > > > >>>>>>>>>>>>>>>>>>>> directly
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> into
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away
> > >> with
> > >> > > > >> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore#newTransaction
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> method, and reduces the number of new
> > >> > > > >> classes
> > >> > > > >>>>>>>> introduced,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> significantly
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reducing the complexity. If it works,
> > >> and
> > >> > > > >> the
> > >> > > > >>>>>>>> complexity
> > >> > > > >>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> drastically
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reduced, I may try bringing back
> > Atomic
> > >> > > > >>>> Checkpoints
> > >> > > > >>>>>>>> into
> > >> > > > >>>>>>>>>>>>>>>> this
> > >> > > > >>>>>>>>>>>>>>>>>> KIP.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno
> > >> Cadonna
> > >> > > > >> <
> > >> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very
> > >> interesting!
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> As far as I understand, you want to
> > >> > > > >> atomically
> > >> > > > >>>>>> update
> > >> > > > >>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>> state
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> store
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush
> > the
> > >> > > > >>>> memtable
> > >> > > > >>>>>> of a
> > >> > > > >>>>>>>>>>>> state
> > >> > > > >>>>>>>>>>>>>>>>>> store
> > >> > > > >>>>>>>>>>>>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write the checkpoint not after the
> > >> commit
> > >> > > > >> time
> > >> > > > >>>>>> elapsed
> > >> > > > >>>>>>>>>>>> but
> > >> > > > >>>>>>>>>>>>>>>>>> after
> > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reached a size
> > that
> > >> > > > >> would
> > >> > > > >>>> lead
> > >> > > > >>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> exceeding
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > statestore.transaction.buffer.max.bytes
> > >> > > > >>>> before the
> > >> > > > >>>>>>>> next
> > >> > > > >>>>>>>>>>>>>>>> commit
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> interval
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> ends.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction
> > would
> > >> > > > >> commit
> > >> > > > >>>>>> every
> > >> > > > >>>>>>>>>>>>>>>> commit
> > >> > > > >>>>>>>>>>>>>>>>>>>> interval
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store will only be
> > atomically
> > >> > > > >>>> updated
> > >> > > > >>>>>>>> roughly
> > >> > > > >>>>>>>>>>>>>>>> every
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> statestore.transaction.buffer.max.bytes of
> > >> > > > >>>> data.
> > >> > > > >>>>>> Also
> > >> > > > >>>>>>>> IQ
> > >> > > > >>>>>>>>>>>>>>>> would
> > >> > > > >>>>>>>>>>>>>>>>>>>> then
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> only
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see new data roughly every
> > >> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> After a failure the state store needs
> > >> to
> > >> > > > >>>> restore
> > >> > > > >>>>>> up to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> statestore.transaction.buffer.max.bytes.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Is this correct?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the
> > >> state
> > >> > > > >> store
> > >> > > > >>>>>> from
> > >> > > > >>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and
> > >> writing
> > >> > > > >> the
> > >> > > > >>>>>>>> checkpoint
> > >> > > > >>>>>>>>>>>>>>>> (thus,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount
> > >> of
> > >> > > > >> data
> > >> > > > >>>>>> and/or
> > >> > > > >>>>>>>>>>>>>>>> number of
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would
> > have
> > >> > > > >> the
> > >> > > > >>>> same
> > >> > > > >>>>>> delay
> > >> > > > >>>>>>>>>>>> for
> > >> > > > >>>>>>>>>>>>>>>>>>>> records
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ
> > >> because
> > >> > > > >> both
> > >> > > > >>>>>> would
> > >> > > > >>>>>>>>>>>>>>>> appear
> > >> > > > >>>>>>>>>>>>>>>>>> when
> > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka transaction is committed.
> > >> However,
> > >> > > > >>>> after a
> > >> > > > >>>>>>>> failure
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > >> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it might restore data that is already
> > >> in
> > >> > > > >> the
> > >> > > > >>>> state
> > >> > > > >>>>>>>> store
> > >> > > > >>>>>>>>>>>>>>>>>> because
> > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last
> > stable
> > >> > > > >> offset
> > >> > > > >>>>>> (i.e.
> > >> > > > >>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>> last
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> committed
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset) of the changelog topics.
> > >> Restoring
> > >> > > > >>>> data
> > >> > > > >>>>>> that
> > >> > > > >>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>> already
> > >> > > > >>>>>>>>>>>>>>>>>>>> in
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos
> > >> should
> > >> > > > >> not
> > >> > > > >>>>>> violated.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> This solution needs at least one new
> > >> > > > >> config to
> > >> > > > >>>>>> specify
> > >> > > > >>>>>>>>>>>>>>>> when a
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> should be written.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> A small correction to your previous
> > >> e-mail
> > >> > > > >>>> that
> > >> > > > >>>>>> does
> > >> > > > >>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>> change
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> anything
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> you said: Under alos the default
> > commit
> > >> > > > >>>> interval
> > >> > > > >>>>>> is 30
> > >> > > > >>>>>>>>>>>>>>>> seconds,
> > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> five
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> seconds.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford
> > wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've begun performance testing my
> > >> branch
> > >> > > > >> on
> > >> > > > >>>> our
> > >> > > > >>>>>>>> staging
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> environment,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
> > >> > > > >>>> non-trivial
> > >> > > > >>>>>>>>>>>>>>>> application.
> > >> > > > >>>>>>>>>>>>>>>>>> I'm
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing the same increased flush
> > >> rate
> > >> > > > >> that
> > >> > > > >>>> we
> > >> > > > >>>>>> saw
> > >> > > > >>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>> last
> > >> > > > >>>>>>>>>>>>>>>>>>>> time
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> attempted to use a version of this
> > >> KIP,
> > >> > > > >> but
> > >> > > > >>>> this
> > >> > > > >>>>>>>> time,
> > >> > > > >>>>>>>>>> I
> > >> > > > >>>>>>>>>>>>>>>>>> think I
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> know
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> why.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit,
> > >> > > > >> which is
> > >> > > > >>>>>> called
> > >> > > > >>>>>>>> at
> > >> > > > >>>>>>>>>>>>>>>> the end
> > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Task
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit process, has the following
> > >> > > > >> behaviour:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under ALOS: checkpoint
> > >> the
> > >> > > > >> state
> > >> > > > >>>>>> stores.
> > >> > > > >>>>>>>>>> This
> > >> > > > >>>>>>>>>>>>>>>>>> includes
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            flushing memtables in
> > >> RocksDB.
> > >> > > > >>>> This is
> > >> > > > >>>>>>>>>>>> acceptable
> > >> > > > >>>>>>>>>>>>>>>>>>>> because the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> default
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit.interval.ms is 5
> > >> > > > >> seconds,
> > >> > > > >>>> so
> > >> > > > >>>>>>>> forcibly
> > >> > > > >>>>>>>>>>>>>>>> flushing
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> every 5
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            seconds is acceptable for
> > >> most
> > >> > > > >>>>>>>> applications.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under EOS:
> > checkpointing
> > >> is
> > >> > > > >> not
> > >> > > > >>>> done,
> > >> > > > >>>>>>>>>> *unless*
> > >> > > > >>>>>>>>>>>>>>>> it's
> > >> > > > >>>>>>>>>>>>>>>>>>>> being
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> forced, due
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to e.g. the Task closing
> > or
> > >> > > > >> being
> > >> > > > >>>>>> revoked.
> > >> > > > >>>>>>>>>> This
> > >> > > > >>>>>>>>>>>>>>>> means
> > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> under
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> normal
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing conditions,
> > the
> > >> > > > >> state
> > >> > > > >>>> stores
> > >> > > > >>>>>>>> will
> > >> > > > >>>>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpointed,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            not have memtables
> > flushed
> > >> at
> > >> > > > >> all ,
> > >> > > > >>>>>> unless
> > >> > > > >>>>>>>>>>>> RocksDB
> > >> > > > >>>>>>>>>>>>>>>>>>>> decides to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush them on
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            its own. Checkpointing
> > >> stores
> > >> > > > >> and
> > >> > > > >>>>>>>>>> force-flushing
> > >> > > > >>>>>>>>>>>>>>>> their
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            done when a Task is being
> > >> > > > >> closed.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to
> > checkpoint
> > >> > > > >>>> stores on
> > >> > > > >>>>>> at
> > >> > > > >>>>>>>>>>>> least
> > >> > > > >>>>>>>>>>>>>>>>>> *some*
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> normal
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the
> > >> > > > >> RocksDB
> > >> > > > >>>>>>>> transaction
> > >> > > > >>>>>>>>>>>>>>>>>> buffers
> > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the
> > >> offsets
> > >> > > > >> are
> > >> > > > >>>>>> synced to
> > >> > > > >>>>>>>>>>>>>>>> disk to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> prevent
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
> > >> > > > >>>> Consequently,
> > >> > > > >>>>>> my
> > >> > > > >>>>>>>>>>>>>>>> current
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every*
> > Task
> > >> > > > >> commit,
> > >> > > > >>>>>> which
> > >> > > > >>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>> far too
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> frequent.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000
> > >> > > > >> records,
> > >> > > > >>>>>> which
> > >> > > > >>>>>>>> is
> > >> > > > >>>>>>>>>> a
> > >> > > > >>>>>>>>>>>>>>>>>> change
> > >> > > > >>>>>>>>>>>>>>>>>>>> in
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
> > >> > > > >> performance
> > >> > > > >>>>>> problems
> > >> > > > >>>>>>>> for
> > >> > > > >>>>>>>>>>>>>>>> some
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> applications.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions,
> > >> and
> > >> > > > >> I'm
> > >> > > > >>>>>>>> currently
> > >> > > > >>>>>>>>>>>>>>>> leaning
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> towards
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> using the
> > >> > > > >>>> statestore.transaction.buffer.max.bytes
> > >> > > > >>>>>>>>>>>>>>>>>> configuration
> > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely
> > to
> > >> > > > >>>> exceed it.
> > >> > > > >>>>>>>> This
> > >> > > > >>>>>>>>>>>>>>>> would
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> complement the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
> > >> > > > >> functionality
> > >> > > > >>>> that
> > >> > > > >>>>>> this
> > >> > > > >>>>>>>>>>>>>>>>>>>> configuration
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Currently, we use
> > >> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> force an
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            early Task commit if
> > >> > processing
> > >> > > > >>>> more
> > >> > > > >>>>>>>> records
> > >> > > > >>>>>>>>>>>> would
> > >> > > > >>>>>>>>>>>>>>>>>> cause
> > >> > > > >>>>>>>>>>>>>>>>>>>> our
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> state
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> store
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            transactions to exceed
> > the
> > >> > > > >> memory
> > >> > > > >>>>>> assigned
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> them.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - New functionality:
> > when a
> > >> > > > >> Task
> > >> > > > >>>> *does*
> > >> > > > >>>>>>>>>> commit,
> > >> > > > >>>>>>>>>>>>>>>> we will
> > >> > > > >>>>>>>>>>>>>>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the stores (and hence
> > flush
> > >> > the
> > >> > > > >>>>>> transaction
> > >> > > > >>>>>>>>>>>>>>>> buffers)
> > >> > > > >>>>>>>>>>>>>>>>>>>> unless
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> expect to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            cross the
> > >> > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > >> > > > >>>>>>>>>>>>>>>>>>>> threshold
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the next
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick
> > >> > > > >> Telford <
> > >> > > > >>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I
> > >> > > > >> meant in
> > >> > > > >>>>>> terms of
> > >> > > > >>>>>>>>>>>>>>>>>>>> understanding
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance profile under various
> > >> > > > >>>>>> circumstances. The
> > >> > > > >>>>>>>>>>>>>>>> more
> > >> > > > >>>>>>>>>>>>>>>>>>>> complex
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it
> > would
> > >> > > > >> be for
> > >> > > > >>>>>> users
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>> understand
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance they see. For example,
> > >> > > > >> spilling
> > >> > > > >>>>>> records
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> disk
> > >> > > > >>>>>>>>>>>>>>>>>>>> when
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a
> > >> threshold
> > >> > > > >>>> would, I
> > >> > > > >>>>>>>>>> expect,
> > >> > > > >>>>>>>>>>>>>>>>>> reduce
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> write
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
> > >> > > > >>>> throughput
> > >> > > > >>>>>> could
> > >> > > > >>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> unexpected,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> > >> > > > >>>> diagnose/understand for
> > >> > > > >>>>>>>>>> users.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early
> > >> > > > >> commit"
> > >> > > > >>>>>> concept is
> > >> > > > >>>>>>>>>>>>>>>>>> relatively
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to
> > >> document,
> > >> > > > >> and
> > >> > > > >>>>>>>>>>>> conceptually
> > >> > > > >>>>>>>>>>>>>>>>>> fairly
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> obvious to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users. We could probably add a
> > >> metric to
> > >> > > > >>>> make it
> > >> > > > >>>>>>>>>> easier
> > >> > > > >>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it happens though.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially*
> > >> an
> > >> > > > >>>> indirect
> > >> > > > >>>>>> way
> > >> > > > >>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>>> telling
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether they should be
> > transactional.
> > >> > > > >>>>>> READ_COMMITTED
> > >> > > > >>>>>>>>>>>>>>>>>>>> essentially
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> requires
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates
> > >> that
> > >> > > > >> two
> > >> > > > >>>>>> threads
> > >> > > > >>>>>>>>>>>>>>>> calling
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see
> > >> writes
> > >> > > > >>>> from
> > >> > > > >>>>>> the
> > >> > > > >>>>>>>>>> other
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> transaction
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> until
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> > >> > > > >>>>>> READ_UNCOMMITTED, all
> > >> > > > >>>>>>>>>>>>>>>> bets are
> > >> > > > >>>>>>>>>>>>>>>>>>>> off,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe
> > >> > > > >> written
> > >> > > > >>>>>> records
> > >> > > > >>>>>>>> at
> > >> > > > >>>>>>>>>>>>>>>> any
> > >> > > > >>>>>>>>>>>>>>>>>>>> time,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> which is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That
> > >> > > > >> said,
> > >> > > > >>>>>>>> StateStores
> > >> > > > >>>>>>>>>>>>>>>> are
> > >> > > > >>>>>>>>>>>>>>>>>> free
> > >> > > > >>>>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implement
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these guarantees however they can,
> > >> > > > >> which is
> > >> > > > >>>> a
> > >> > > > >>>>>> bit
> > >> > > > >>>>>>>> more
> > >> > > > >>>>>>>>>>>>>>>>>> relaxed
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> than
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dictating "you must use
> > >> transactions".
> > >> > > > >> For
> > >> > > > >>>>>> example,
> > >> > > > >>>>>>>>>>>> with
> > >> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED
> > ==
> > >> > > > >>>> WBWI-based
> > >> > > > >>>>>>>>>>>>>>>>>> "transactions",
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes
> > to
> > >> the
> > >> > > > >>>>>> database.
> > >> > > > >>>>>>>> But
> > >> > > > >>>>>>>>>>>>>>>> with
> > >> > > > >>>>>>>>>>>>>>>>>>>> other
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> storage
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to
> > >> > > > >> *always*
> > >> > > > >>>> use
> > >> > > > >>>>>>>>>>>>>>>> transactions,
> > >> > > > >>>>>>>>>>>>>>>>>>>> even
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be
> > storage
> > >> > > > >> engines
> > >> > > > >>>>>> that
> > >> > > > >>>>>>>>>> don't
> > >> > > > >>>>>>>>>>>>>>>>>> provide
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation
> > >> > > > >> guarantees
> > >> > > > >>>> can
> > >> > > > >>>>>> be
> > >> > > > >>>>>>>> met
> > >> > > > >>>>>>>>>>>>>>>> using a
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> different
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> technique.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the
> > >> > > > >> StateStore
> > >> > > > >>>>>> interface
> > >> > > > >>>>>>>> as
> > >> > > > >>>>>>>>>>>>>>>>>> loosely
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> coupled
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as
> > possible,
> > >> to
> > >> > > > >> give
> > >> > > > >>>>>>>>>>>>>>>> implementers
> > >> > > > >>>>>>>>>>>>>>>>>> more
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> freedom, and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal
> > >> knowledge
> > >> > > > >>>>>> required.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> > >> > > > >>>> "IsolationLevel"
> > >> > > > >>>>>> might
> > >> > > > >>>>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> right
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make
> > >> it
> > >> > > > >> much
> > >> > > > >>>> more
> > >> > > > >>>>>>>>>>>>>>>> explicit if
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> required, e.g.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7-8.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I can make these changes either
> > later
> > >> > > > >> today
> > >> > > > >>>> or
> > >> > > > >>>>>>>>>>>> tomorrow.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Small update:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and
> > >> > > > >> fixed a
> > >> > > > >>>>>> bunch of
> > >> > > > >>>>>>>>>>>>>>>> issues
> > >> > > > >>>>>>>>>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the
> > tests
> > >> > > > >> pass,
> > >> > > > >>>>>> which is
> > >> > > > >>>>>>>>>>>>>>>>>> promising,
> > >> > > > >>>>>>>>>>>>>>>>>>>> but
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> it
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to undergo some performance
> > >> > > > >> testing. I
> > >> > > > >>>>>> haven't
> > >> > > > >>>>>>>>>>>>>>>> (yet)
> > >> > > > >>>>>>>>>>>>>>>>>>>> worked
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> on
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()`
> > >> stuff,
> > >> > > > >> but I
> > >> > > > >>>>>> would
> > >> > > > >>>>>>>>>>>>>>>> expect
> > >> > > > >>>>>>>>>>>>>>>>>> that,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
> > >> > > > >>>> difference. The
> > >> > > > >>>>>>>>>> branch
> > >> > > > >>>>>>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> available
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> at
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> > >> > > > >>>>>>>>>> if
> > >> > > > >>>>>>>>>>>>>>>>>> anyone
> > >> > > > >>>>>>>>>>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno
> > >> > > > >> Cadonna
> > >> > > > >>>> <
> > >> > > > >>>>>>>>>>>>>>>>>>>> cadonna@apache.org>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was
> > >> > > > >> actually
> > >> > > > >>>> also
> > >> > > > >>>>>> my
> > >> > > > >>>>>>>>>>>>>>>> point. I
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understood
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that John was proposing the
> > >> ingestion
> > >> > > > >> path
> > >> > > > >>>> as
> > >> > > > >>>>>> a way
> > >> > > > >>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> avoid
> > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> early
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I
> > misinterpreted
> > >> the
> > >> > > > >>>> intent.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that
> > >> actually
> > >> > > > >> it is
> > >> > > > >>>>>> public
> > >> > > > >>>>>>>>>>>>>>>> API. My
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> question
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects
> > >> normal
> > >> > > > >>>>>> processing.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for
> > the
> > >> > > > >> size
> > >> > > > >>>> of the
> > >> > > > >>>>>>>>>>>>>>>> transaction
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> buffer
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit
> > >> > > > >> affects
> > >> > > > >>>> the
> > >> > > > >>>>>> whole
> > >> > > > >>>>>>>>>>>>>>>>>> processing
> > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a
> > >> > > > >> state
> > >> > > > >>>> store
> > >> > > > >>>>>> is
> > >> > > > >>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>>>>>> confined to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills
> > over
> > >> and
> > >> > > > >>>>>> changes the
> > >> > > > >>>>>>>>>>>>>>>> behavior
> > >> > > > >>>>>>>>>>>>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with
> > >> you
> > >> > > > >> that
> > >> > > > >>>> it
> > >> > > > >>>>>> is a
> > >> > > > >>>>>>>>>>>>>>>> decent
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compromise. I
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the
> > downsides
> > >> > > > >> and
> > >> > > > >>>> list
> > >> > > > >>>>>> the
> > >> > > > >>>>>>>>>>>>>>>> options to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> overcome
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that
> > all
> > >> > > > >>>> options
> > >> > > > >>>>>> seem
> > >> > > > >>>>>>>>>>>> quite
> > >> > > > >>>>>>>>>>>>>>>>>> heavy
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compared
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand
> > >> > > > >> what you
> > >> > > > >>>>>> mean
> > >> > > > >>>>>>>> with
> > >> > > > >>>>>>>>>>>>>>>> "less
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> predictable
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
> > >> > > > >>>> alternatives
> > >> > > > >>>>>>>> really
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> interesting.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> But I
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds
> > >> good
> > >> > > > >> and
> > >> > > > >>>> we
> > >> > > > >>>>>> should
> > >> > > > >>>>>>>>>>>>>>>> continue
> > >> > > > >>>>>>>>>>>>>>>>>>>> with
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> it!
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my
> > >> > > > >> e-mail on
> > >> > > > >>>>>> June
> > >> > > > >>>>>>>>>> 20th:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the
> > reasoning
> > >> > > > >> behind
> > >> > > > >>>>>> putting
> > >> > > > >>>>>>>>>>>>>>>> isolation
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> level
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks!
> > >> Should
> > >> > > > >>>> that
> > >> > > > >>>>>> also
> > >> > > > >>>>>>>> be
> > >> > > > >>>>>>>>>> a
> > >> > > > >>>>>>>>>>>>>>>> way
> > >> > > > >>>>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> give
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to
> > >> > > > >> decide
> > >> > > > >>>>>> whether
> > >> > > > >>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> turn on
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more
> > >> concerned
> > >> > > > >> about
> > >> > > > >>>>>> how do
> > >> > > > >>>>>>>>>> you
> > >> > > > >>>>>>>>>>>>>>>> know
> > >> > > > >>>>>>>>>>>>>>>>>>>> if a
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be
> > written
> > >> > > > >> under
> > >> > > > >>>> EOS,
> > >> > > > >>>>>> if
> > >> > > > >>>>>>>> you
> > >> > > > >>>>>>>>>>>>>>>> do not
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> have a
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> way
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
> > >> > > > >>>> transactional or
> > >> > > > >>>>>> not.
> > >> > > > >>>>>>>>>> If
> > >> > > > >>>>>>>>>>>>>>>> a
> > >> > > > >>>>>>>>>>>>>>>>>> state
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file
> > >> can
> > >> > > > >> be
> > >> > > > >>>>>> written
> > >> > > > >>>>>>>>>>>>>>>> during
> > >> > > > >>>>>>>>>>>>>>>>>>>> normal
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state
> > >> > > > >> store
> > >> > > > >>>> is not
> > >> > > > >>>>>>>>>>>>>>>>>> transactional,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be
> > written
> > >> > > > >> under
> > >> > > > >>>> EOS.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My point was about not only
> > >> > > > >> considering the
> > >> > > > >>>>>> bytes
> > >> > > > >>>>>>>> in
> > >> > > > >>>>>>>>>>>>>>>> memory
> > >> > > > >>>>>>>>>>>>>>>>>> in
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> config
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes,
> > >> but
> > >> > > > >> also
> > >> > > > >>>>>> bytes
> > >> > > > >>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>> might
> > >> > > > >>>>>>>>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> spilled
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was
> > wondering
> > >> > > > >>>> whether you
> > >> > > > >>>>>>>>>> should
> > >> > > > >>>>>>>>>>>>>>>>>> remove
> > >> > > > >>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of
> > >> memory
> > >> > > > >>>> bytes to
> > >> > > > >>>>>> be
> > >> > > > >>>>>>>>>> used
> > >> > > > >>>>>>>>>>>>>>>> to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store
> > >> > > > >> records." My
> > >> > > > >>>>>>>> thinking
> > >> > > > >>>>>>>>>>>>>>>> was
> > >> > > > >>>>>>>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> even
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted
> > >> bytes to
> > >> > > > >>>> disk,
> > >> > > > >>>>>>>>>> limiting
> > >> > > > >>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> overall
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> bytes
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about
> > it
> > >> > > > >> again
> > >> > > > >>>> and
> > >> > > > >>>>>>>>>>>>>>>> considering
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> recent
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too
> > >> much
> > >> > > > >>>> sense
> > >> > > > >>>>>>>> anymore.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I like the name
> > >> > > > >>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes that
> > >> > > > >>>>>>>>>>>>>>>>>>>> you
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 8.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> > >> > > > >>>>>> implementation
> > >> > > > >>>>>>>>>>>>>>>> details) of
> > >> > > > >>>>>>>>>>>>>>>>>>>> how
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of
> > >> > > > >> changelog
> > >> > > > >>>>>>>>>>>>>>>> transactions,
> > >> > > > >>>>>>>>>>>>>>>>>>>> state
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing
> > >> would be
> > >> > > > >>>> great.
> > >> > > > >>>>>>>> Would
> > >> > > > >>>>>>>>>>>> be
> > >> > > > >>>>>>>>>>>>>>>>>> great
> > >> > > > >>>>>>>>>>>>>>>>>>>> if
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> you
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could also add some sentences
> > about
> > >> the
> > >> > > > >>>>>> behavior in
> > >> > > > >>>>>>>>>>>>>>>> case of
> > >> > > > >>>>>>>>>>>>>>>>>> a
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> failure.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For instance how does a
> > >> transactional
> > >> > > > >> state
> > >> > > > >>>>>> store
> > >> > > > >>>>>>>>>>>>>>>> recover
> > >> > > > >>>>>>>>>>>>>>>>>>>> after a
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
> > >> > > > >>>> transaction
> > >> > > > >>>>>>>> buffer,
> > >> > > > >>>>>>>>>>>>>>>> etc.
> > >> > > > >>>>>>>>>>>>>>>>>>>> (that
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> what
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point
> > 9.)
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford
> > >> wrote:
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue
> > >> that
> > >> > > > >>>>>>>>>>>>>>>> WriteBatchWithIndex
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or
> > likely
> > >> to
> > >> > > > >>>> exceed)
> > >> > > > >>>>>>>>>>>>>>>> configured
> > >> > > > >>>>>>>>>>>>>>>>>>>> memory
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needs to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
> > >> > > > >>>> Ultimately,
> > >> > > > >>>>>> any
> > >> > > > >>>>>>>>>>>>>>>> approach
> > >> > > > >>>>>>>>>>>>>>>>>>>> based
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> on
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cracking
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using
> > >> it in
> > >> > > > >>>> ways
> > >> > > > >>>>>> it's
> > >> > > > >>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>> really
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> designed
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen
> > >> > > > >> performance
> > >> > > > >>>> or
> > >> > > > >>>>>>>>>>>>>>>> consistency
> > >> > > > >>>>>>>>>>>>>>>>>>>> issues.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for
> > removing
> > >> > > > >> these
> > >> > > > >>>>>> early
> > >> > > > >>>>>>>>>>>>>>>> commits?
> > >> > > > >>>>>>>>>>>>>>>>>>>> While
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> not
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise
> > >> to
> > >> > > > >>>> ensure
> > >> > > > >>>>>>>>>>>>>>>> consistency
> > >> > > > >>>>>>>>>>>>>>>>>>>> whilst
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maintaining
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas
> > seem
> > >> > > > >> *very*
> > >> > > > >>>>>>>>>>>>>>>> complicated, and
> > >> > > > >>>>>>>>>>>>>>>>>>>> might
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable
> > for
> > >> > > > >> users
> > >> > > > >>>> as a
> > >> > > > >>>>>>>>>>>>>>>> consequence.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the
> > scope
> > >> of
> > >> > > > >> this
> > >> > > > >>>>>> KIP is
> > >> > > > >>>>>>>>>>>>>>>> growing a
> > >> > > > >>>>>>>>>>>>>>>>>>>> bit
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> out
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to
> > discuss
> > >> > > > >> ideas
> > >> > > > >>>> for
> > >> > > > >>>>>>>> future
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> improvements, I
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > >> > > > >>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the
> > scope
> > >> > > > >> down
> > >> > > > >>>> to a
> > >> > > > >>>>>>>> design
> > >> > > > >>>>>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> achieves
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> most
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant
> > sized
> > >> > > > >>>>>> restorations
> > >> > > > >>>>>>>>>>>>>>>> during
> > >> > > > >>>>>>>>>>>>>>>>>> dirty
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any
> > >> design
> > >> > > > >> that
> > >> > > > >>>>>> this KIP
> > >> > > > >>>>>>>>>>>>>>>> produces
> > >> > > > >>>>>>>>>>>>>>>>>>>> can
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ultimately
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future,
> > >> especially
> > >> > > > >> if
> > >> > > > >>>> the
> > >> > > > >>>>>> bulk
> > >> > > > >>>>>>>> of
> > >> > > > >>>>>>>>>>>>>>>> it is
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> internal
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next
> > >> week
> > >> > > > >>>> trying
> > >> > > > >>>>>> to
> > >> > > > >>>>>>>>>>>>>>>> re-work
> > >> > > > >>>>>>>>>>>>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to
> > >> remove
> > >> > > > >> the
> > >> > > > >>>>>>>>>>>>>>>> newTransaction()
> > >> > > > >>>>>>>>>>>>>>>>>>>>>> method,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> such
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation
> > detail
> > >> of
> > >> > > > >>>>>>>> RocksDBStore.
> > >> > > > >>>>>>>>>>>>>>>> That
> > >> > > > >>>>>>>>>>>>>>>>>>>> way, if
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in
> > the
> > >> > > > >> future,
> > >> > > > >>>>>> like
> > >> > > > >>>>>>>> the
> > >> > > > >>>>>>>>>>>>>>>> SST
> > >> > > > >>>>>>>>>>>>>>>>>> file
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> management
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do
> > so
> > >> > > > >> with
> > >> > > > >>>>>> little/no
> > >> > > > >>>>>>>>>>>> API
> > >> > > > >>>>>>>>>>>>>>>>>>>> changes.
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>>
> > >> > > > >>>>>>>>>>>
> > >> > > > >>>>>>>>>>
> > >> > > > >>>>>>>>>
> > >> > > > >>>>>>>>
> > >> > > > >>>>>>
> > >> > > > >>>>>>
> > >> > > > >>>>
> > >> > > > >>
> > >> > > > >
> > >> > >
> > >> >
> > >>
> > >
> >

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

Posted by Sophie Blee-Goldman <so...@responsive.dev>.
Makes sense to me! Regarding the new config name -- doesn't seem like
we have any *public *configs in StreamsConfig that set a standard yet, so
imo we're free to pick whatever we think sounds good.

However we do have a few internal configs that follow the pattern
*feature.name.enabled* so if we want to play it safe and adhere to the
established pattern, we should call it "transactional.statestores.enabled"

I also personally think x.y.enabled sounds better than enable.x.y, but
honestly I don't feel too strongly either way. I don't think it's worth
prolonging
the discussion any further over

I'll take a look at KIP-892 as soon as I can since it's a prereq for this
one.
Thanks for the updates!


On Thu, Apr 11, 2024 at 12:02 PM Nick Telford <ni...@gmail.com>
wrote:

> Hi everyone,
>
> After various off-list discussions, it's become clear there are still some
> contentious points to this KIP.
>
> 1.
> We agreed to change the "default.state.isolation.level" config property to
> be a transactional state store feature toggle. What should we name this?
> "enable.transactional.statestores"? What's the convention for this kind of
> flag?
>
> 2.
> The "atomic checkpointing" section has been broken out into KIP-1035 (with
> one significant change, and otherwise just more explanation). This is now a
> hard dependency of KIP-892.
>
> Regards,
> Nick
>
> On Mon, 6 Nov 2023 at 17:00, Nick Telford <ni...@gmail.com> wrote:
>
> > Hi everyone,
> >
> > Sorry for the delay.
> >
> > I've updated the KIP based on our discussion. I've also added some
> details
> > on "Transactional support under READ_UNCOMMITTED" and  "Query-time
> > Isolation Levels" in the "Rejected Alternatives" section, to highlight
> the
> > potential to improve things in a future KIP.
> >
> > If there are no more requested changes or objections, I'll kick off the
> > vote thread tomorrow, approximately 24 hours from now.
> >
> > While I'd love for this KIP to make it into 3.7.0, I can't commit to it
> > being code-complete by the December 20th deadline. While the branch I
> have
> > is broadly working, there are areas that will likely require improvement,
> > and others that are untested:
> >
> > - Untested outside of unit tests: Windowed stores, Versioned stores,
> > Global stores, IQv2. They should all work as expected, but the Position
> > file logic might be a bit janky.
> > - Requires improvement: Position data handling. Very clunky right now.
> > - Requires implementation: The "feature flag", where we disable
> > transactionality under EOS+READ_UNCOMMITTED. I've started this, but it's
> > far from simple. Plus this will require fleshing out the test suite
> quite a
> > bit.
> >
> > For the curious, my active development branch has been rebased against
> > 3.6.0: https://github.com/nicktelford/kafka/tree/KIP-892-3.6.0
> >
> > Regards,
> > Nick
> >
> > On Mon, 30 Oct 2023 at 23:32, Sophie Blee-Goldman <sophie@responsive.dev
> >
> > wrote:
> >
> >> Hey Nick, sounds like things are moving along here. I think you're
> already
> >> aware, but just as a reminder to us all, the KIP freeze deadline for 3.7
> >> is
> >> approaching and is currently set for Nov 18. I think we can all agree
> that
> >> it would be great to have this KIP accepted by then, and it sounds like
> >> much of the implementation is already completed, in which case this
> might
> >> be able to make it into 3.7.
> >>
> >> It sounds like this discussion is wrapping up so I just want to
> summarize
> >> my understanding of the current plan regarding configs since it doesn't
> >> seem like the KIP has been updated to include this yet.
> >>
> >> Basically we're all in agreement to go with option #1 that you presented
> >> earlier, right? Assuming so, I just want to clear up some details around
> >> the default behavior. What happens if:
> >> 1. User sets EOS and sets READ_UNCOMMITTED: txn state stores will be
> >> disabled/feature-flagged
> >> 2. User sets EOS and does not set anything for the isolation level:
> >> although the default is READ_UNCOMMITTED, enabling EOS will change the
> >> default to READ_COMMITTED and txn state stores will be used
> >>
> >> As for the future, it sounds like when READ_UNCOMMITTED mode is
> >> implemented, we will basically just remove this "feature flag" and txn
> >> state stores will always be used for all EOS cases. EOS users will be
> able
> >> to configure the isolation level independently, although it will still
> >> default to READ_COMMITTED when EOS is enabled and it wasn't explicitly
> >> configured.
> >>
> >> Is this understanding correct? I think this was the right choice
> >> regardless, as it will give people a way to disable the txn stores in an
> >> emergency -- as a project we went a long time with little pressure to
> >> feature flag things, and our users paid the cost for that. Even if we
> >> managed to ship something without bugs, it was often only after an
> intense
> >> period of successive blocker bugs that delayed the entire released for
> >> weeks. Other times, major bugs slipped through and some versions became
> >> unusable for certain use cases. So having some way to disable the txn
> >> state
> >> stores when EOS is used feels like a good strategy, since you just never
> >> know what might get through all the testing we do.
> >>
> >> If this ends up slipping to 4.0 and you manage to implement the
> >> READ_UNCOMMITTED mode within the same release, I think it's worth
> >> considering to add in an additional feature flag, even if it's just a
> >> backdoor internal config (eg as we did in KIP-441 with the internal task
> >> assignor config). But for now let's assume the first release this KIP
> >> appears in will have the behavior as described above, with
> >> READ_UNCOMMITTED
> >> mode acting as a feature flag
> >>
> >> Nick -- please let us know when you've updated the KIP to clarify the
> >> config behavior, and are ready for a vote!
> >>
> >>
> >> On Sun, Oct 29, 2023 at 12:02 PM Colt McNealy <co...@littlehorse.io>
> >> wrote:
> >>
> >> > Guozhang—I agree, I am in favor of moving forward with the KIP now
> that
> >> the
> >> > Transactional State Stores will be behind a feature flag.
> >> >
> >> > Nick—I just did a bit more light testing of your branch
> `KIP-892-3.5.0`
> >> > with your most recent changes. I couldn't detect a performance
> >> difference
> >> > versus trunk (in the past there was a slight degradation of
> performance
> >> on
> >> > the restoration path, but that has been fixed). I don't believe that
> >> your
> >> > branch has the state updater thread enabled, so I didn't test that
> path
> >> too
> >> > heavily.
> >> >
> >> > As expected, however, our internal correctness tests failed due to the
> >> IQ
> >> > read-your-own-writes issue we discussed previously. The community as a
> >> > whole would vastly benefit from this KIP getting over the finish line
> in
> >> > 3.7.0, and so long as it is behind a feature flag so that we at
> >> LittleHorse
> >> > can still guarantee RYOW for our users, I think it's purely a win 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 <
> >> guozhang.wang.us@gmail.com
> >> > >
> >> > wrote:
> >> >
> >> > > I'd agree with you guys that as long as we are in agreement about
> the
> >> > > configuration semantics, that would be a big win to move forward for
> >> > > this KIP. As for the TaskCorruptedException handling like wiping
> state
> >> > > stores, we can discuss that in the PR rather than in the KIP.
> >> > >
> >> > > Just to clarify, I'm onboard with the latest proposal, and probably
> we
> >> > > can move on for voting on this KIP now?
> >> > >
> >> > > Guozhang
> >> > >
> >> > > On Thu, Oct 19, 2023 at 5:33 AM Bruno Cadonna <ca...@apache.org>
> >> > wrote:
> >> > > >
> >> > > > Hi Nick,
> >> > > >
> >> > > > What you and Lucas wrote about the different configurations of
> >> ALOS/EOS
> >> > > > and READ_COMMITTED/READ_UNCOMMITTED make sense to me. My earlier
> >> > > > concerns about changelogs diverging from the content of the local
> >> state
> >> > > > stores turned out to not apply. So I think, we can move on with
> >> those
> >> > > > configurations.
> >> > > >
> >> > > > Regarding the TaskCorruptedException and wiping out the state
> stores
> >> > > > under EOS, couldn't we abort the transaction on the state store
> and
> >> > > > close the task dirty? If the Kafka transaction was indeed
> committed,
> >> > the
> >> > > > store would restore the missing part from the changelog topic. If
> >> the
> >> > > > Kafka transaction was not committed, changelog topic and state
> store
> >> > are
> >> > > > in-sync.
> >> > > >
> >> > > > In any case, IMO those are implementation details that we do not
> >> need
> >> > to
> >> > > > discuss and solve in the KIP discussion. We can solve them on the
> >> PR.
> >> > > > The important thing is that the processing guarantees hold.
> >> > > >
> >> > > > Best,
> >> > > > Bruno
> >> > > >
> >> > > > On 10/18/23 3:56 PM, Nick Telford wrote:
> >> > > > > Hi Lucas,
> >> > > > >
> >> > > > > TaskCorruptedException is how Streams signals that the Task
> state
> >> > > needs to
> >> > > > > be wiped, so we can't retain that exception without also wiping
> >> state
> >> > > on
> >> > > > > timeouts.
> >> > > > >
> >> > > > > Regards,
> >> > > > > Nick
> >> > > > >
> >> > > > > On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <
> >> lbrutschy@confluent.io
> >> > > .invalid>
> >> > > > > wrote:
> >> > > > >
> >> > > > >> Hi Nick,
> >> > > > >>
> >> > > > >> I think indeed the better behavior would be to retry
> >> > commitTransaction
> >> > > > >> until we risk running out of time to meet `
> max.poll.interval.ms
> >> `.
> >> > > > >>
> >> > > > >> However, if it's handled as a `TaskCorruptedException` at the
> >> > moment,
> >> > > > >> I would do the same in this KIP, and leave exception handling
> >> > > > >> improvements to future work. 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 <
> >> > nick.telford@gmail.com>
> >> > > > >> wrote:
> >> > > > >>>
> >> > > > >>> Hi Lucas,
> >> > > > >>>
> >> > > > >>> Yeah, this is pretty much the direction I'm thinking of going
> in
> >> > > now. You
> >> > > > >>> make an interesting point about committing on-error under
> >> > > > >>> ALOS/READ_COMMITTED, although I haven't had a chance to think
> >> > > through the
> >> > > > >>> implications yet.
> >> > > > >>>
> >> > > > >>> Something that I ran into earlier this week is an issue with
> the
> >> > new
> >> > > > >>> handling of TimeoutException. Without TX stores,
> >> TimeoutException
> >> > > under
> >> > > > >> EOS
> >> > > > >>> throws a TaskCorruptedException, which wipes the stores.
> >> However,
> >> > > with TX
> >> > > > >>> stores, TimeoutException is now just bubbled up and dealt with
> >> as
> >> > it
> >> > > is
> >> > > > >>> under ALOS. The problem arises when the
> >> Producer#commitTransaction
> >> > > call
> >> > > > >>> times out: Streams attempts to ignore the error and continue
> >> > > producing,
> >> > > > >>> which causes the next call to Producer#send to throw
> >> > > > >>> "IllegalStateException: Cannot attempt operation `send`
> because
> >> the
> >> > > > >>> previous call to `commitTransaction` timed out and must be
> >> > retried".
> >> > > > >>>
> >> > > > >>> I'm not sure what we should do here: retrying the
> >> commitTransaction
> >> > > seems
> >> > > > >>> logical, but what if it times out again? Where do we draw the
> >> line
> >> > > and
> >> > > > >>> shutdown the instance?
> >> > > > >>>
> >> > > > >>> Regards,
> >> > > > >>> Nick
> >> > > > >>>
> >> > > > >>> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <
> >> > lbrutschy@confluent.io
> >> > > > >> .invalid>
> >> > > > >>> wrote:
> >> > > > >>>
> >> > > > >>>> Hi all,
> >> > > > >>>>
> >> > > > >>>> I think I liked your suggestion of allowing EOS with
> >> > > READ_UNCOMMITTED,
> >> > > > >>>> but keep wiping the state on error, and I'd vote for this
> >> solution
> >> > > > >>>> when introducing `default.state.isolation.level`. This way,
> >> we'd
> >> > > have
> >> > > > >>>> the most low-risk roll-out of this feature (no behavior
> change
> >> > > without
> >> > > > >>>> reconfiguration), with the possibility of switching to the
> most
> >> > > sane /
> >> > > > >>>> battle-tested default settings in 4.0. Essentially, we'd
> have a
> >> > > > >>>> feature flag but call it `default.state.isolation.level` and
> >> don't
> >> > > > >>>> have to deprecate it later.
> >> > > > >>>>
> >> > > > >>>> So the possible configurations would then be this:
> >> > > > >>>>
> >> > > > >>>> 1. ALOS/READ_UNCOMMITTED (default) = processing uses
> >> direct-to-DB,
> >> > > IQ
> >> > > > >>>> reads from DB.
> >> > > > >>>> 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads
> >> from
> >> > > > >>>> WriteBatch/DB. Flush on error (see note below).
> >> > > > >>>> 3. EOS/READ_UNCOMMITTED (default) = processing uses
> >> direct-to-DB,
> >> > IQ
> >> > > > >>>> reads from DB. Wipe state on error.
> >> > > > >>>> 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads
> >> from
> >> > > > >>>> WriteBatch/DB.
> >> > > > >>>>
> >> > > > >>>> I believe the feature is important enough that we will see
> good
> >> > > > >>>> adoption even without changing the default. In 4.0, when we
> >> have
> >> > > seen
> >> > > > >>>> this being adopted and is battle-tested, we make
> READ_COMMITTED
> >> > the
> >> > > > >>>> default for EOS, or even READ_COMITTED always the default,
> >> > depending
> >> > > > >>>> on our experiences. And we could add a clever implementation
> of
> >> > > > >>>> READ_UNCOMITTED with WriteBatches later.
> >> > > > >>>>
> >> > > > >>>> The only smell here is that `default.state.isolation.level`
> >> > wouldn't
> >> > > > >>>> be purely an IQ setting, but it would also (slightly) change
> >> the
> >> > > > >>>> behavior of the processing, but that seems unavoidable as
> long
> >> as
> >> > we
> >> > > > >>>> haven't solve READ_UNCOMITTED IQ with WriteBatches.
> >> > > > >>>>
> >> > > > >>>> Minor: As for Bruno's point 4, I think if we are concerned
> >> about
> >> > > this
> >> > > > >>>> 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).
> >> > > > >>>>
> >> > > > >>>> Cheers,
> >> > > > >>>> Lucas
> >> > > > >>>>
> >> > > > >>>> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <
> >> > > nick.telford@gmail.com>
> >> > > > >>>> wrote:
> >> > > > >>>>>
> >> > > > >>>>> Hi Guozhang,
> >> > > > >>>>>
> >> > > > >>>>> The KIP as it stands introduces a new configuration,
> >> > > > >>>>> default.state.isolation.level, which is independent of
> >> > > > >> processing.mode.
> >> > > > >>>>> It's intended that this new configuration be used to
> >> configure a
> >> > > > >> global
> >> > > > >>>> IQ
> >> > > > >>>>> isolation level in the short term, with a future KIP
> >> introducing
> >> > > the
> >> > > > >>>>> capability to change the isolation level on a per-query
> basis,
> >> > > > >> falling
> >> > > > >>>> back
> >> > > > >>>>> to the "default" defined by this config. That's why I called
> >> it
> >> > > > >>>> "default",
> >> > > > >>>>> for future-proofing.
> >> > > > >>>>>
> >> > > > >>>>> However, it currently includes the caveat that
> >> READ_UNCOMMITTED
> >> > is
> >> > > > >> not
> >> > > > >>>>> available under EOS. I think this is the coupling you are
> >> > alluding
> >> > > > >> to?
> >> > > > >>>>>
> >> > > > >>>>> This isn't intended to be a restriction of the API, but is
> >> > > currently
> >> > > > >> a
> >> > > > >>>>> technical limitation. However, after discussing with some
> >> users
> >> > > about
> >> > > > >>>>> use-cases that would require READ_UNCOMMITTED under EOS, I'm
> >> > > > >> inclined to
> >> > > > >>>>> remove that clause and put in the necessary work to make
> that
> >> > > > >> combination
> >> > > > >>>>> possible now.
> >> > > > >>>>>
> >> > > > >>>>> I currently see two possible approaches:
> >> > > > >>>>>
> >> > > > >>>>>     1. Disable TX StateStores internally when the
> >> IsolationLevel
> >> > is
> >> > > > >>>>>     READ_UNCOMMITTED and the processing.mode is EOS. This is
> >> more
> >> > > > >>>> difficult
> >> > > > >>>>>     than it sounds, as there are many assumptions being made
> >> > > > >> throughout
> >> > > > >>>> the
> >> > > > >>>>>     internals about the guarantees StateStores provide. It
> >> would
> >> > > > >>>> definitely add
> >> > > > >>>>>     a lot of extra "if (read_uncommitted && eos)" branches,
> >> > > > >> complicating
> >> > > > >>>>>     maintenance and testing.
> >> > > > >>>>>     2. Invest the time *now* to make READ_UNCOMMITTED of EOS
> >> > > > >> StateStores
> >> > > > >>>>>     possible. I have some ideas on how this could be
> achieved,
> >> > but
> >> > > > >> they
> >> > > > >>>> would
> >> > > > >>>>>     need testing and could introduce some additional issues.
> >> The
> >> > > > >> benefit
> >> > > > >>>> of
> >> > > > >>>>>     this approach is that it would make query-time
> >> > IsolationLevels
> >> > > > >> much
> >> > > > >>>> simpler
> >> > > > >>>>>     to implement in the future.
> >> > > > >>>>>
> >> > > > >>>>> Unfortunately, both will require considerable work that will
> >> > > further
> >> > > > >>>> delay
> >> > > > >>>>> this KIP, which was the reason I placed the restriction in
> the
> >> > KIP
> >> > > > >> in the
> >> > > > >>>>> first place.
> >> > > > >>>>>
> >> > > > >>>>> Regards,
> >> > > > >>>>> Nick
> >> > > > >>>>>
> >> > > > >>>>> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
> >> > > > >> guozhang.wang.us@gmail.com>
> >> > > > >>>>> wrote:
> >> > > > >>>>>
> >> > > > >>>>>> Hello Nick,
> >> > > > >>>>>>
> >> > > > >>>>>> First of all, thanks a lot for the great effort you've put
> in
> >> > > > >> driving
> >> > > > >>>>>> this KIP! I really like it coming through finally, as many
> >> > people
> >> > > > >> in
> >> > > > >>>>>> the community have raised this. At the same time I honestly
> >> > feel a
> >> > > > >> bit
> >> > > > >>>>>> ashamed for not putting enough of my time supporting it and
> >> > > > >> pushing it
> >> > > > >>>>>> through the finish line (you raised this KIP almost a year
> >> ago).
> >> > > > >>>>>>
> >> > > > >>>>>> I briefly passed through the DISCUSS thread so far, not
> sure
> >> > I've
> >> > > > >> 100
> >> > > > >>>>>> percent digested all the bullet points. But with the goal
> of
> >> > > > >> trying to
> >> > > > >>>>>> help take it through the finish line in mind, I'd want to
> >> throw
> >> > > > >>>>>> thoughts on top of my head only on the point #4 above
> which I
> >> > felt
> >> > > > >> may
> >> > > > >>>>>> be the main hurdle for the current KIP to drive to a
> >> consensus
> >> > > now.
> >> > > > >>>>>>
> >> > > > >>>>>> The general question I asked myself is, whether we want to
> >> > couple
> >> > > > >> "IQ
> >> > > > >>>>>> reading mode" with "processing mode". While technically I
> >> tend
> >> > to
> >> > > > >>>>>> agree with you that, it's feels like a bug if some single
> >> user
> >> > > > >> chose
> >> > > > >>>>>> "EOS" for processing mode while choosing "read uncommitted"
> >> for
> >> > IQ
> >> > > > >>>>>> reading mode, at the same time, I'm thinking if it's
> possible
> >> > that
> >> > > > >>>>>> there could be two different persons (or even two teams)
> that
> >> > > > >> would be
> >> > > > >>>>>> using the stream API to build the app, and the IQ API to
> >> query
> >> > the
> >> > > > >>>>>> running state of the app. I know this is less of a
> technical
> >> > thing
> >> > > > >> but
> >> > > > >>>>>> rather a more design stuff, but if it could be ever the
> case,
> >> > I'm
> >> > > > >>>>>> wondering if the personale using the IQ API knows about the
> >> > risks
> >> > > > >> of
> >> > > > >>>>>> using read uncommitted but still chose so for the favor of
> >> > > > >>>>>> performance, no matter if the underlying stream processing
> >> mode
> >> > > > >>>>>> configured by another personale is EOS or not. In that
> >> regard,
> >> > I'm
> >> > > > >>>>>> leaning towards a "leaving the door open, and close it
> later
> >> if
> >> > we
> >> > > > >>>>>> found it's a bad idea" aspect with a configuration 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 can still
> >> enforce
> >> > > > >>>>>> coupling them.
> >> > > > >>>>>>
> >> > > > >>>>>> Again, just my 2c here. Thanks again for the great patience
> >> and
> >> > > > >>>>>> diligence on this KIP.
> >> > > > >>>>>>
> >> > > > >>>>>>
> >> > > > >>>>>> Guozhang
> >> > > > >>>>>>
> >> > > > >>>>>>
> >> > > > >>>>>>
> >> > > > >>>>>> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
> >> > > > >> nick.telford@gmail.com>
> >> > > > >>>>>> wrote:
> >> > > > >>>>>>>
> >> > > > >>>>>>> Hi Bruno,
> >> > > > >>>>>>>
> >> > > > >>>>>>> 4.
> >> > > > >>>>>>> I'll hold off on making that change until we have a
> >> consensus
> >> > as
> >> > > > >> to
> >> > > > >>>> what
> >> > > > >>>>>>> configuration to use to control all of this, as it'll be
> >> > > > >> affected by
> >> > > > >>>> the
> >> > > > >>>>>>> decision on EOS isolation levels.
> >> > > > >>>>>>>
> >> > > > >>>>>>> 5.
> >> > > > >>>>>>> Done. I've chosen "committedOffsets".
> >> > > > >>>>>>>
> >> > > > >>>>>>> Regards,
> >> > > > >>>>>>> Nick
> >> > > > >>>>>>>
> >> > > > >>>>>>> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <
> >> > cadonna@apache.org>
> >> > > > >>>> wrote:
> >> > > > >>>>>>>
> >> > > > >>>>>>>> Hi Nick,
> >> > > > >>>>>>>>
> >> > > > >>>>>>>> 1.
> >> > > > >>>>>>>> Yeah, you are probably right that it does not make too
> much
> >> > > > >> sense.
> >> > > > >>>>>>>> Thanks for the clarification!
> >> > > > >>>>>>>>
> >> > > > >>>>>>>>
> >> > > > >>>>>>>> 4.
> >> > > > >>>>>>>> Yes, sorry for the back and forth, but I think for the
> >> sake of
> >> > > > >> the
> >> > > > >>>> KIP
> >> > > > >>>>>>>> it is better to let the ALOS behavior as it is for now
> due
> >> to
> >> > > > >> the
> >> > > > >>>>>>>> possible issues you would run into. Maybe we can find a
> >> > > > >> solution
> >> > > > >>>> in the
> >> > > > >>>>>>>> future. Now the question returns to whether we really
> need
> >> > > > >>>>>>>> default.state.isolation.level. Maybe the config could be
> >> the
> >> > > > >>>> feature
> >> > > > >>>>>>>> flag Sophie requested.
> >> > > > >>>>>>>>
> >> > > > >>>>>>>>
> >> > > > >>>>>>>> 5.
> >> > > > >>>>>>>> There 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
> >> committedOffsetFor().
> >> > > > >>>>>>>>
> >> > > > >>>>>>>>
> >> > > > >>>>>>>> Best,
> >> > > > >>>>>>>> Bruno
> >> > > > >>>>>>>>
> >> > > > >>>>>>>>
> >> > > > >>>>>>>> On 10/13/23 3:21 PM, Nick Telford wrote:
> >> > > > >>>>>>>>> Hi Bruno,
> >> > > > >>>>>>>>>
> >> > > > >>>>>>>>> Thanks for getting back to me.
> >> > > > >>>>>>>>>
> >> > > > >>>>>>>>> 1.
> >> > > > >>>>>>>>> I think this should be possible. Are you thinking of the
> >> > > > >>>> situation
> >> > > > >>>>>> where
> >> > > > >>>>>>>> a
> >> > > > >>>>>>>>> user may downgrade to a previous version of Kafka
> >> Streams? In
> >> > > > >>>> that
> >> > > > >>>>>> case,
> >> > > > >>>>>>>>> sadly, the RocksDBStore would get wiped by the older
> >> version
> >> > > > >> of
> >> > > > >>>> Kafka
> >> > > > >>>>>>>>> Streams anyway, because that version wouldn't understand
> >> the
> >> > > > >>>> extra
> >> > > > >>>>>> column
> >> > > > >>>>>>>>> family (that holds offsets), so the missing Position
> file
> >> > > > >> would
> >> > > > >>>>>>>>> automatically get rebuilt when the store is rebuilt from
> >> the
> >> > > > >>>>>> changelog.
> >> > > > >>>>>>>>> Are there other situations than downgrade where a
> >> > > > >> transactional
> >> > > > >>>> store
> >> > > > >>>>>>>> could
> >> > > > >>>>>>>>> be replaced by a non-transactional one? I can't think of
> >> any.
> >> > > > >>>>>>>>>
> >> > > > >>>>>>>>> 2.
> >> > > > >>>>>>>>> Ahh yes, the Test Plan - my Kryptonite! This section
> >> > > > >> definitely
> >> > > > >>>>>> needs to
> >> > > > >>>>>>>> be
> >> > > > >>>>>>>>> fleshed out. I'll work on that. How much detail do you
> >> need?
> >> > > > >>>>>>>>>
> >> > > > >>>>>>>>> 3.
> >> > > > >>>>>>>>> See my previous email discussing this.
> >> > > > >>>>>>>>>
> >> > > > >>>>>>>>> 4.
> >> > > > >>>>>>>>> Hmm, this is an interesting point. Are you suggesting
> that
> >> > > > >> under
> >> > > > >>>> ALOS
> >> > > > >>>>>>>>> READ_COMMITTED should not be supported?
> >> > > > >>>>>>>>>
> >> > > > >>>>>>>>> Regards,
> >> > > > >>>>>>>>> Nick
> >> > > > >>>>>>>>>
> >> > > > >>>>>>>>> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
> >> > > > >> cadonna@apache.org>
> >> > > > >>>>>> wrote:
> >> > > > >>>>>>>>>
> >> > > > >>>>>>>>>> Hi Nick,
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>> I think the KIP is converging!
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>> 1.
> >> > > > >>>>>>>>>> I am wondering whether it makes sense to write the
> >> position
> >> > > > >> file
> >> > > > >>>>>> during
> >> > > > >>>>>>>>>> close as we do for the checkpoint file, so that in case
> >> the
> >> > > > >>>> state
> >> > > > >>>>>> store
> >> > > > >>>>>>>>>> is replaced with a non-transactional state store the
> >> > > > >>>>>> non-transactional
> >> > > > >>>>>>>>>> state store finds the position file. I think, this is
> not
> >> > > > >>>> strictly
> >> > > > >>>>>>>>>> needed, but would be a nice behavior instead of just
> >> > > > >> deleting
> >> > > > >>>> the
> >> > > > >>>>>>>>>> position file.
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>> 2.
> >> > > > >>>>>>>>>> The test plan does not mention integration tests. Do
> you
> >> not
> >> > > > >>>> need to
> >> > > > >>>>>>>>>> extend existing ones and add new ones. Also for
> upgrading
> >> > > > >> and
> >> > > > >>>>>>>>>> downgrading you might need integration and/or system
> >> tests.
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>> 3.
> >> > > > >>>>>>>>>> I think Sophie made a point. Although, IQ reading from
> >> > > > >>>> uncommitted
> >> > > > >>>>>> data
> >> > > > >>>>>>>>>> under EOS might be considered a bug by some people.
> Thus,
> >> > > > >> your
> >> > > > >>>> KIP
> >> > > > >>>>>> would
> >> > > > >>>>>>>>>> fix a bug rather than changing the intended behavior.
> >> > > > >> However, I
> >> > > > >>>>>> also
> >> > > > >>>>>>>>>> see that a feature flag would help users that rely on
> >> this
> >> > > > >> buggy
> >> > > > >>>>>>>>>> behavior (at least until AK 4.0).
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>> 4.
> >> > > > >>>>>>>>>> This is related to the previous point. I assume that
> the
> >> > > > >>>> difference
> >> > > > >>>>>>>>>> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is
> >> > > > >> that in
> >> > > > >>>> the
> >> > > > >>>>>>>>>> former you enable transactions on the state store and
> in
> >> the
> >> > > > >>>> latter
> >> > > > >>>>>> you
> >> > > > >>>>>>>>>> disable them. If my assumption is correct, I think that
> >> is
> >> > > > >> an
> >> > > > >>>> issue.
> >> > > > >>>>>>>>>> Let's assume under ALOS Streams fails over a couple of
> >> times
> >> > > > >>>> more or
> >> > > > >>>>>>>>>> less at the same step in processing after value 3 is
> >> added
> >> > > > >> to an
> >> > > > >>>>>>>>>> aggregation but the offset of the corresponding input
> >> record
> >> > > > >>>> was not
> >> > > > >>>>>>>>>> committed. Without transactions disabled, the
> aggregation
> >> > > > >> value
> >> > > > >>>>>> would
> >> > > > >>>>>>>>>> increase by 3 for each failover. With transactions
> >> enabled,
> >> > > > >>>> value 3
> >> > > > >>>>>>>>>> would only be added to the aggregation once when the
> >> offset
> >> > > > >> of
> >> > > > >>>> the
> >> > > > >>>>>> input
> >> > > > >>>>>>>>>> record is committed and the transaction finally
> >> completes.
> >> > > > >> So
> >> > > > >>>> the
> >> > > > >>>>>>>>>> content of the state store would change depending on
> the
> >> > > > >>>>>> configuration
> >> > > > >>>>>>>>>> for IQ. IMO, the content of the state store should be
> >> > > > >>>> independent
> >> > > > >>>>>> from
> >> > > > >>>>>>>>>> IQ. Given this issue, I propose to not use transactions
> >> with
> >> > > > >>>> ALOS at
> >> > > > >>>>>>>>>> all. I was a big proponent of using transactions with
> >> ALOS,
> >> > > > >> but
> >> > > > >>>> I
> >> > > > >>>>>>>>>> realized that transactions with ALOS is not as easy as
> >> > > > >> enabling
> >> > > > >>>>>>>>>> transactions on state stores. Another aspect that is
> >> > > > >>>> problematic is
> >> > > > >>>>>> that
> >> > > > >>>>>>>>>> the changelog topic which actually replicates the state
> >> > > > >> store
> >> > > > >>>> is not
> >> > > > >>>>>>>>>> transactional under ALOS. Thus, it might happen that
> the
> >> > > > >> state
> >> > > > >>>>>> store and
> >> > > > >>>>>>>>>> the changelog differ in their content. All of this is
> >> maybe
> >> > > > >>>> solvable
> >> > > > >>>>>>>>>> somehow, but for the sake of this KIP, I would leave it
> >> for
> >> > > > >> the
> >> > > > >>>>>> future.
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>> Best,
> >> > > > >>>>>>>>>> Bruno
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> >> > > > >>>>>>>>>>> Hey Nick! First of all thanks for taking up this
> awesome
> >> > > > >>>> feature,
> >> > > > >>>>>> I'm
> >> > > > >>>>>>>>>> sure
> >> > > > >>>>>>>>>>> every single
> >> > > > >>>>>>>>>>> Kafka Streams user and dev would agree that it is
> sorely
> >> > > > >>>> needed.
> >> > > > >>>>>>>>>>>
> >> > > > >>>>>>>>>>> I've just been catching up on the KIP and surrounding
> >> > > > >>>> discussion,
> >> > > > >>>>>> so
> >> > > > >>>>>>>>>> please
> >> > > > >>>>>>>>>>> forgive me
> >> > > > >>>>>>>>>>> for any misunderstandings or misinterpretations of the
> >> > > > >> current
> >> > > > >>>>>> plan and
> >> > > > >>>>>>>>>>> don't hesitate to
> >> > > > >>>>>>>>>>> correct me.
> >> > > > >>>>>>>>>>>
> >> > > > >>>>>>>>>>> Before I jump in, I just want to say that having seen
> >> this
> >> > > > >>>> drag on
> >> > > > >>>>>> for
> >> > > > >>>>>>>> so
> >> > > > >>>>>>>>>>> long, my singular
> >> > > > >>>>>>>>>>> goal in responding is to help this KIP past a
> perceived
> >> > > > >>>> impasse so
> >> > > > >>>>>> we
> >> > > > >>>>>>>> can
> >> > > > >>>>>>>>>>> finally move on
> >> > > > >>>>>>>>>>> to voting and implementing it. Long discussions are to
> >> be
> >> > > > >>>> expected
> >> > > > >>>>>> for
> >> > > > >>>>>>>>>>> major features like
> >> > > > >>>>>>>>>>> this but it's completely on us as the Streams devs to
> >> make
> >> > > > >> sure
> >> > > > >>>>>> there
> >> > > > >>>>>>>> is
> >> > > > >>>>>>>>>> an
> >> > > > >>>>>>>>>>> end in sight
> >> > > > >>>>>>>>>>> for any ongoing discussion.
> >> > > > >>>>>>>>>>>
> >> > > > >>>>>>>>>>> With that said, it's my understanding that the KIP as
> >> > > > >> currently
> >> > > > >>>>>>>> proposed
> >> > > > >>>>>>>>>> is
> >> > > > >>>>>>>>>>> just not tenable
> >> > > > >>>>>>>>>>> for Kafka Streams, and would prevent some EOS users
> from
> >> > > > >>>> upgrading
> >> > > > >>>>>> to
> >> > > > >>>>>>>> the
> >> > > > >>>>>>>>>>> version it
> >> > > > >>>>>>>>>>> first appears in. Given that we can't predict or
> >> guarantee
> >> > > > >>>> whether
> >> > > > >>>>>> any
> >> > > > >>>>>>>> of
> >> > > > >>>>>>>>>>> the followup KIPs
> >> > > > >>>>>>>>>>> would be completed in the same release cycle as this
> >> one,
> >> > > > >> we
> >> > > > >>>> need
> >> > > > >>>>>> to
> >> > > > >>>>>>>> make
> >> > > > >>>>>>>>>>> sure that the
> >> > > > >>>>>>>>>>> feature is either compatible with all current users or
> >> else
> >> > > > >>>>>>>>>> feature-flagged
> >> > > > >>>>>>>>>>> so that they may
> >> > > > >>>>>>>>>>> opt in/out.
> >> > > > >>>>>>>>>>>
> >> > > > >>>>>>>>>>> Therefore, IIUC we need to have either (or both) of
> >> these
> >> > > > >> as
> >> > > > >>>>>>>>>>> fully-implemented config options:
> >> > > > >>>>>>>>>>> 1. default.state.isolation.level
> >> > > > >>>>>>>>>>> 2. enable.transactional.state.stores
> >> > > > >>>>>>>>>>>
> >> > > > >>>>>>>>>>> This way EOS users for whom read_committed semantics
> are
> >> > > > >> not
> >> > > > >>>>>> viable can
> >> > > > >>>>>>>>>>> still upgrade,
> >> > > > >>>>>>>>>>> and either use the isolation.level config to leverage
> >> the
> >> > > > >> new
> >> > > > >>>> txn
> >> > > > >>>>>> state
> >> > > > >>>>>>>>>>> stores without sacrificing
> >> > > > >>>>>>>>>>> their application semantics, or else simply keep the
> >> > > > >>>> transactional
> >> > > > >>>>>>>> state
> >> > > > >>>>>>>>>>> stores disabled until we
> >> > > > >>>>>>>>>>> are able to fully implement the isolation level
> >> > > > >> configuration
> >> > > > >>>> at
> >> > > > >>>>>> either
> >> > > > >>>>>>>>>> an
> >> > > > >>>>>>>>>>> application or query level.
> >> > > > >>>>>>>>>>>
> >> > > > >>>>>>>>>>> Frankly you are the expert here and know much more
> about
> >> > > > >> the
> >> > > > >>>>>> tradeoffs
> >> > > > >>>>>>>> in
> >> > > > >>>>>>>>>>> both semantics and
> >> > > > >>>>>>>>>>> effort level of implementing one of these configs vs
> the
> >> > > > >>>> other. In
> >> > > > >>>>>> my
> >> > > > >>>>>>>>>>> opinion, either option would
> >> > > > >>>>>>>>>>> be fine and I would leave the decision of which one to
> >> > > > >> include
> >> > > > >>>> in
> >> > > > >>>>>> this
> >> > > > >>>>>>>>>> KIP
> >> > > > >>>>>>>>>>> completely up to you.
> >> > > > >>>>>>>>>>> I just don't see a way for the KIP to proceed without
> >> some
> >> > > > >>>>>> variation of
> >> > > > >>>>>>>>>> the
> >> > > > >>>>>>>>>>> above that would allow
> >> > > > >>>>>>>>>>> EOS users to opt-out of read_committed.
> >> > > > >>>>>>>>>>>
> >> > > > >>>>>>>>>>> (If it's all the same to you, I would recommend 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 that gets released.
> >> > > > >>>>>>>>>>> So personally, my choice would be to add the feature
> >> flag
> >> > > > >> and
> >> > > > >>>>>> leave it
> >> > > > >>>>>>>>>> off
> >> > > > >>>>>>>>>>> by default. If all 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
> >> > > > >>>>>>>>>>>
> >> > > > >>>>>>>>>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> >> > > > >>>>>> nick.telford@gmail.com>
> >> > > > >>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>
> >> > > > >>>>>>>>>>>> Hi Bruno,
> >> > > > >>>>>>>>>>>>
> >> > > > >>>>>>>>>>>> Agreed, I can live with that for now.
> >> > > > >>>>>>>>>>>>
> >> > > > >>>>>>>>>>>> In an effort to keep the scope of this KIP from
> >> > > > >> expanding, I'm
> >> > > > >>>>>> leaning
> >> > > > >>>>>>>>>>>> towards just providing a configurable
> >> > > > >>>>>> default.state.isolation.level
> >> > > > >>>>>>>> and
> >> > > > >>>>>>>>>>>> removing IsolationLevel from the StateStoreContext.
> >> This
> >> > > > >>>> would be
> >> > > > >>>>>>>>>>>> compatible with adding support for query-time
> >> > > > >> IsolationLevels
> >> > > > >>>> in
> >> > > > >>>>>> the
> >> > > > >>>>>>>>>>>> future, whilst providing a way for users to select an
> >> > > > >>>> isolation
> >> > > > >>>>>> level
> >> > > > >>>>>>>>>> now.
> >> > > > >>>>>>>>>>>>
> >> > > > >>>>>>>>>>>> The big problem with this, however, is that if a user
> >> > > > >> selects
> >> > > > >>>>>>>>>>>> processing.mode
> >> > > > >>>>>>>>>>>> = "exactly-once(-v2|-beta)", and
> >> > > > >>>> default.state.isolation.level =
> >> > > > >>>>>>>>>>>> "READ_UNCOMMITTED", we need to guarantee that the
> data
> >> > > > >> isn't
> >> > > > >>>>>> written
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>> disk until commit() is called, but we also need to
> >> permit
> >> > > > >> IQ
> >> > > > >>>>>> threads
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>> read from the ongoing transaction.
> >> > > > >>>>>>>>>>>>
> >> > > > >>>>>>>>>>>> A simple solution would be to (temporarily) forbid
> this
> >> > > > >>>>>> combination of
> >> > > > >>>>>>>>>>>> configuration, and have default.state.isolation.level
> >> > > > >>>>>> automatically
> >> > > > >>>>>>>>>> switch
> >> > > > >>>>>>>>>>>> to READ_COMMITTED when processing.mode is anything
> >> other
> >> > > > >> than
> >> > > > >>>>>>>>>>>> at-least-once. Do you think this would be acceptable?
> >> > > > >>>>>>>>>>>>
> >> > > > >>>>>>>>>>>> In a later KIP, we can add support for query-time
> >> > > > >> isolation
> >> > > > >>>>>> levels and
> >> > > > >>>>>>>>>>>> solve this particular problem there, which would
> relax
> >> > > > >> this
> >> > > > >>>>>>>> restriction.
> >> > > > >>>>>>>>>>>>
> >> > > > >>>>>>>>>>>> Regards,
> >> > > > >>>>>>>>>>>> Nick
> >> > > > >>>>>>>>>>>>
> >> > > > >>>>>>>>>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> >> > > > >>>> cadonna@apache.org>
> >> > > > >>>>>>>>>> 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,
> >> > > > >>>>>>>>>>>>> Bruno
> >> > > > >>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> >> > > > >>>>>>>>>>>>>> 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 <
> >> > > > >>>>>> nick.telford@gmail.com
> >> > > > >>>>>>>>>
> >> > > > >>>>>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> Hi everyone,
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> I agree that having IsolationLevel be determined
> at
> >> > > > >>>> query-time
> >> > > > >>>>>> is
> >> > > > >>>>>>>> the
> >> > > > >>>>>>>>>>>>>>> ideal design, but there are a few sticking points:
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> 1.
> >> > > > >>>>>>>>>>>>>>> There needs to be some way to communicate the
> >> > > > >>>> IsolationLevel
> >> > > > >>>>>> down
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>> RocksDBStore itself, so that the query can respect
> >> it.
> >> > > > >>>> Since
> >> > > > >>>>>> stores
> >> > > > >>>>>>>>>>>> are
> >> > > > >>>>>>>>>>>>>>> "layered" in functionality (i.e.
> ChangeLoggingStore,
> >> > > > >>>>>> MeteredStore,
> >> > > > >>>>>>>>>>>>> etc.),
> >> > > > >>>>>>>>>>>>>>> we need some way to deliver that information to
> the
> >> > > > >> bottom
> >> > > > >>>>>> layer.
> >> > > > >>>>>>>> For
> >> > > > >>>>>>>>>>>>> IQv2,
> >> > > > >>>>>>>>>>>>>>> we can use the existing State#query() method, but
> >> IQv1
> >> > > > >> has
> >> > > > >>>> no
> >> > > > >>>>>> way
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>> do
> >> > > > >>>>>>>>>>>>>>> this.
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> A simple approach, which would potentially open up
> >> > > > >> other
> >> > > > >>>>>> options,
> >> > > > >>>>>>>>>>>> would
> >> > > > >>>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> >> > > > >>>>>>>>>>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> >> > > > >>>>>>>> ReadOnlyKeyValueStore
> >> > > > >>>>>>>>>>>>> (and
> >> > > > >>>>>>>>>>>>>>> similar to ReadOnlyWindowStore,
> >> ReadOnlySessionStore,
> >> > > > >>>> etc.).
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> 2.
> >> > > > >>>>>>>>>>>>>>> As mentioned above, RocksDB WriteBatches are not
> >> > > > >>>> thread-safe,
> >> > > > >>>>>> which
> >> > > > >>>>>>>>>>>>> causes
> >> > > > >>>>>>>>>>>>>>> a problem if we want to provide READ_UNCOMMITTED
> >> > > > >>>> Iterators. I
> >> > > > >>>>>> also
> >> > > > >>>>>>>>>>>> had a
> >> > > > >>>>>>>>>>>>>>> look at RocksDB Transactions[1], but they solve a
> >> very
> >> > > > >>>>>> different
> >> > > > >>>>>>>>>>>>> problem,
> >> > > > >>>>>>>>>>>>>>> and have the same thread-safety issue.
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> One possible approach that I mentioned is chaining
> >> > > > >>>>>> WriteBatches:
> >> > > > >>>>>>>>>> every
> >> > > > >>>>>>>>>>>>>>> time a new Interactive Query is received (i.e.
> >> > > > >>>> readOnlyView,
> >> > > > >>>>>> see
> >> > > > >>>>>>>>>>>> above,
> >> > > > >>>>>>>>>>>>>>> is called) we "freeze" the existing WriteBatch,
> and
> >> > > > >> start a
> >> > > > >>>>>> new one
> >> > > > >>>>>>>>>>>> for
> >> > > > >>>>>>>>>>>>> new
> >> > > > >>>>>>>>>>>>>>> writes. The Interactive Query queries the "chain"
> of
> >> > > > >>>> previous
> >> > > > >>>>>>>>>>>>> WriteBatches
> >> > > > >>>>>>>>>>>>>>> + the underlying database; while the StreamThread
> >> > > > >> starts
> >> > > > >>>>>> writing to
> >> > > > >>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>> *new* WriteBatch. On-commit, the StreamThread
> would
> >> > > > >> write
> >> > > > >>>> *all*
> >> > > > >>>>>>>>>>>>>>> WriteBatches in the chain to the database (that
> have
> >> > > > >> not
> >> > > > >>>> yet
> >> > > > >>>>>> been
> >> > > > >>>>>>>>>>>>> written).
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> WriteBatches would be closed/freed only when they
> >> have
> >> > > > >> been
> >> > > > >>>>>> both
> >> > > > >>>>>>>>>>>>>>> committed, and all open Interactive Queries on
> them
> >> > > > >> have
> >> > > > >>>> been
> >> > > > >>>>>>>> closed.
> >> > > > >>>>>>>>>>>>> This
> >> > > > >>>>>>>>>>>>>>> would require some reference counting.
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> Obviously a drawback of this approach is the
> >> potential
> >> > > > >> for
> >> > > > >>>>>>>> increased
> >> > > > >>>>>>>>>>>>>>> memory usage: if an Interactive Query is
> long-lived,
> >> > > > >> for
> >> > > > >>>>>> example by
> >> > > > >>>>>>>>>>>>> doing a
> >> > > > >>>>>>>>>>>>>>> full scan over a large database, or even just
> >> pausing
> >> > > > >> in
> >> > > > >>>> the
> >> > > > >>>>>> middle
> >> > > > >>>>>>>>>> of
> >> > > > >>>>>>>>>>>>> an
> >> > > > >>>>>>>>>>>>>>> iteration, then the existing chain of WriteBatches
> >> > > > >> could be
> >> > > > >>>>>> kept
> >> > > > >>>>>>>>>>>> around
> >> > > > >>>>>>>>>>>>> for
> >> > > > >>>>>>>>>>>>>>> a long time, potentially forever.
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> --
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> A.
> >> > > > >>>>>>>>>>>>>>> Going off on a tangent, it looks like in addition
> to
> >> > > > >>>> supporting
> >> > > > >>>>>>>>>>>>>>> READ_COMMITTED queries, we could go further and
> >> support
> >> > > > >>>>>>>>>>>> REPEATABLE_READ
> >> > > > >>>>>>>>>>>>>>> queries (i.e. where subsequent reads to the same
> key
> >> > > > >> in the
> >> > > > >>>>>> same
> >> > > > >>>>>>>>>>>>>>> Interactive Query are guaranteed to yield the same
> >> > > > >> value)
> >> > > > >>>> by
> >> > > > >>>>>> making
> >> > > > >>>>>>>>>>>> use
> >> > > > >>>>>>>>>>>>> of
> >> > > > >>>>>>>>>>>>>>> RocksDB Snapshots[2]. These are fairly
> lightweight,
> >> so
> >> > > > >> the
> >> > > > >>>>>>>>>> performance
> >> > > > >>>>>>>>>>>>>>> impact is likely to be negligible, but they do
> >> require
> >> > > > >>>> that the
> >> > > > >>>>>>>>>>>>> Interactive
> >> > > > >>>>>>>>>>>>>>> Query session can be explicitly closed.
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> This could be achieved if we made the above
> >> > > > >> readOnlyView
> >> > > > >>>>>> interface
> >> > > > >>>>>>>>>>>> look
> >> > > > >>>>>>>>>>>>>>> more like:
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> >> > > > >>>>>>>>>>>> ReadOnlyKeyValueStore<K,
> >> > > > >>>>>>>>>>>>>>> V>, AutoCloseable {}
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> >> > > > >>>>>>>>>>>>>>>         ...
> >> > > > >>>>>>>>>>>>>>>         ReadOnlyKeyValueView<K, V>
> >> > > > >>>> readOnlyView(IsolationLevel
> >> > > > >>>>>>>>>>>>> isolationLevel);
> >> > > > >>>>>>>>>>>>>>> }
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> But this would be a breaking change, as existing
> >> IQv1
> >> > > > >>>> queries
> >> > > > >>>>>> are
> >> > > > >>>>>>>>>>>>>>> guaranteed to never call store.close(), and
> >> therefore
> >> > > > >> these
> >> > > > >>>>>> would
> >> > > > >>>>>>>>>> leak
> >> > > > >>>>>>>>>>>>>>> memory under REPEATABLE_READ.
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> B.
> >> > > > >>>>>>>>>>>>>>> One thing that's notable: MyRocks states that they
> >> > > > >> support
> >> > > > >>>>>>>>>>>>> READ_COMMITTED
> >> > > > >>>>>>>>>>>>>>> and REPEATABLE_READ, but they make no mention of
> >> > > > >>>>>>>>>>>> READ_UNCOMMITTED[3][4].
> >> > > > >>>>>>>>>>>>>>> This could be because doing so is technically
> >> > > > >>>>>> difficult/impossible
> >> > > > >>>>>>>>>>>> using
> >> > > > >>>>>>>>>>>>>>> the primitives available in RocksDB.
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> --
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> Lucas, to address your points:
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> U1.
> >> > > > >>>>>>>>>>>>>>> It's only "SHOULD" to permit alternative (i.e.
> >> > > > >> non-RocksDB)
> >> > > > >>>>>>>>>>>>>>> implementations of StateStore that do not support
> >> > > > >> atomic
> >> > > > >>>>>> writes.
> >> > > > >>>>>>>>>>>>> Obviously
> >> > > > >>>>>>>>>>>>>>> in those cases, the guarantees Kafka Streams
> >> > > > >>>> provides/expects
> >> > > > >>>>>> would
> >> > > > >>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>> relaxed. Do you think we should require all
> >> > > > >>>> implementations to
> >> > > > >>>>>>>>>> support
> >> > > > >>>>>>>>>>>>>>> atomic writes?
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> U2.
> >> > > > >>>>>>>>>>>>>>> Stores can support multiple IsolationLevels. As
> >> we've
> >> > > > >>>> discussed
> >> > > > >>>>>>>>>> above,
> >> > > > >>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>> ideal scenario would be to specify the
> >> IsolationLevel
> >> > > > >> at
> >> > > > >>>>>>>> query-time.
> >> > > > >>>>>>>>>>>>>>> Failing that, I think the second-best approach is
> to
> >> > > > >>>> define the
> >> > > > >>>>>>>>>>>>>>> IsolationLevel for *all* queries based on the
> >> > > > >>>> processing.mode,
> >> > > > >>>>>>>> which
> >> > > > >>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>> what the default
> StateStoreContext#isolationLevel()
> >> > > > >>>> achieves.
> >> > > > >>>>>> Would
> >> > > > >>>>>>>>>>>> you
> >> > > > >>>>>>>>>>>>>>> prefer an alternative?
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> While the existing implementation is equivalent to
> >> > > > >>>>>>>> READ_UNCOMMITTED,
> >> > > > >>>>>>>>>>>>> this
> >> > > > >>>>>>>>>>>>>>> can yield unexpected results/errors under EOS, if
> a
> >> > > > >>>>>> transaction is
> >> > > > >>>>>>>>>>>>> rolled
> >> > > > >>>>>>>>>>>>>>> back. While this would be a change in behaviour
> for
> >> > > > >> users,
> >> > > > >>>> it
> >> > > > >>>>>> would
> >> > > > >>>>>>>>>>>> look
> >> > > > >>>>>>>>>>>>>>> more like a bug fix than a breaking change. That
> >> said,
> >> > > > >> we
> >> > > > >>>>>> *could*
> >> > > > >>>>>>>>>> make
> >> > > > >>>>>>>>>>>>> it
> >> > > > >>>>>>>>>>>>>>> configurable, and default to the existing
> behaviour
> >> > > > >>>>>>>>>> (READ_UNCOMMITTED)
> >> > > > >>>>>>>>>>>>>>> instead of inferring it from the processing.mode?
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> N1, N2.
> >> > > > >>>>>>>>>>>>>>> These were only primitives to avoid boxing costs,
> >> but
> >> > > > >> since
> >> > > > >>>>>> this is
> >> > > > >>>>>>>>>>>> not
> >> > > > >>>>>>>>>>>>> a
> >> > > > >>>>>>>>>>>>>>> performance sensitive area, it should be fine to
> >> > > > >> change if
> >> > > > >>>>>> that's
> >> > > > >>>>>>>>>>>>> desirable.
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> N3.
> >> > > > >>>>>>>>>>>>>>> It's because the store "manages its own offsets",
> >> which
> >> > > > >>>>>> includes
> >> > > > >>>>>>>> both
> >> > > > >>>>>>>>>>>>>>> committing the offset, *and providing it* via
> >> > > > >>>>>> getCommittedOffset().
> >> > > > >>>>>>>>>>>>>>> Personally, I think "managesOffsets" conveys this
> >> best,
> >> > > > >>>> but I
> >> > > > >>>>>> don't
> >> > > > >>>>>>>>>>>> mind
> >> > > > >>>>>>>>>>>>>>> changing it if the nomenclature is unclear.
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> Sorry for the massive emails/essays!
> >> > > > >>>>>>>>>>>>>>> --
> >> > > > >>>>>>>>>>>>>>> Nick
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> 1:
> >> > > > >> https://github.com/facebook/rocksdb/wiki/Transactions
> >> > > > >>>>>>>>>>>>>>> 2:
> >> https://github.com/facebook/rocksdb/wiki/Snapshot
> >> > > > >>>>>>>>>>>>>>> 3:
> >> > > > >>>>>>>>
> >> > > > >>
> https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> >> > > > >>>>>>>>>>>>>>> 4:
> >> > > > >>>> https://mariadb.com/kb/en/myrocks-transactional-isolation/
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> >> > > > >>>>>>>>>>>>>>> <lb...@confluent.io.invalid> wrote:
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>> Hi Nick,
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>> since I last read it in April, the KIP has become
> >> much
> >> > > > >>>>>> cleaner and
> >> > > > >>>>>>>>>>>>>>>> easier to read. Great work!
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>> It feels to me the last big open point is whether
> >> we
> >> > > > >> can
> >> > > > >>>>>> implement
> >> > > > >>>>>>>>>>>>>>>> isolation level as a query parameter. I
> understand
> >> > > > >> that
> >> > > > >>>> there
> >> > > > >>>>>> are
> >> > > > >>>>>>>>>>>>>>>> implementation concerns, but as Colt says, it
> would
> >> > > > >> be a
> >> > > > >>>> great
> >> > > > >>>>>>>>>>>>>>>> addition, and would also simplify the migration
> >> path
> >> > > > >> for
> >> > > > >>>> this
> >> > > > >>>>>>>>>> change.
> >> > > > >>>>>>>>>>>>>>>> Is the implementation problem you mentioned
> caused
> >> by
> >> > > > >> the
> >> > > > >>>>>>>> WriteBatch
> >> > > > >>>>>>>>>>>>>>>> not having a notion of a snapshot, as the
> >> underlying
> >> > > > >> DB
> >> > > > >>>>>> iterator
> >> > > > >>>>>>>>>>>> does?
> >> > > > >>>>>>>>>>>>>>>> In that case, I am not sure a chain of
> WriteBatches
> >> > > > >> as you
> >> > > > >>>>>> propose
> >> > > > >>>>>>>>>>>>>>>> would fully solve the problem, but maybe I didn't
> >> dig
> >> > > > >>>> enough
> >> > > > >>>>>> into
> >> > > > >>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>> details to fully understand it.
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>> If it's not possible to implement it now, would
> it
> >> be
> >> > > > >> an
> >> > > > >>>>>> option to
> >> > > > >>>>>>>>>>>>>>>> make sure in this KIP that we do not fully close
> >> the
> >> > > > >> door
> >> > > > >>>> on
> >> > > > >>>>>>>>>>>> per-query
> >> > > > >>>>>>>>>>>>>>>> isolation levels in the interface, as it may be
> >> > > > >> possible
> >> > > > >>>> to
> >> > > > >>>>>>>>>> implement
> >> > > > >>>>>>>>>>>>>>>> the missing primitives in RocksDB or Speedb in
> the
> >> > > > >> future.
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>> Understanding:
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>> * U1) Why is it only "SHOULD" for
> changelogOffsets
> >> to
> >> > > > >> be
> >> > > > >>>>>> persisted
> >> > > > >>>>>>>>>>>>>>>> atomically with the records?
> >> > > > >>>>>>>>>>>>>>>> * U2) Don't understand the default implementation
> >> of
> >> > > > >>>>>>>>>>>> `isolationLevel`.
> >> > > > >>>>>>>>>>>>>>>> The isolation level should be a property of the
> >> > > > >> underlying
> >> > > > >>>>>> store,
> >> > > > >>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>> not be defined by the default config? Existing
> >> stores
> >> > > > >>>> probably
> >> > > > >>>>>>>> don't
> >> > > > >>>>>>>>>>>>>>>> guarantee READ_COMMITTED, so the default should
> be
> >> to
> >> > > > >>>> return
> >> > > > >>>>>>>>>>>>>>>> READ_UNCOMMITTED.
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>> Nits:
> >> > > > >>>>>>>>>>>>>>>> * N1) Could `getComittedOffset` use an
> >> `OptionalLong`
> >> > > > >>>> return
> >> > > > >>>>>> type,
> >> > > > >>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>> avoid the `null`?
> >> > > > >>>>>>>>>>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use
> an
> >> > > > >>>>>> `OptionalLong`
> >> > > > >>>>>>>>>>>>>>>> return type, to avoid the `-1`?
> >> > > > >>>>>>>>>>>>>>>> * N3) I don't understand why `managesOffsets`
> uses
> >> the
> >> > > > >>>>>> 'manage'
> >> > > > >>>>>>>>>> verb,
> >> > > > >>>>>>>>>>>>>>>> whereas all other methods use the "commits" verb.
> >> I'd
> >> > > > >>>> suggest
> >> > > > >>>>>>>>>>>>>>>> `commitsOffsets`.
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>> Either way, it feels this KIP is very close to
> the
> >> > > > >> finish
> >> > > > >>>>>> line,
> >> > > > >>>>>>>> I'm
> >> > > > >>>>>>>>>>>>>>>> looking forward to seeing this in production!
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>> Cheers,
> >> > > > >>>>>>>>>>>>>>>> Lucas
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> >> > > > >>>>>> colt@littlehorse.io
> >> > > > >>>>>>>>>
> >> > > > >>>>>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>> Making IsolationLevel a query-time constraint,
> >> > > > >> rather
> >> > > > >>>> than
> >> > > > >>>>>>>> linking
> >> > > > >>>>>>>>>>>> it
> >> > > > >>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>> the processing.guarantee.
> >> > > > >>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>> As I understand it, would this allow even a user
> >> of
> >> > > > >> EOS
> >> > > > >>>> to
> >> > > > >>>>>>>> control
> >> > > > >>>>>>>>>>>>>>>> whether
> >> > > > >>>>>>>>>>>>>>>>> reading committed or uncommitted records? If so,
> >> I am
> >> > > > >>>> highly
> >> > > > >>>>>> in
> >> > > > >>>>>>>>>>>> favor
> >> > > > >>>>>>>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>> this.
> >> > > > >>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>> I know that I was one of the early people to
> point
> >> > > > >> out
> >> > > > >>>> the
> >> > > > >>>>>>>> current
> >> > > > >>>>>>>>>>>>>>>>> shortcoming that IQ reads uncommitted records,
> but
> >> > > > >> just
> >> > > > >>>> this
> >> > > > >>>>>>>>>>>> morning I
> >> > > > >>>>>>>>>>>>>>>>> realized a pattern we use which means that (for
> >> > > > >> certain
> >> > > > >>>>>> queries)
> >> > > > >>>>>>>>>> our
> >> > > > >>>>>>>>>>>>>>>> system
> >> > > > >>>>>>>>>>>>>>>>> needs to be able to read uncommitted records,
> >> which
> >> > > > >> is
> >> > > > >>>> the
> >> > > > >>>>>>>> current
> >> > > > >>>>>>>>>>>>>>>> behavior
> >> > > > >>>>>>>>>>>>>>>>> of Kafka Streams in EOS.***
> >> > > > >>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>> If IsolationLevel being a query-time decision
> >> allows
> >> > > > >> for
> >> > > > >>>>>> this,
> >> > > > >>>>>>>> then
> >> > > > >>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>> would be amazing. I would also vote that the
> >> default
> >> > > > >>>> behavior
> >> > > > >>>>>>>>>> should
> >> > > > >>>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>> for
> >> > > > >>>>>>>>>>>>>>>>> reading uncommitted records, because it is
> totally
> >> > > > >>>> possible
> >> > > > >>>>>> for a
> >> > > > >>>>>>>>>>>>> valid
> >> > > > >>>>>>>>>>>>>>>>> application to depend on that behavior, and
> >> breaking
> >> > > > >> it
> >> > > > >>>> in a
> >> > > > >>>>>>>> minor
> >> > > > >>>>>>>>>>>>>>>> release
> >> > > > >>>>>>>>>>>>>>>>> might be a bit strong.
> >> > > > >>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>> *** (Note, for the curious reader....) Our
> >> > > > >> use-case/query
> >> > > > >>>>>> pattern
> >> > > > >>>>>>>>>>>> is a
> >> > > > >>>>>>>>>>>>>>>> bit
> >> > > > >>>>>>>>>>>>>>>>> complex, but reading "uncommitted" records is
> >> > > > >> actually
> >> > > > >>>> safe
> >> > > > >>>>>> in
> >> > > > >>>>>>>> our
> >> > > > >>>>>>>>>>>>> case
> >> > > > >>>>>>>>>>>>>>>>> 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 soon as the processor
> >> processes
> >> > > > >> the
> >> > > > >>>>>> result.
> >> > > > >>>>>>>> If
> >> > > > >>>>>>>>>>>>> they
> >> > > > >>>>>>>>>>>>>>>>> can't read uncommitted records, they may
> receive a
> >> > > > >> "201 -
> >> > > > >>>>>>>> Created"
> >> > > > >>>>>>>>>>>>>>>>> response, immediately followed by a "404 - 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 <
> >> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> >> > > > >>>>>>>>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>> Addendum:
> >> > > > >>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>> I think we would also face the same problem
> with
> >> the
> >> > > > >>>>>> approach
> >> > > > >>>>>>>> John
> >> > > > >>>>>>>>>>>>>>>> outlined
> >> > > > >>>>>>>>>>>>>>>>>> earlier (using the record cache as a
> transaction
> >> > > > >> buffer
> >> > > > >>>> and
> >> > > > >>>>>>>>>>>> flushing
> >> > > > >>>>>>>>>>>>>>>> it
> >> > > > >>>>>>>>>>>>>>>>>> straight to SST files). This is because the
> >> record
> >> > > > >> cache
> >> > > > >>>>>> (the
> >> > > > >>>>>>>>>>>>>>>> ThreadCache
> >> > > > >>>>>>>>>>>>>>>>>> class) is not thread-safe, so every commit
> would
> >> > > > >>>> invalidate
> >> > > > >>>>>> open
> >> > > > >>>>>>>>>> IQ
> >> > > > >>>>>>>>>>>>>>>>>> Iterators in the same way that RocksDB
> >> WriteBatches
> >> > > > >> do.
> >> > > > >>>>>>>>>>>>>>>>>> --
> >> > > > >>>>>>>>>>>>>>>>>> Nick
> >> > > > >>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> >> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> >> > > > >>>>>>>>>>>>>>>>>> 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.
> >> > > > >>>>>>>>>>>>>>>>>>> 2. Making IsolationLevel a query-time
> >> constraint,
> >> > > > >>>> rather
> >> > > > >>>>>> than
> >> > > > >>>>>>>>>>>>>>>> linking it
> >> > > > >>>>>>>>>>>>>>>>>>> to the processing.guarantee.
> >> > > > >>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>> There's a wrinkle that makes this a challenge:
> >> > > > >>>> Interactive
> >> > > > >>>>>>>>>> Queries
> >> > > > >>>>>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>>> open an Iterator, when using transactions and
> >> > > > >>>>>> READ_UNCOMMITTED.
> >> > > > >>>>>>>>>>>>>>>>>>> The problem is that under READ_UNCOMMITTED,
> >> queries
> >> > > > >>>> need
> >> > > > >>>>>> to be
> >> > > > >>>>>>>>>>>> able
> >> > > > >>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>> read records from the currently uncommitted
> >> > > > >> transaction
> >> > > > >>>>>> buffer
> >> > > > >>>>>>>>>>>>>>>>>>> (WriteBatch). This includes for Iterators,
> which
> >> > > > >> should
> >> > > > >>>>>> iterate
> >> > > > >>>>>>>>>>>>>>>> both the
> >> > > > >>>>>>>>>>>>>>>>>>> transaction buffer and underlying database
> >> (using
> >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch#iteratorWithBase()).
> >> > > > >>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>> The issue is that when the StreamThread
> >> commits, it
> >> > > > >>>> writes
> >> > > > >>>>>> the
> >> > > > >>>>>>>>>>>>>>>> current
> >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch to RocksDB *and then clears the
> >> > > > >> WriteBatch*.
> >> > > > >>>>>>>> Clearing
> >> > > > >>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch while an Interactive Query holds an
> >> open
> >> > > > >>>>>> Iterator on
> >> > > > >>>>>>>>>> it
> >> > > > >>>>>>>>>>>>>>>> will
> >> > > > >>>>>>>>>>>>>>>>>>> invalidate the Iterator. Worse, it turns out
> >> that
> >> > > > >>>> Iterators
> >> > > > >>>>>>>> over
> >> > > > >>>>>>>>>> a
> >> > > > >>>>>>>>>>>>>>>>>>> WriteBatch become invalidated not just when
> the
> >> > > > >>>> WriteBatch
> >> > > > >>>>>> is
> >> > > > >>>>>>>>>>>>>>>> cleared,
> >> > > > >>>>>>>>>>>>>>>>>> but
> >> > > > >>>>>>>>>>>>>>>>>>> also when the Iterators' current key receives
> a
> >> new
> >> > > > >>>> write.
> >> > > > >>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>> Now that I'm writing this, I remember that
> this
> >> is
> >> > > > >> the
> >> > > > >>>>>> major
> >> > > > >>>>>>>>>>>> reason
> >> > > > >>>>>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>> I
> >> > > > >>>>>>>>>>>>>>>>>>> switched the original design from having a
> >> > > > >> query-time
> >> > > > >>>>>>>>>>>>>>>> IsolationLevel to
> >> > > > >>>>>>>>>>>>>>>>>>> having the IsolationLevel linked to the
> >> > > > >>>> transactionality
> >> > > > >>>>>> of the
> >> > > > >>>>>>>>>>>>>>>> stores
> >> > > > >>>>>>>>>>>>>>>>>>> themselves.
> >> > > > >>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>> It *might* be possible to resolve this, by
> >> having a
> >> > > > >>>>>> "chain" of
> >> > > > >>>>>>>>>>>>>>>>>>> WriteBatches, with the StreamThread switching
> >> to a
> >> > > > >> new
> >> > > > >>>>>>>> WriteBatch
> >> > > > >>>>>>>>>>>>>>>>>> whenever
> >> > > > >>>>>>>>>>>>>>>>>>> a new Interactive Query attempts to read from
> >> the
> >> > > > >>>>>> database, but
> >> > > > >>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>> could
> >> > > > >>>>>>>>>>>>>>>>>>> cause some performance problems/memory
> pressure
> >> > > > >> when
> >> > > > >>>>>> subjected
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>> a
> >> > > > >>>>>>>>>>>>>>>> high
> >> > > > >>>>>>>>>>>>>>>>>>> Interactive Query load. It would also reduce
> the
> >> > > > >>>>>> efficiency of
> >> > > > >>>>>>>>>>>>>>>>>> WriteBatches
> >> > > > >>>>>>>>>>>>>>>>>>> on-commit, as we'd have to write N
> WriteBatches,
> >> > > > >> where
> >> > > > >>>> N
> >> > > > >>>>>> is the
> >> > > > >>>>>>>>>>>>>>>> number of
> >> > > > >>>>>>>>>>>>>>>>>>> Interactive Queries since the last commit.
> >> > > > >>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>> I realise this is getting into the weeds of
> the
> >> > > > >>>>>> implementation,
> >> > > > >>>>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>> you'd
> >> > > > >>>>>>>>>>>>>>>>>>> rather we focus on the API for now, but I
> think
> >> > > > >> it's
> >> > > > >>>>>> important
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>> consider
> >> > > > >>>>>>>>>>>>>>>>>>> how to implement the desired API, in case we
> >> come
> >> > > > >> up
> >> > > > >>>> with
> >> > > > >>>>>> an
> >> > > > >>>>>>>> API
> >> > > > >>>>>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>>> cannot be implemented efficiently, or even at
> >> all!
> >> > > > >>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>> Thoughts?
> >> > > > >>>>>>>>>>>>>>>>>>> --
> >> > > > >>>>>>>>>>>>>>>>>>> Nick
> >> > > > >>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> >> > > > >>>>>>>> cadonna@apache.org
> >> > > > >>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> >> > > > >>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>> 6.
> >> > > > >>>>>>>>>>>>>>>>>>>> Of course, you are right! My bad!
> >> > > > >>>>>>>>>>>>>>>>>>>> Wiping out the state in the downgrading case
> is
> >> > > > >> fine.
> >> > > > >>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>> 3a.
> >> > > > >>>>>>>>>>>>>>>>>>>> Focus on the public facing changes for the
> >> KIP. We
> >> > > > >>>> will
> >> > > > >>>>>> manage
> >> > > > >>>>>>>>>> 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.
> >> > > > >>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>> Best,
> >> > > > >>>>>>>>>>>>>>>>>>>> Bruno
> >> > > > >>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> Thanks for getting back to me!
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> 2.
> >> > > > >>>>>>>>>>>>>>>>>>>>> The fact that implementations can always
> track
> >> > > > >>>> estimated
> >> > > > >>>>>>>> memory
> >> > > > >>>>>>>>>>>>>>>> usage
> >> > > > >>>>>>>>>>>>>>>>>> in
> >> > > > >>>>>>>>>>>>>>>>>>>>> the wrapper is a good point. I can remove -1
> >> as
> >> > > > >> an
> >> > > > >>>>>> option,
> >> > > > >>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>> I'll
> >> > > > >>>>>>>>>>>>>>>>>>>> clarify
> >> > > > >>>>>>>>>>>>>>>>>>>>> the JavaDoc that 0 is not just for
> >> > > > >> non-transactional
> >> > > > >>>>>> stores,
> >> > > > >>>>>>>>>>>>>>>> which is
> >> > > > >>>>>>>>>>>>>>>>>>>>> currently misleading.
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> 6.
> >> > > > >>>>>>>>>>>>>>>>>>>>> The problem with catching the exception in
> the
> >> > > > >>>> downgrade
> >> > > > >>>>>>>>>> process
> >> > > > >>>>>>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>>>>> would require new code in the Kafka version
> >> being
> >> > > > >>>>>> downgraded
> >> > > > >>>>>>>>>> to.
> >> > > > >>>>>>>>>>>>>>>> Since
> >> > > > >>>>>>>>>>>>>>>>>>>>> users could conceivably downgrade to almost
> >> *any*
> >> > > > >>>> older
> >> > > > >>>>>>>> version
> >> > > > >>>>>>>>>>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>>> Kafka
> >> > > > >>>>>>>>>>>>>>>>>>>>> Streams, I'm not sure how we could add that
> >> code?
> >> > > > >>>>>>>>>>>>>>>>>>>>> The only way I can think of doing it would
> be
> >> to
> >> > > > >>>> provide
> >> > > > >>>>>> a
> >> > > > >>>>>>>>>>>>>>>> dedicated
> >> > > > >>>>>>>>>>>>>>>>>>>>> downgrade tool, that goes through every
> local
> >> > > > >> store
> >> > > > >>>> and
> >> > > > >>>>>>>> removes
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>> offsets column families. But that seems like
> >> an
> >> > > > >>>>>> unnecessary
> >> > > > >>>>>>>>>>>>>>>> amount of
> >> > > > >>>>>>>>>>>>>>>>>>>> extra
> >> > > > >>>>>>>>>>>>>>>>>>>>> code to maintain just to handle a somewhat
> >> niche
> >> > > > >>>>>> situation,
> >> > > > >>>>>>>>>> when
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>> alternative (automatically wipe and restore
> >> > > > >> stores)
> >> > > > >>>>>> should be
> >> > > > >>>>>>>>>>>>>>>>>>>> acceptable.
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes
> you've
> >> > > > >>>> requested.
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> 3a.
> >> > > > >>>>>>>>>>>>>>>>>>>>> I agree that IsolationLevel makes more sense
> >> at
> >> > > > >>>>>> query-time,
> >> > > > >>>>>>>> and
> >> > > > >>>>>>>>>>>> I
> >> > > > >>>>>>>>>>>>>>>>>>>> actually
> >> > > > >>>>>>>>>>>>>>>>>>>>> initially attempted to place the
> >> IsolationLevel
> >> > > > >> at
> >> > > > >>>>>>>> query-time,
> >> > > > >>>>>>>>>>>>>>>> but I
> >> > > > >>>>>>>>>>>>>>>>>> ran
> >> > > > >>>>>>>>>>>>>>>>>>>>> into some problems:
> >> > > > >>>>>>>>>>>>>>>>>>>>> - The key issue is that, under ALOS we're
> not
> >> > > > >> staging
> >> > > > >>>>>> writes
> >> > > > >>>>>>>> in
> >> > > > >>>>>>>>>>>>>>>>>>>>> transactions, so can't perform writes at the
> >> > > > >>>>>> READ_COMMITTED
> >> > > > >>>>>>>>>>>>>>>> isolation
> >> > > > >>>>>>>>>>>>>>>>>>>>> level. However, this may be addressed if we
> >> > > > >> decide to
> >> > > > >>>>>>>> *always*
> >> > > > >>>>>>>>>>>>>>>> use
> >> > > > >>>>>>>>>>>>>>>>>>>>> transactions as discussed under 3b.
> >> > > > >>>>>>>>>>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> >> > > > >>>> implementations. I
> >> > > > >>>>>>>>>> remember
> >> > > > >>>>>>>>>>>>>>>>>> having
> >> > > > >>>>>>>>>>>>>>>>>>>>> some difficulty understanding the IQv1
> >> internals,
> >> > > > >>>> which
> >> > > > >>>>>> made
> >> > > > >>>>>>>> it
> >> > > > >>>>>>>>>>>>>>>>>>>> difficult
> >> > > > >>>>>>>>>>>>>>>>>>>>> to determine what needed to be changed.
> >> However,
> >> > > > >> I
> >> > > > >>>>>> *think*
> >> > > > >>>>>>>> this
> >> > > > >>>>>>>>>>>>>>>> can be
> >> > > > >>>>>>>>>>>>>>>>>>>>> addressed for both implementations by
> wrapping
> >> > > > >> the
> >> > > > >>>>>>>> RocksDBStore
> >> > > > >>>>>>>>>>>>>>>> in an
> >> > > > >>>>>>>>>>>>>>>>>>>>> IsolationLevel-dependent wrapper, that
> >> overrides
> >> > > > >> read
> >> > > > >>>>>> methods
> >> > > > >>>>>>>>>>>>>>>> (get,
> >> > > > >>>>>>>>>>>>>>>>>>>> etc.)
> >> > > > >>>>>>>>>>>>>>>>>>>>> to either read directly from the database or
> >> > > > >> from the
> >> > > > >>>>>> ongoing
> >> > > > >>>>>>>>>>>>>>>>>>>> transaction.
> >> > > > >>>>>>>>>>>>>>>>>>>>> But IQv1 might still be difficult.
> >> > > > >>>>>>>>>>>>>>>>>>>>> - If IsolationLevel becomes a query
> >> constraint,
> >> > > > >> then
> >> > > > >>>> all
> >> > > > >>>>>>>> other
> >> > > > >>>>>>>>>>>>>>>>>>>> StateStores
> >> > > > >>>>>>>>>>>>>>>>>>>>> will need to respect it, including the
> >> in-memory
> >> > > > >>>> stores.
> >> > > > >>>>>> This
> >> > > > >>>>>>>>>>>>>>>> would
> >> > > > >>>>>>>>>>>>>>>>>>>> require
> >> > > > >>>>>>>>>>>>>>>>>>>>> us to adapt in-memory stores to stage their
> >> > > > >> writes so
> >> > > > >>>>>> they
> >> > > > >>>>>>>> can
> >> > > > >>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>>>> isolated
> >> > > > >>>>>>>>>>>>>>>>>>>>> from READ_COMMITTTED queries. It would also
> >> > > > >> become an
> >> > > > >>>>>>>> important
> >> > > > >>>>>>>>>>>>>>>>>>>>> consideration for third-party stores on
> >> upgrade,
> >> > > > >> as
> >> > > > >>>>>> without
> >> > > > >>>>>>>>>>>>>>>> changes,
> >> > > > >>>>>>>>>>>>>>>>>>>> they
> >> > > > >>>>>>>>>>>>>>>>>>>>> would not support READ_COMMITTED queries
> >> > > > >> correctly.
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> Ultimately, I may need some help making the
> >> > > > >> necessary
> >> > > > >>>>>> change
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>> IQv1
> >> > > > >>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>> support this, but I don't think it's
> >> > > > >> fundamentally
> >> > > > >>>>>>>> impossible,
> >> > > > >>>>>>>>>>>>>>>> if we
> >> > > > >>>>>>>>>>>>>>>>>>>> want
> >> > > > >>>>>>>>>>>>>>>>>>>>> to pursue this route.
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> 3b.
> >> > > > >>>>>>>>>>>>>>>>>>>>> The main reason I chose to keep ALOS
> >> > > > >> un-transactional
> >> > > > >>>>>> was to
> >> > > > >>>>>>>>>>>>>>>> minimize
> >> > > > >>>>>>>>>>>>>>>>>>>>> behavioural change for most users (I believe
> >> most
> >> > > > >>>> Streams
> >> > > > >>>>>>>> users
> >> > > > >>>>>>>>>>>>>>>> use
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>> default configuration, which is ALOS). That
> >> said,
> >> > > > >>>> it's
> >> > > > >>>>>> clear
> >> > > > >>>>>>>>>>>>>>>> that if
> >> > > > >>>>>>>>>>>>>>>>>>>> ALOS
> >> > > > >>>>>>>>>>>>>>>>>>>>> also used transactional stores, the only
> >> change
> >> > > > >> in
> >> > > > >>>>>> behaviour
> >> > > > >>>>>>>>>>>>>>>> would be
> >> > > > >>>>>>>>>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>>>>> it would become *more correct*, which could
> be
> >> > > > >>>>>> considered a
> >> > > > >>>>>>>>>> "bug
> >> > > > >>>>>>>>>>>>>>>> fix"
> >> > > > >>>>>>>>>>>>>>>>>> by
> >> > > > >>>>>>>>>>>>>>>>>>>>> users, rather than a change they need to
> >> handle.
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> I believe that performance using
> transactions
> >> > > > >> (aka.
> >> > > > >>>>>> RocksDB
> >> > > > >>>>>>>>>>>>>>>>>>>> WriteBatches)
> >> > > > >>>>>>>>>>>>>>>>>>>>> should actually be *better* than the
> >> un-batched
> >> > > > >>>>>> write-path
> >> > > > >>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>>> currently used[1]. The only "performance"
> >> > > > >>>> consideration
> >> > > > >>>>>> will
> >> > > > >>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>> increased memory usage that transactions
> >> require.
> >> > > > >>>> Given
> >> > > > >>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>> mitigations
> >> > > > >>>>>>>>>>>>>>>>>>>> for
> >> > > > >>>>>>>>>>>>>>>>>>>>> this memory that we have in place, I would
> >> expect
> >> > > > >>>> that
> >> > > > >>>>>> this
> >> > > > >>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>> not a
> >> > > > >>>>>>>>>>>>>>>>>>>>> problem for most users.
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> If we're happy to do so, we can make ALOS
> also
> >> > > > >> use
> >> > > > >>>>>>>>>> transactions.
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> Regards,
> >> > > > >>>>>>>>>>>>>>>>>>>>> Nick
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> Link 1:
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>
> >> > > > >>>>
> >> > > > >>
> >> > >
> >> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno
> Cadonna <
> >> > > > >>>>>>>>>>>> cadonna@apache.org
> >> > > > >>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> Thanks for the updates and sorry for the
> >> delay
> >> > > > >> on my
> >> > > > >>>>>> side!
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> 1.
> >> > > > >>>>>>>>>>>>>>>>>>>>>> Making the default implementation for
> >> flush() a
> >> > > > >>>> no-op
> >> > > > >>>>>> sounds
> >> > > > >>>>>>>>>>>>>>>> good to
> >> > > > >>>>>>>>>>>>>>>>>>>> me.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> 2.
> >> > > > >>>>>>>>>>>>>>>>>>>>>> I think what was bugging me here is that a
> >> > > > >>>> third-party
> >> > > > >>>>>> state
> >> > > > >>>>>>>>>>>>>>>> store
> >> > > > >>>>>>>>>>>>>>>>>>>> needs
> >> > > > >>>>>>>>>>>>>>>>>>>>>> to implement the state store interface.
> That
> >> > > > >> means
> >> > > > >>>> they
> >> > > > >>>>>> need
> >> > > > >>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>> implement a wrapper around the actual state
> >> > > > >> store
> >> > > > >>>> as we
> >> > > > >>>>>> do
> >> > > > >>>>>>>> for
> >> > > > >>>>>>>>>>>>>>>>>> RocksDB
> >> > > > >>>>>>>>>>>>>>>>>>>>>> with RocksDBStore. So, a third-party state
> >> > > > >> store can
> >> > > > >>>>>> always
> >> > > > >>>>>>>>>>>>>>>> estimate
> >> > > > >>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> uncommitted bytes, if it wants, because the
> >> > > > >> wrapper
> >> > > > >>>> can
> >> > > > >>>>>>>> record
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>> added
> >> > > > >>>>>>>>>>>>>>>>>>>>>> bytes.
> >> > > > >>>>>>>>>>>>>>>>>>>>>> One case I can think of where returning -1
> >> makes
> >> > > > >>>> sense
> >> > > > >>>>>> is
> >> > > > >>>>>>>> when
> >> > > > >>>>>>>>>>>>>>>>>> Streams
> >> > > > >>>>>>>>>>>>>>>>>>>>>> does not need to estimate the size of the
> >> write
> >> > > > >>>> batch
> >> > > > >>>>>> and
> >> > > > >>>>>>>>>>>>>>>> trigger
> >> > > > >>>>>>>>>>>>>>>>>>>>>> extraordinary commits, because the
> >> third-party
> >> > > > >> state
> >> > > > >>>>>> store
> >> > > > >>>>>>>>>>>>>>>> takes care
> >> > > > >>>>>>>>>>>>>>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>>>>>>> memory. But in that case the method could
> >> also
> >> > > > >> just
> >> > > > >>>>>> return
> >> > > > >>>>>>>> 0.
> >> > > > >>>>>>>>>>>>>>>> Even
> >> > > > >>>>>>>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>>>>>> case would be better solved with a method
> >> that
> >> > > > >>>> returns
> >> > > > >>>>>>>> whether
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>> state
> >> > > > >>>>>>>>>>>>>>>>>>>>>> store manages itself the memory used for
> >> > > > >> uncommitted
> >> > > > >>>>>> bytes
> >> > > > >>>>>>>> or
> >> > > > >>>>>>>>>>>>>>>> not.
> >> > > > >>>>>>>>>>>>>>>>>>>>>> Said that, I am fine with keeping the -1
> >> return
> >> > > > >>>> value,
> >> > > > >>>>>> I was
> >> > > > >>>>>>>>>>>>>>>> just
> >> > > > >>>>>>>>>>>>>>>>>>>>>> wondering when and if it will be used.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> Regarding returning 0 for transactional
> state
> >> > > > >> stores
> >> > > > >>>>>> when
> >> > > > >>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>> batch
> >> > > > >>>>>>>>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>>>> empty, I was just wondering because you
> >> > > > >> explicitly
> >> > > > >>>>>> stated
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> "or {@code 0} if this StateStore does not
> >> > > > >> support
> >> > > > >>>>>>>>>>>> transactions."
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> So it seemed to me returning 0 could only
> >> > > > >> happen for
> >> > > > >>>>>>>>>>>>>>>>>> non-transactional
> >> > > > >>>>>>>>>>>>>>>>>>>>>> state stores.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> 3.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> a) What do you think if we move the
> isolation
> >> > > > >> level
> >> > > > >>>> to
> >> > > > >>>>>> IQ
> >> > > > >>>>>>>> (v1
> >> > > > >>>>>>>>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>> v2)?
> >> > > > >>>>>>>>>>>>>>>>>>>>>> In the end this is the only component that
> >> > > > >> really
> >> > > > >>>> needs
> >> > > > >>>>>> to
> >> > > > >>>>>>>>>>>>>>>> specify
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> isolation level. It is similar to the Kafka
> >> > > > >> consumer
> >> > > > >>>>>> that
> >> > > > >>>>>>>> can
> >> > > > >>>>>>>>>>>>>>>> choose
> >> > > > >>>>>>>>>>>>>>>>>>>>>> with what isolation level to read the input
> >> > > > >> topic.
> >> > > > >>>>>>>>>>>>>>>>>>>>>> For IQv1 the isolation level should go into
> >> > > > >>>>>>>>>>>>>>>> StoreQueryParameters. For
> >> > > > >>>>>>>>>>>>>>>>>>>>>> IQv2, I would add it to the Query
> interface.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> b) Point a) raises the question what should
> >> > > > >> happen
> >> > > > >>>>>> during
> >> > > > >>>>>>>>>>>>>>>>>> at-least-once
> >> > > > >>>>>>>>>>>>>>>>>>>>>> processing when the state store does not
> use
> >> > > > >>>>>> transactions?
> >> > > > >>>>>>>>>> John
> >> > > > >>>>>>>>>>>>>>>> in
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> past proposed to also use transactions on
> >> state
> >> > > > >>>> stores
> >> > > > >>>>>> for
> >> > > > >>>>>>>>>>>>>>>>>>>>>> at-least-once. I like that idea, because it
> >> > > > >> avoids
> >> > > > >>>>>>>> aggregating
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>> same
> >> > > > >>>>>>>>>>>>>>>>>>>>>> records over and over again in the case of
> a
> >> > > > >>>> failure. We
> >> > > > >>>>>>>> had a
> >> > > > >>>>>>>>>>>>>>>> case
> >> > > > >>>>>>>>>>>>>>>>>> in
> >> > > > >>>>>>>>>>>>>>>>>>>>>> the past where a Streams applications in
> >> > > > >>>> at-least-once
> >> > > > >>>>>> mode
> >> > > > >>>>>>>>>> was
> >> > > > >>>>>>>>>>>>>>>>>> failing
> >> > > > >>>>>>>>>>>>>>>>>>>>>> continuously for some reasons I do not
> >> remember
> >> > > > >>>> before
> >> > > > >>>>>>>>>>>>>>>> committing the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> offsets. After each failover, the app
> >> aggregated
> >> > > > >>>> again
> >> > > > >>>>>> and
> >> > > > >>>>>>>>>>>>>>>> again the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> same records. Of course the aggregate
> >> increased
> >> > > > >> to
> >> > > > >>>> very
> >> > > > >>>>>>>> wrong
> >> > > > >>>>>>>>>>>>>>>> values
> >> > > > >>>>>>>>>>>>>>>>>>>>>> just because of the failover. With
> >> transactions
> >> > > > >> on
> >> > > > >>>> the
> >> > > > >>>>>> state
> >> > > > >>>>>>>>>>>>>>>> stores
> >> > > > >>>>>>>>>>>>>>>>>> we
> >> > > > >>>>>>>>>>>>>>>>>>>>>> could have avoided this. The app would have
> >> > > > >> output
> >> > > > >>>> the
> >> > > > >>>>>> same
> >> > > > >>>>>>>>>>>>>>>> aggregate
> >> > > > >>>>>>>>>>>>>>>>>>>>>> multiple times (i.e., after each failover)
> >> but
> >> > > > >> at
> >> > > > >>>> least
> >> > > > >>>>>> the
> >> > > > >>>>>>>>>>>>>>>> value of
> >> > > > >>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> aggregate would not depend on the number of
> >> > > > >>>> failovers.
> >> > > > >>>>>>>>>>>>>>>> Outputting the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> same aggregate multiple times would be
> >> incorrect
> >> > > > >>>> under
> >> > > > >>>>>>>>>>>>>>>> exactly-once
> >> > > > >>>>>>>>>>>>>>>>>> but
> >> > > > >>>>>>>>>>>>>>>>>>>>>> it is OK for at-least-once.
> >> > > > >>>>>>>>>>>>>>>>>>>>>> If it makes sense to add a config to turn
> on
> >> > > > >> and off
> >> > > > >>>>>>>>>>>>>>>> transactions on
> >> > > > >>>>>>>>>>>>>>>>>>>>>> state stores under at-least-once or just
> use
> >> > > > >>>>>> transactions in
> >> > > > >>>>>>>>>>>>>>>> any case
> >> > > > >>>>>>>>>>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>>>> a question we should also discuss in this
> >> KIP.
> >> > > > >> It
> >> > > > >>>>>> depends a
> >> > > > >>>>>>>>>> bit
> >> > > > >>>>>>>>>>>>>>>> on
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I
> >> would
> >> > > > >>>> add a
> >> > > > >>>>>>>> config.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> 4.
> >> > > > >>>>>>>>>>>>>>>>>>>>>> Your points are all valid. I tend to say to
> >> > > > >> keep the
> >> > > > >>>>>> metrics
> >> > > > >>>>>>>>>>>>>>>> around
> >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() until we remove flush() completely
> >> from
> >> > > > >> the
> >> > > > >>>>>>>> interface.
> >> > > > >>>>>>>>>>>>>>>> Calls
> >> > > > >>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() might still exist since existing
> >> > > > >> processors
> >> > > > >>>>>> might
> >> > > > >>>>>>>>>> still
> >> > > > >>>>>>>>>>>>>>>> call
> >> > > > >>>>>>>>>>>>>>>>>>>>>> flush() explicitly as you mentioned in 1).
> >> For
> >> > > > >>>> sure, we
> >> > > > >>>>>> need
> >> > > > >>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>> document
> >> > > > >>>>>>>>>>>>>>>>>>>>>> how the metrics change due to the
> >> transactions
> >> > > > >> in
> >> > > > >>>> the
> >> > > > >>>>>>>> upgrade
> >> > > > >>>>>>>>>>>>>>>> notes.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> 5.
> >> > > > >>>>>>>>>>>>>>>>>>>>>> I see. Then you should describe how the
> >> > > > >> .position
> >> > > > >>>> files
> >> > > > >>>>>> are
> >> > > > >>>>>>>>>>>>>>>> handled
> >> > > > >>>>>>>>>>>>>>>>>> in
> >> > > > >>>>>>>>>>>>>>>>>>>>>> a dedicated section of the KIP or
> incorporate
> >> > > > >> the
> >> > > > >>>>>>>> description
> >> > > > >>>>>>>>>>>>>>>> in the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> "Atomic Checkpointing" section instead of
> >> only
> >> > > > >>>>>> mentioning it
> >> > > > >>>>>>>>>> in
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> "Compatibility, 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:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>
> >> > > > >>>>>>
> >> > > > >>>>
> >> > > > >>
> >> > >
> >> >
> >>
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> It is a bit brittle, but it works.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> Best,
> >> > > > >>>>>>>>>>>>>>>>>>>>>> Bruno
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for taking the time to review the
> >> KIP.
> >> > > > >> I'm
> >> > > > >>>> back
> >> > > > >>>>>> from
> >> > > > >>>>>>>>>>>>>>>> leave
> >> > > > >>>>>>>>>>>>>>>>>> now
> >> > > > >>>>>>>>>>>>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> intend to move this forwards as quickly
> as I
> >> > > > >> can.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Addressing your points:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Because flush() is part of the StateStore
> >> API,
> >> > > > >> it's
> >> > > > >>>>>> exposed
> >> > > > >>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>> custom
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors, which might be making calls to
> >> > > > >> flush().
> >> > > > >>>>>> This
> >> > > > >>>>>>>> was
> >> > > > >>>>>>>>>>>>>>>>>> actually
> >> > > > >>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> case in a few integration tests.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> To maintain as much compatibility as
> >> possible,
> >> > > > >> I'd
> >> > > > >>>>>> prefer
> >> > > > >>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>> make
> >> > > > >>>>>>>>>>>>>>>>>>>>>> this
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> an UnsupportedOperationException, as it
> will
> >> > > > >> cause
> >> > > > >>>>>>>> previously
> >> > > > >>>>>>>>>>>>>>>>>> working
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors to start throwing exceptions at
> >> > > > >> runtime.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that it doesn't make sense for it
> to
> >> > > > >> proxy
> >> > > > >>>>>>>> commit(),
> >> > > > >>>>>>>>>>>>>>>> though,
> >> > > > >>>>>>>>>>>>>>>>>>>> as
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> that would cause it to violate the
> >> "StateStores
> >> > > > >>>> commit
> >> > > > >>>>>> only
> >> > > > >>>>>>>>>>>>>>>> when the
> >> > > > >>>>>>>>>>>>>>>>>>>> Task
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> commits" rule.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Instead, I think we should make this a
> >> no-op.
> >> > > > >> That
> >> > > > >>>> way,
> >> > > > >>>>>>>>>>>>>>>> existing
> >> > > > >>>>>>>>>>>>>>>>>> user
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors will continue to work
> as-before,
> >> > > > >> without
> >> > > > >>>>>>>> violation
> >> > > > >>>>>>>>>>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>>> store
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> consistency that would be caused by
> >> premature
> >> > > > >>>>>> flush/commit
> >> > > > >>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>>>>> StateStore
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> data to disk.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> What do you think?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> As stated in the JavaDoc, when a
> StateStore
> >> > > > >>>>>> implementation
> >> > > > >>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, but is unable to estimate
> the
> >> > > > >>>>>> uncommitted
> >> > > > >>>>>>>>>>>> memory
> >> > > > >>>>>>>>>>>>>>>>>> usage,
> >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> method will return -1.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> The intention here is to permit
> third-party
> >> > > > >>>>>> implementations
> >> > > > >>>>>>>>>>>>>>>> that may
> >> > > > >>>>>>>>>>>>>>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> able to estimate memory usage.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been
> >> > > > >> written to
> >> > > > >>>> the
> >> > > > >>>>>>>> store
> >> > > > >>>>>>>>>>>>>>>> yet. I
> >> > > > >>>>>>>>>>>>>>>>>>>>>> thought
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> that was implied by "This method will
> >> return an
> >> > > > >>>>>>>> approximation
> >> > > > >>>>>>>>>>>>>>>> of the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> memory
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> would be freed by the next call to {@link
> >> > > > >>>>>> #commit(Map)}"
> >> > > > >>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>> "@return
> >> > > > >>>>>>>>>>>>>>>>>>>> The
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> approximate size of all records awaiting
> >> {@link
> >> > > > >>>>>>>>>>>> #commit(Map)}",
> >> > > > >>>>>>>>>>>>>>>>>>>> however,
> >> > > > >>>>>>>>>>>>>>>>>>>>>> I
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if
> you
> >> > > > >> think
> >> > > > >>>> this
> >> > > > >>>>>> is
> >> > > > >>>>>>>>>>>>>>>> unclear?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> I realise this is probably the most
> >> contentious
> >> > > > >>>> point
> >> > > > >>>>>> in my
> >> > > > >>>>>>>>>>>>>>>> design,
> >> > > > >>>>>>>>>>>>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>>>>>> I'm
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> open to changing it if I'm unable to
> >> convince
> >> > > > >> you
> >> > > > >>>> of
> >> > > > >>>>>> the
> >> > > > >>>>>>>>>>>>>>>> benefits.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Nevertheless, here's my argument:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are
> >> directly
> >> > > > >>>> provided
> >> > > > >>>>>>>>>>>>>>>> StateStores
> >> > > > >>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> query, and it may be important for users
> to
> >> > > > >>>>>>>> programmatically
> >> > > > >>>>>>>>>>>>>>>> know
> >> > > > >>>>>>>>>>>>>>>>>>>> which
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> mode the StateStore is operating under. If
> >> we
> >> > > > >>>> simply
> >> > > > >>>>>>>> provide
> >> > > > >>>>>>>>>>>> an
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout
> the
> >> > > > >>>> internal
> >> > > > >>>>>>>> streams
> >> > > > >>>>>>>>>>>>>>>>>>>> engine), or
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> similar, then users will need to
> understand
> >> the
> >> > > > >>>>>> operation
> >> > > > >>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>>>>>> consequences
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> of each available processing mode and how
> it
> >> > > > >>>> pertains
> >> > > > >>>>>> to
> >> > > > >>>>>>>>>> their
> >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Interactive Query users aren't the only
> >> people
> >> > > > >> that
> >> > > > >>>>>> care
> >> > > > >>>>>>>>>> about
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> processing.mode/IsolationLevel of a
> >> StateStore:
> >> > > > >>>>>>>> implementers
> >> > > > >>>>>>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>>> custom
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStores also need to understand the
> >> > > > >> behaviour
> >> > > > >>>>>> expected
> >> > > > >>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>> their
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> implementation. KIP-892 introduces some
> >> > > > >> assumptions
> >> > > > >>>>>> into
> >> > > > >>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>> Streams
> >> > > > >>>>>>>>>>>>>>>>>>>>>> Engine
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> about how StateStores operate under each
> >> > > > >> processing
> >> > > > >>>>>> mode,
> >> > > > >>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>> it's
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> important that custom implementations
> >> adhere to
> >> > > > >>>> those
> >> > > > >>>>>>>>>>>>>>>> assumptions in
> >> > > > >>>>>>>>>>>>>>>>>>>>>> order
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> to maintain the consistency guarantees.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> IsolationLevels provide a high-level
> >> contract
> >> > > > >> on
> >> > > > >>>> the
> >> > > > >>>>>>>>>> behaviour
> >> > > > >>>>>>>>>>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStore: a user knows that under
> >> > > > >> READ_COMMITTED,
> >> > > > >>>>>> they
> >> > > > >>>>>>>> will
> >> > > > >>>>>>>>>>>>>>>> see
> >> > > > >>>>>>>>>>>>>>>>>>>> writes
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> only after the Task has committed, and
> under
> >> > > > >>>>>>>> READ_UNCOMMITTED
> >> > > > >>>>>>>>>>>>>>>> they
> >> > > > >>>>>>>>>>>>>>>>>>>> will
> >> > > > >>>>>>>>>>>>>>>>>>>>>> see
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> writes immediately. No understanding of
> the
> >> > > > >>>> details of
> >> > > > >>>>>> each
> >> > > > >>>>>>>>>>>>>>>>>>>>>> processing.mode
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> is required, either for IQ users or
> >> StateStore
> >> > > > >>>>>>>> implementers.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> An argument can be made that these
> >> contractual
> >> > > > >>>>>> guarantees
> >> > > > >>>>>>>> can
> >> > > > >>>>>>>>>>>>>>>> simply
> >> > > > >>>>>>>>>>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> documented for the processing.mode (i.e.
> >> that
> >> > > > >>>>>> exactly-once
> >> > > > >>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED
> >> and
> >> > > > >>>>>>>> at-least-once
> >> > > > >>>>>>>>>>>>>>>> behaves
> >> > > > >>>>>>>>>>>>>>>>>>>> like
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several
> >> small
> >> > > > >>>> issues
> >> > > > >>>>>> with
> >> > > > >>>>>>>>>>>>>>>> this I'd
> >> > > > >>>>>>>>>>>>>>>>>>>>>> prefer
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> to avoid:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - Where would we document these
> >> > > > >> contracts,
> >> > > > >>>> in
> >> > > > >>>>>> a way
> >> > > > >>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>> difficult
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          for users/implementers to
> >> miss/ignore?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It's not clear to users that
> the
> >> > > > >>>> processing
> >> > > > >>>>>> mode
> >> > > > >>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>> communicating
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          an expectation of read isolation,
> >> > > > >> unless
> >> > > > >>>> they
> >> > > > >>>>>> read
> >> > > > >>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> documentation. Users
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          rarely consult documentation
> unless
> >> > > > >> they
> >> > > > >>>> feel
> >> > > > >>>>>> they
> >> > > > >>>>>>>>>> need
> >> > > > >>>>>>>>>>>>>>>> to, so
> >> > > > >>>>>>>>>>>>>>>>>>>> it's
> >> > > > >>>>>>>>>>>>>>>>>>>>>> likely
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          this detail would get missed by
> >> many
> >> > > > >> users.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It tightly couples processing
> >> modes
> >> > > > >> to
> >> > > > >>>> read
> >> > > > >>>>>>>>>> isolation.
> >> > > > >>>>>>>>>>>>>>>> Adding
> >> > > > >>>>>>>>>>>>>>>>>>>> new
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          processing modes, or changing the
> >> read
> >> > > > >>>>>> isolation of
> >> > > > >>>>>>>>>>>>>>>> existing
> >> > > > >>>>>>>>>>>>>>>>>>>>>> processing
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          modes would be
> >> difficult/impossible.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Ultimately, the cost of introducing
> >> > > > >>>> IsolationLevels is
> >> > > > >>>>>>>> just a
> >> > > > >>>>>>>>>>>>>>>> single
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> method, since we re-use the existing
> >> > > > >> IsolationLevel
> >> > > > >>>>>> enum
> >> > > > >>>>>>>> from
> >> > > > >>>>>>>>>>>>>>>> Kafka.
> >> > > > >>>>>>>>>>>>>>>>>>>> This
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> gives us a clear place to document the
> >> > > > >> contractual
> >> > > > >>>>>>>> guarantees
> >> > > > >>>>>>>>>>>>>>>>>> expected
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> of/provided by StateStores, that is
> >> accessible
> >> > > > >>>> both by
> >> > > > >>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>> StateStore
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> itself, and by IQ users.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> (Writing this I've just realised that the
> >> > > > >>>> StateStore
> >> > > > >>>>>> and IQ
> >> > > > >>>>>>>>>>>>>>>> APIs
> >> > > > >>>>>>>>>>>>>>>>>>>> actually
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> don't provide access to StateStoreContext
> >> that
> >> > > > >> IQ
> >> > > > >>>> users
> >> > > > >>>>>>>> would
> >> > > > >>>>>>>>>>>>>>>> have
> >> > > > >>>>>>>>>>>>>>>>>>>> direct
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> access to... Perhaps StateStore should
> >> expose
> >> > > > >>>>>>>>>> isolationLevel()
> >> > > > >>>>>>>>>>>>>>>>>> itself
> >> > > > >>>>>>>>>>>>>>>>>>>>>> too?)
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> 4.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the
> >> metrics
> >> > > > >>>> in-place
> >> > > > >>>>>>>>>>>>>>>> either, as
> >> > > > >>>>>>>>>>>>>>>>>>>> it's a
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> backwards incompatible change. My concern
> is
> >> > > > >> that,
> >> > > > >>>> if
> >> > > > >>>>>> we
> >> > > > >>>>>>>>>> leave
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> existing
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics in place, they will be
> >> > > > >> confusing to
> >> > > > >>>>>> users.
> >> > > > >>>>>>>>>>>>>>>> Right
> >> > > > >>>>>>>>>>>>>>>>>> now,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics record explicit flushes to
> >> > > > >> disk,
> >> > > > >>>> but
> >> > > > >>>>>> under
> >> > > > >>>>>>>>>>>>>>>> KIP-892,
> >> > > > >>>>>>>>>>>>>>>>>>>> even
> >> > > > >>>>>>>>>>>>>>>>>>>>>> a
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() will not explicitly flush data to
> >> > > > >> disk -
> >> > > > >>>>>> RocksDB
> >> > > > >>>>>>>>>> will
> >> > > > >>>>>>>>>>>>>>>>>> decide
> >> > > > >>>>>>>>>>>>>>>>>>>> on
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> when to flush memtables to disk itself.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> If we keep the existing "flush" metrics,
> >> we'd
> >> > > > >> have
> >> > > > >>>> two
> >> > > > >>>>>>>>>>>> options,
> >> > > > >>>>>>>>>>>>>>>>>> which
> >> > > > >>>>>>>>>>>>>>>>>>>>>> both
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> seem pretty bad to me:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          1. Have them record calls to
> >> commit(),
> >> > > > >>>> which
> >> > > > >>>>>> would
> >> > > > >>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>>>> misleading, as
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          data is no longer explicitly
> >> "flushed"
> >> > > > >> to
> >> > > > >>>> disk
> >> > > > >>>>>> by
> >> > > > >>>>>>>> this
> >> > > > >>>>>>>>>>>>>>>> call.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          2. Have them record nothing at
> all,
> >> > > > >> which
> >> > > > >>>> is
> >> > > > >>>>>>>>>> equivalent
> >> > > > >>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>> removing
> >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          metrics, except that users will
> see
> >> > the
> >> > > > >>>> metric
> >> > > > >>>>>>>> still
> >> > > > >>>>>>>>>>>>>>>> exists and
> >> > > > >>>>>>>>>>>>>>>>>>>> so
> >> > > > >>>>>>>>>>>>>>>>>>>>>> assume
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          that the metric is correct, and
> >> that
> >> > > > >>>> there's a
> >> > > > >>>>>>>> problem
> >> > > > >>>>>>>>>>>>>>>> with
> >> > > > >>>>>>>>>>>>>>>>>> their
> >> > > > >>>>>>>>>>>>>>>>>>>>>> system
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>          when there isn't.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that removing them is also a bad
> >> > > > >> solution,
> >> > > > >>>> and
> >> > > > >>>>>> I'd
> >> > > > >>>>>>>>>>>>>>>> like some
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> guidance on the best path forward here.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> 5.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Position files are updated on every write
> >> to a
> >> > > > >>>>>> StateStore.
> >> > > > >>>>>>>>>>>>>>>> Since our
> >> > > > >>>>>>>>>>>>>>>>>>>>>> writes
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> are now buffered until commit(), we can't
> >> > > > >> update
> >> > > > >>>> the
> >> > > > >>>>>>>> Position
> >> > > > >>>>>>>>>>>>>>>> file
> >> > > > >>>>>>>>>>>>>>>>>>>> until
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() has been called, otherwise it
> >> would be
> >> > > > >>>>>>>> inconsistent
> >> > > > >>>>>>>>>>>>>>>> with
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> data
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> in the event of a rollback. Consequently,
> we
> >> > > > >> need
> >> > > > >>>> to
> >> > > > >>>>>> manage
> >> > > > >>>>>>>>>>>>>>>> these
> >> > > > >>>>>>>>>>>>>>>>>>>> offsets
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> the same way we manage the checkpoint
> >> offsets,
> >> > > > >> and
> >> > > > >>>>>> ensure
> >> > > > >>>>>>>>>>>>>>>> they're
> >> > > > >>>>>>>>>>>>>>>>>> only
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> written on commit().
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> 6.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet
> >> what
> >> > > > >>>> tests to
> >> > > > >>>>>>>>>> write.
> >> > > > >>>>>>>>>>>>>>>> How
> >> > > > >>>>>>>>>>>>>>>>>>>>>> explicit
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> do we need to be here in the KIP?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is
> >> designed
> >> > > > >> to be
> >> > > > >>>>>>>> seamless,
> >> > > > >>>>>>>>>>>>>>>> and we
> >> > > > >>>>>>>>>>>>>>>>>>>>>> should
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> definitely add some tests around that.
> >> > > > >> Downgrade,
> >> > > > >>>> it
> >> > > > >>>>>>>>>>>>>>>> transpires,
> >> > > > >>>>>>>>>>>>>>>>>> isn't
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> currently possible, as the extra column
> >> family
> >> > > > >> for
> >> > > > >>>>>> offset
> >> > > > >>>>>>>>>>>>>>>> storage is
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> incompatible with the pre-KIP-892
> >> > > > >> implementation:
> >> > > > >>>> when
> >> > > > >>>>>> you
> >> > > > >>>>>>>>>>>>>>>> open a
> >> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> database, you must open all available
> column
> >> > > > >>>> families
> >> > > > >>>>>> or
> >> > > > >>>>>>>>>>>>>>>> receive an
> >> > > > >>>>>>>>>>>>>>>>>>>>>> error.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> What currently happens on downgrade is
> that
> >> it
> >> > > > >>>>>> attempts to
> >> > > > >>>>>>>>>>>>>>>> open the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> store,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> throws an error about the offsets column
> >> > > > >> family not
> >> > > > >>>>>> being
> >> > > > >>>>>>>>>>>>>>>> opened,
> >> > > > >>>>>>>>>>>>>>>>>>>> which
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task.
> >> Given
> >> > > > >> that
> >> > > > >>>>>>>>>> downgrades
> >> > > > >>>>>>>>>>>>>>>>>> should
> >> > > > >>>>>>>>>>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> uncommon, I think this is acceptable
> >> > > > >> behaviour, as
> >> > > > >>>> the
> >> > > > >>>>>>>>>>>>>>>> end-state is
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> consistent, even if it results in an
> >> > > > >> undesirable
> >> > > > >>>> state
> >> > > > >>>>>>>>>>>> restore.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Should I document the upgrade/downgrade
> >> > > > >> behaviour
> >> > > > >>>>>>>> explicitly
> >> > > > >>>>>>>>>>>>>>>> in the
> >> > > > >>>>>>>>>>>>>>>>>>>> KIP?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> --
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> Nick
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno
> Cadonna
> >> <
> >> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> >> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Nick!
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates!
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> StateStore#commit(Collections.emptyMap())?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Since calls to flush() will not exist
> >> anymore
> >> > > > >>>> after
> >> > > > >>>>>> this
> >> > > > >>>>>>>> KIP
> >> > > > >>>>>>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> released, I would rather throw an
> >> unsupported
> >> > > > >>>>>> operation
> >> > > > >>>>>>>>>>>>>>>> exception
> >> > > > >>>>>>>>>>>>>>>>>> by
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> default.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> When would a state store return -1 from
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> StateStore#approximateNumUncommittedBytes()
> >> > > > >> while
> >> > > > >>>>>> being
> >> > > > >>>>>>>>>>>>>>>>>>>> transactional?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Wouldn't
> >> > > > >>>> StateStore#approximateNumUncommittedBytes()
> >> > > > >>>>>> also
> >> > > > >>>>>>>>>>>>>>>> return 0
> >> > > > >>>>>>>>>>>>>>>>>> if
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the state store is transactional but
> >> nothing
> >> > > > >> has
> >> > > > >>>> been
> >> > > > >>>>>>>>>> written
> >> > > > >>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> state store yet?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Sorry for bringing this up again. Does
> this
> >> > > > >> KIP
> >> > > > >>>> really
> >> > > > >>>>>>>> need
> >> > > > >>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>> introduce
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> >> > > > >>>> StateStoreContext
> >> > > > >>>>>> has
> >> > > > >>>>>>>>>>>>>>>> already
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> appConfigs() which basically exposes the
> >> same
> >> > > > >>>>>> information,
> >> > > > >>>>>>>>>>>>>>>> i.e., if
> >> > > > >>>>>>>>>>>>>>>>>>>> EOS
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> is enabled or not.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> In one of your previous e-mails you
> wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> "My idea was to try to keep the
> StateStore
> >> > > > >>>> interface
> >> > > > >>>>>> as
> >> > > > >>>>>>>>>>>>>>>> loosely
> >> > > > >>>>>>>>>>>>>>>>>>>> coupled
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to
> >> give
> >> > > > >>>>>> implementers
> >> > > > >>>>>>>>>>>> more
> >> > > > >>>>>>>>>>>>>>>>>>>> freedom,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and reduce the amount of internal
> knowledge
> >> > > > >>>> required."
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> While I understand the intent, I doubt
> >> that it
> >> > > > >>>>>> decreases
> >> > > > >>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>> coupling of
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> a StateStore interface and the Streams
> >> engine.
> >> > > > >>>>>>>>>> READ_COMMITTED
> >> > > > >>>>>>>>>>>>>>>> only
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> applies to IQ but not to reads by
> >> processors.
> >> > > > >>>> Thus,
> >> > > > >>>>>>>>>>>>>>>> implementers
> >> > > > >>>>>>>>>>>>>>>>>>>> need to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand how Streams accesses the state
> >> > > > >> stores.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> I would like to hear what others think
> >> about
> >> > > > >> this.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> 4.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Great exposing new metrics for
> >> transactional
> >> > > > >> state
> >> > > > >>>>>> stores!
> >> > > > >>>>>>>>>>>>>>>>>> However, I
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> would prefer to add new metrics and
> >> deprecate
> >> > > > >> (in
> >> > > > >>>> the
> >> > > > >>>>>>>> docs)
> >> > > > >>>>>>>>>>>>>>>> the 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 also need to adapt/add
> >> > > > >> integration
> >> > > > >>>>>> tests
> >> > > > >>>>>>>>>>>>>>>> besides
> >> > > > >>>>>>>>>>>>>>>>>> unit
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> tests. Additionally, we probably need
> >> > > > >> integration
> >> > > > >>>> or
> >> > > > >>>>>>>> system
> >> > > > >>>>>>>>>>>>>>>> tests
> >> > > > >>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> verify that upgrades and downgrades
> between
> >> > > > >>>>>> transactional
> >> > > > >>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> non-transactional state stores work as
> >> > > > >> expected.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Best,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Bruno
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> One more thing: I noted John's
> suggestion
> >> in
> >> > > > >> the
> >> > > > >>>> KIP,
> >> > > > >>>>>>>> under
> >> > > > >>>>>>>>>>>>>>>>>>>> "Rejected
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> Alternatives". I still think it's an
> idea
> >> > > > >> worth
> >> > > > >>>>>> pursuing,
> >> > > > >>>>>>>>>>>>>>>> but I
> >> > > > >>>>>>>>>>>>>>>>>>>> believe
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> that it's out of the scope of this KIP,
> >> > > > >> because
> >> > > > >>>> it
> >> > > > >>>>>>>> solves a
> >> > > > >>>>>>>>>>>>>>>>>>>> different
> >> > > > >>>>>>>>>>>>>>>>>>>>>> set
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> of problems to this KIP, and the scope
> of
> >> > > > >> this
> >> > > > >>>> one
> >> > > > >>>>>> has
> >> > > > >>>>>>>>>>>>>>>> already
> >> > > > >>>>>>>>>>>>>>>>>> grown
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> quite
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> large!
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick
> >> Telford <
> >> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP (
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>
> >> > > > >>>>>>
> >> > > > >>>>
> >> > > > >>
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> )
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with the latest changes; mostly
> bringing
> >> > > > >> back
> >> > > > >>>>>> "Atomic
> >> > > > >>>>>>>>>>>>>>>>>>>> Checkpointing"
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> (for
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> what feels like the 10th time!). I
> think
> >> > > > >> the one
> >> > > > >>>>>> thing
> >> > > > >>>>>>>>>>>>>>>> missing is
> >> > > > >>>>>>>>>>>>>>>>>>>> some
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> changes to metrics (notably the store
> >> > > > >> "flush"
> >> > > > >>>>>> metrics
> >> > > > >>>>>>>> will
> >> > > > >>>>>>>>>>>>>>>> need
> >> > > > >>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> renamed to "commit").
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> The reason I brought back Atomic
> >> > > > >> Checkpointing
> >> > > > >>>> was
> >> > > > >>>>>> to
> >> > > > >>>>>>>>>>>>>>>> decouple
> >> > > > >>>>>>>>>>>>>>>>>>>> store
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flush
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from store commit. This is important,
> >> > > > >> because
> >> > > > >>>> with
> >> > > > >>>>>>>>>>>>>>>> Transactional
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> StateStores, we now need to call
> "flush"
> >> on
> >> > > > >>>> *every*
> >> > > > >>>>>> Task
> >> > > > >>>>>>>>>>>>>>>> commit,
> >> > > > >>>>>>>>>>>>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> just when the StateStore is closing,
> >> > > > >> otherwise
> >> > > > >>>> our
> >> > > > >>>>>>>>>>>>>>>> transaction
> >> > > > >>>>>>>>>>>>>>>>>>>> buffer
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> will
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> never be written and persisted, instead
> >> > > > >> growing
> >> > > > >>>>>>>> unbounded!
> >> > > > >>>>>>>>>>>> I
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> experimented
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with some simple solutions, like
> forcing
> >> a
> >> > > > >> store
> >> > > > >>>>>> flush
> >> > > > >>>>>>>>>>>>>>>> whenever
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed
> >> its
> >> > > > >>>>>> configured
> >> > > > >>>>>>>>>>>>>>>> size, but
> >> > > > >>>>>>>>>>>>>>>>>>>> this
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> brittle: it prevented the transaction
> >> buffer
> >> > > > >>>> from
> >> > > > >>>>>> being
> >> > > > >>>>>>>>>>>>>>>>>> configured
> >> > > > >>>>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> unbounded, and it still would have
> >> required
> >> > > > >>>> explicit
> >> > > > >>>>>>>>>>>>>>>> flushes of
> >> > > > >>>>>>>>>>>>>>>>>>>>>> RocksDB,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yielding sub-optimal performance and
> >> memory
> >> > > > >>>>>> utilization.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the
> >> > > > >> "right"
> >> > > > >>>> way
> >> > > > >>>>>> to
> >> > > > >>>>>>>>>>>>>>>> resolve
> >> > > > >>>>>>>>>>>>>>>>>> this
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog
> >> > > > >> offsets
> >> > > > >>>> that
> >> > > > >>>>>>>>>>>>>>>> correspond
> >> > > > >>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> most
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> recently written records are always
> >> > > > >> atomically
> >> > > > >>>>>> written
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (by writing them to the same
> transaction
> >> > > > >>>> buffer),
> >> > > > >>>>>> we can
> >> > > > >>>>>>>>>>>>>>>> avoid
> >> > > > >>>>>>>>>>>>>>>>>>>>>> forcibly
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk,
> >> > > > >> letting
> >> > > > >>>>>> RocksDB
> >> > > > >>>>>>>>>>>>>>>> flush
> >> > > > >>>>>>>>>>>>>>>>>> them
> >> > > > >>>>>>>>>>>>>>>>>>>>>> only
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when necessary, without losing any of
> our
> >> > > > >>>>>> consistency
> >> > > > >>>>>>>>>>>>>>>> guarantees.
> >> > > > >>>>>>>>>>>>>>>>>>>> See
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> updated KIP for more info.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I have fully implemented these changes,
> >> > > > >>>> although I'm
> >> > > > >>>>>>>> still
> >> > > > >>>>>>>>>>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>>>>>>>> entirely
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> happy with the implementation for
> >> segmented
> >> > > > >>>>>> StateStores,
> >> > > > >>>>>>>>>> so
> >> > > > >>>>>>>>>>>>>>>> I
> >> > > > >>>>>>>>>>>>>>>>>> plan
> >> > > > >>>>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests
> >> > > > >> pass. If
> >> > > > >>>>>> you'd
> >> > > > >>>>>>>> like
> >> > > > >>>>>>>>>>>>>>>> to try
> >> > > > >>>>>>>>>>>>>>>>>>>> out
> >> > > > >>>>>>>>>>>>>>>>>>>>>> or
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> review this highly experimental and
> >> > > > >> incomplete
> >> > > > >>>>>> branch,
> >> > > > >>>>>>>>>> it's
> >> > > > >>>>>>>>>>>>>>>>>>>> available
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> here:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> >> > > > >>>>>>>> .
> >> > > > >>>>>>>>>>>>>>>> Note:
> >> > > > >>>>>>>>>>>>>>>>>>>> it's
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> built
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a
> >> stable
> >> > > > >> base
> >> > > > >>>> to
> >> > > > >>>>>> build
> >> > > > >>>>>>>>>>>>>>>> and test
> >> > > > >>>>>>>>>>>>>>>>>>>> it
> >> > > > >>>>>>>>>>>>>>>>>>>>>> on,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples
> >> > > > >> comparisons
> >> > > > >>>> in a
> >> > > > >>>>>>>> live
> >> > > > >>>>>>>>>>>>>>>>>>>>>> environment. I
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> plan to rebase it against trunk once
> it's
> >> > > > >> nearer
> >> > > > >>>>>>>>>> completion
> >> > > > >>>>>>>>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>> has
> >> > > > >>>>>>>>>>>>>>>>>>>>>> been
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> proven on our main application.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I would really appreciate help in
> >> reviewing
> >> > > > >> and
> >> > > > >>>>>> testing:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and
> >> Window)
> >> > > > >>>> stores
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Global stores
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> As I do not currently use either of
> >> these,
> >> > > > >> so my
> >> > > > >>>>>> primary
> >> > > > >>>>>>>>>>>>>>>> test
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> environment
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> doesn't test these areas.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting
> next
> >> > > > >> week
> >> > > > >>>> for
> >> > > > >>>>>> a few
> >> > > > >>>>>>>>>>>>>>>> weeks,
> >> > > > >>>>>>>>>>>>>>>>>> so
> >> > > > >>>>>>>>>>>>>>>>>>>>>> will
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> not have time to move this forward
> until
> >> > > > >> late
> >> > > > >>>>>> August.
> >> > > > >>>>>>>> That
> >> > > > >>>>>>>>>>>>>>>> said,
> >> > > > >>>>>>>>>>>>>>>>>>>> your
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I
> >> just
> >> > > > >>>> won't be
> >> > > > >>>>>>>> able
> >> > > > >>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>> respond
> >> > > > >>>>>>>>>>>>>>>>>>>>>> as
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> quickly as usual.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick
> >> Telford <
> >> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, that's correct, although the
> impact
> >> > > > >> on IQ
> >> > > > >>>> is
> >> > > > >>>>>> not
> >> > > > >>>>>>>>>>>>>>>> something
> >> > > > >>>>>>>>>>>>>>>>>> I
> >> > > > >>>>>>>>>>>>>>>>>>>> had
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> considered.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the
> state
> >> > > > >> store
> >> > > > >>>>>> from the
> >> > > > >>>>>>>>>>>>>>>>>>>> transaction
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and
> >> writing
> >> > > > >> the
> >> > > > >>>>>>>> checkpoint
> >> > > > >>>>>>>>>>>>>>>> (thus,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount
> >> of
> >> > > > >> data
> >> > > > >>>>>> and/or
> >> > > > >>>>>>>>>>>>>>>> number of
> >> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you
> >> > > > >> suggesting
> >> > > > >>>>>> that we
> >> > > > >>>>>>>>>>>>>>>> add an
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> additional
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> config for the max number of commit
> >> > > > >> intervals
> >> > > > >>>>>> between
> >> > > > >>>>>>>>>>>>>>>>>> checkpoints?
> >> > > > >>>>>>>>>>>>>>>>>>>>>> That
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when
> >> the
> >> > > > >>>>>> transaction
> >> > > > >>>>>>>>>>>>>>>> buffers
> >> > > > >>>>>>>>>>>>>>>>>> are
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> nearly
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number
> of
> >> > > > >> commit
> >> > > > >>>>>>>> intervals
> >> > > > >>>>>>>>>>>>>>>> have
> >> > > > >>>>>>>>>>>>>>>>>>>>>> elapsed,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> whichever comes first?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> That certainly seems reasonable,
> >> although
> >> > > > >> this
> >> > > > >>>>>>>> re-ignites
> >> > > > >>>>>>>>>>>>>>>> an
> >> > > > >>>>>>>>>>>>>>>>>>>> earlier
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> debate about whether a config should
> be
> >> > > > >>>> measured in
> >> > > > >>>>>>>>>>>>>>>> "number of
> >> > > > >>>>>>>>>>>>>>>>>>>> commit
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> intervals", instead of just an
> absolute
> >> > > > >> time.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is
> the
> >> > > > >> reason
> >> > > > >>>> I
> >> > > > >>>>>> was
> >> > > > >>>>>>>>>>>>>>>> pursuing
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> Atomic
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable
> >> > > > >> flush
> >> > > > >>>> from
> >> > > > >>>>>>>>>>>>>>>> checkpointing,
> >> > > > >>>>>>>>>>>>>>>>>>>>>> which
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every
> >> > > > >> commit
> >> > > > >>>>>> without
> >> > > > >>>>>>>> any
> >> > > > >>>>>>>>>>>>>>>>>>>> performance
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is
> >> definitely
> >> > > > >> the
> >> > > > >>>>>> "best"
> >> > > > >>>>>>>>>>>>>>>> solution,
> >> > > > >>>>>>>>>>>>>>>>>>>> but
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm not
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sure if this is enough to bring it
> back
> >> > > > >> into
> >> > > > >>>> this
> >> > > > >>>>>> KIP.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm currently working on moving all
> the
> >> > > > >>>>>> transactional
> >> > > > >>>>>>>>>>>> logic
> >> > > > >>>>>>>>>>>>>>>>>>>> directly
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> into
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away
> >> with
> >> > > > >> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore#newTransaction
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> method, and reduces the number of new
> >> > > > >> classes
> >> > > > >>>>>>>> introduced,
> >> > > > >>>>>>>>>>>>>>>>>>>>>> significantly
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reducing the complexity. If it works,
> >> and
> >> > > > >> the
> >> > > > >>>>>>>> complexity
> >> > > > >>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>>>> drastically
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reduced, I may try bringing back
> Atomic
> >> > > > >>>> Checkpoints
> >> > > > >>>>>>>> into
> >> > > > >>>>>>>>>>>>>>>> this
> >> > > > >>>>>>>>>>>>>>>>>> KIP.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno
> >> Cadonna
> >> > > > >> <
> >> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> >> > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very
> >> interesting!
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> As far as I understand, you want to
> >> > > > >> atomically
> >> > > > >>>>>> update
> >> > > > >>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>> state
> >> > > > >>>>>>>>>>>>>>>>>>>>>> store
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush
> the
> >> > > > >>>> memtable
> >> > > > >>>>>> of a
> >> > > > >>>>>>>>>>>> state
> >> > > > >>>>>>>>>>>>>>>>>> store
> >> > > > >>>>>>>>>>>>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write the checkpoint not after the
> >> commit
> >> > > > >> time
> >> > > > >>>>>> elapsed
> >> > > > >>>>>>>>>>>> but
> >> > > > >>>>>>>>>>>>>>>>>> after
> >> > > > >>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reached a size
> that
> >> > > > >> would
> >> > > > >>>> lead
> >> > > > >>>>>> to
> >> > > > >>>>>>>>>>>>>>>> exceeding
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> statestore.transaction.buffer.max.bytes
> >> > > > >>>> before the
> >> > > > >>>>>>>> next
> >> > > > >>>>>>>>>>>>>>>> commit
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> interval
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> ends.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction
> would
> >> > > > >> commit
> >> > > > >>>>>> every
> >> > > > >>>>>>>>>>>>>>>> commit
> >> > > > >>>>>>>>>>>>>>>>>>>> interval
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store will only be
> atomically
> >> > > > >>>> updated
> >> > > > >>>>>>>> roughly
> >> > > > >>>>>>>>>>>>>>>> every
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> statestore.transaction.buffer.max.bytes of
> >> > > > >>>> data.
> >> > > > >>>>>> Also
> >> > > > >>>>>>>> IQ
> >> > > > >>>>>>>>>>>>>>>> would
> >> > > > >>>>>>>>>>>>>>>>>>>> then
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> only
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see new data roughly every
> >> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> After a failure the state store needs
> >> to
> >> > > > >>>> restore
> >> > > > >>>>>> up to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> statestore.transaction.buffer.max.bytes.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Is this correct?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the
> >> state
> >> > > > >> store
> >> > > > >>>>>> from
> >> > > > >>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>> transaction
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and
> >> writing
> >> > > > >> the
> >> > > > >>>>>>>> checkpoint
> >> > > > >>>>>>>>>>>>>>>> (thus,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount
> >> of
> >> > > > >> data
> >> > > > >>>>>> and/or
> >> > > > >>>>>>>>>>>>>>>> number of
> >> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would
> have
> >> > > > >> the
> >> > > > >>>> same
> >> > > > >>>>>> delay
> >> > > > >>>>>>>>>>>> for
> >> > > > >>>>>>>>>>>>>>>>>>>> records
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ
> >> because
> >> > > > >> both
> >> > > > >>>>>> would
> >> > > > >>>>>>>>>>>>>>>> appear
> >> > > > >>>>>>>>>>>>>>>>>> when
> >> > > > >>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka transaction is committed.
> >> However,
> >> > > > >>>> after a
> >> > > > >>>>>>>> failure
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>> state
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> still needs to restore up to
> >> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it might restore data that is already
> >> in
> >> > > > >> the
> >> > > > >>>> state
> >> > > > >>>>>>>> store
> >> > > > >>>>>>>>>>>>>>>>>> because
> >> > > > >>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last
> stable
> >> > > > >> offset
> >> > > > >>>>>> (i.e.
> >> > > > >>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>> last
> >> > > > >>>>>>>>>>>>>>>>>>>>>> committed
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset) of the changelog topics.
> >> Restoring
> >> > > > >>>> data
> >> > > > >>>>>> that
> >> > > > >>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>> already
> >> > > > >>>>>>>>>>>>>>>>>>>> in
> >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos
> >> should
> >> > > > >> not
> >> > > > >>>>>> violated.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> This solution needs at least one new
> >> > > > >> config to
> >> > > > >>>>>> specify
> >> > > > >>>>>>>>>>>>>>>> when a
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> should be written.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> A small correction to your previous
> >> e-mail
> >> > > > >>>> that
> >> > > > >>>>>> does
> >> > > > >>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>> change
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> anything
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> you said: Under alos the default
> commit
> >> > > > >>>> interval
> >> > > > >>>>>> is 30
> >> > > > >>>>>>>>>>>>>>>> seconds,
> >> > > > >>>>>>>>>>>>>>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> five
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> seconds.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford
> wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've begun performance testing my
> >> branch
> >> > > > >> on
> >> > > > >>>> our
> >> > > > >>>>>>>> staging
> >> > > > >>>>>>>>>>>>>>>>>>>>>> environment,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
> >> > > > >>>> non-trivial
> >> > > > >>>>>>>>>>>>>>>> application.
> >> > > > >>>>>>>>>>>>>>>>>> I'm
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> already
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing the same increased flush
> >> rate
> >> > > > >> that
> >> > > > >>>> we
> >> > > > >>>>>> saw
> >> > > > >>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>> last
> >> > > > >>>>>>>>>>>>>>>>>>>> time
> >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> attempted to use a version of this
> >> KIP,
> >> > > > >> but
> >> > > > >>>> this
> >> > > > >>>>>>>> time,
> >> > > > >>>>>>>>>> I
> >> > > > >>>>>>>>>>>>>>>>>> think I
> >> > > > >>>>>>>>>>>>>>>>>>>>>> know
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> why.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit,
> >> > > > >> which is
> >> > > > >>>>>> called
> >> > > > >>>>>>>> at
> >> > > > >>>>>>>>>>>>>>>> the end
> >> > > > >>>>>>>>>>>>>>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Task
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit process, has the following
> >> > > > >> behaviour:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under ALOS: checkpoint
> >> the
> >> > > > >> state
> >> > > > >>>>>> stores.
> >> > > > >>>>>>>>>> This
> >> > > > >>>>>>>>>>>>>>>>>> includes
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            flushing memtables in
> >> RocksDB.
> >> > > > >>>> This is
> >> > > > >>>>>>>>>>>> acceptable
> >> > > > >>>>>>>>>>>>>>>>>>>> because the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> default
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit.interval.ms is 5
> >> > > > >> seconds,
> >> > > > >>>> so
> >> > > > >>>>>>>> forcibly
> >> > > > >>>>>>>>>>>>>>>> flushing
> >> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> every 5
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            seconds is acceptable for
> >> most
> >> > > > >>>>>>>> applications.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under EOS:
> checkpointing
> >> is
> >> > > > >> not
> >> > > > >>>> done,
> >> > > > >>>>>>>>>> *unless*
> >> > > > >>>>>>>>>>>>>>>> it's
> >> > > > >>>>>>>>>>>>>>>>>>>> being
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> forced, due
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to e.g. the Task closing
> or
> >> > > > >> being
> >> > > > >>>>>> revoked.
> >> > > > >>>>>>>>>> This
> >> > > > >>>>>>>>>>>>>>>> means
> >> > > > >>>>>>>>>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>>>>>> under
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> normal
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing conditions,
> the
> >> > > > >> state
> >> > > > >>>> stores
> >> > > > >>>>>>>> will
> >> > > > >>>>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpointed,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            not have memtables
> flushed
> >> at
> >> > > > >> all ,
> >> > > > >>>>>> unless
> >> > > > >>>>>>>>>>>> RocksDB
> >> > > > >>>>>>>>>>>>>>>>>>>> decides to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush them on
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            its own. Checkpointing
> >> stores
> >> > > > >> and
> >> > > > >>>>>>>>>> force-flushing
> >> > > > >>>>>>>>>>>>>>>> their
> >> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            done when a Task is being
> >> > > > >> closed.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to
> checkpoint
> >> > > > >>>> stores on
> >> > > > >>>>>> at
> >> > > > >>>>>>>>>>>> least
> >> > > > >>>>>>>>>>>>>>>>>> *some*
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> normal
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the
> >> > > > >> RocksDB
> >> > > > >>>>>>>> transaction
> >> > > > >>>>>>>>>>>>>>>>>> buffers
> >> > > > >>>>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the
> >> offsets
> >> > > > >> are
> >> > > > >>>>>> synced to
> >> > > > >>>>>>>>>>>>>>>> disk to
> >> > > > >>>>>>>>>>>>>>>>>>>>>> prevent
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
> >> > > > >>>> Consequently,
> >> > > > >>>>>> my
> >> > > > >>>>>>>>>>>>>>>> current
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every*
> Task
> >> > > > >> commit,
> >> > > > >>>>>> which
> >> > > > >>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>> far too
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> frequent.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000
> >> > > > >> records,
> >> > > > >>>>>> which
> >> > > > >>>>>>>> is
> >> > > > >>>>>>>>>> a
> >> > > > >>>>>>>>>>>>>>>>>> change
> >> > > > >>>>>>>>>>>>>>>>>>>> in
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
> >> > > > >> performance
> >> > > > >>>>>> problems
> >> > > > >>>>>>>> for
> >> > > > >>>>>>>>>>>>>>>> some
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> applications.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions,
> >> and
> >> > > > >> I'm
> >> > > > >>>>>>>> currently
> >> > > > >>>>>>>>>>>>>>>> leaning
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> towards
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> using the
> >> > > > >>>> statestore.transaction.buffer.max.bytes
> >> > > > >>>>>>>>>>>>>>>>>> configuration
> >> > > > >>>>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely
> to
> >> > > > >>>> exceed it.
> >> > > > >>>>>>>> This
> >> > > > >>>>>>>>>>>>>>>> would
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> complement the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
> >> > > > >> functionality
> >> > > > >>>> that
> >> > > > >>>>>> this
> >> > > > >>>>>>>>>>>>>>>>>>>> configuration
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Currently, we use
> >> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> >> > > > >>>>>>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> force an
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            early Task commit if
> >> > processing
> >> > > > >>>> more
> >> > > > >>>>>>>> records
> >> > > > >>>>>>>>>>>> would
> >> > > > >>>>>>>>>>>>>>>>>> cause
> >> > > > >>>>>>>>>>>>>>>>>>>> our
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> state
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> store
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            transactions to exceed
> the
> >> > > > >> memory
> >> > > > >>>>>> assigned
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>> them.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - New functionality:
> when a
> >> > > > >> Task
> >> > > > >>>> *does*
> >> > > > >>>>>>>>>> commit,
> >> > > > >>>>>>>>>>>>>>>> we will
> >> > > > >>>>>>>>>>>>>>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the stores (and hence
> flush
> >> > the
> >> > > > >>>>>> transaction
> >> > > > >>>>>>>>>>>>>>>> buffers)
> >> > > > >>>>>>>>>>>>>>>>>>>> unless
> >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> expect to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            cross the
> >> > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> >> > > > >>>>>>>>>>>>>>>>>>>> threshold
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the next
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick
> >> > > > >> Telford <
> >> > > > >>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I
> >> > > > >> meant in
> >> > > > >>>>>> terms of
> >> > > > >>>>>>>>>>>>>>>>>>>> understanding
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance profile under various
> >> > > > >>>>>> circumstances. The
> >> > > > >>>>>>>>>>>>>>>> more
> >> > > > >>>>>>>>>>>>>>>>>>>> complex
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it
> would
> >> > > > >> be for
> >> > > > >>>>>> users
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>> understand
> >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance they see. For example,
> >> > > > >> spilling
> >> > > > >>>>>> records
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>> disk
> >> > > > >>>>>>>>>>>>>>>>>>>> when
> >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a
> >> threshold
> >> > > > >>>> would, I
> >> > > > >>>>>>>>>> expect,
> >> > > > >>>>>>>>>>>>>>>>>> reduce
> >> > > > >>>>>>>>>>>>>>>>>>>>>> write
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
> >> > > > >>>> throughput
> >> > > > >>>>>> could
> >> > > > >>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>>>>>> unexpected,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> >> > > > >>>> diagnose/understand for
> >> > > > >>>>>>>>>> users.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early
> >> > > > >> commit"
> >> > > > >>>>>> concept is
> >> > > > >>>>>>>>>>>>>>>>>> relatively
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to
> >> document,
> >> > > > >> and
> >> > > > >>>>>>>>>>>> conceptually
> >> > > > >>>>>>>>>>>>>>>>>> fairly
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> obvious to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users. We could probably add a
> >> metric to
> >> > > > >>>> make it
> >> > > > >>>>>>>>>> easier
> >> > > > >>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it happens though.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially*
> >> an
> >> > > > >>>> indirect
> >> > > > >>>>>> way
> >> > > > >>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>>> telling
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether they should be
> transactional.
> >> > > > >>>>>> READ_COMMITTED
> >> > > > >>>>>>>>>>>>>>>>>>>> essentially
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> requires
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates
> >> that
> >> > > > >> two
> >> > > > >>>>>> threads
> >> > > > >>>>>>>>>>>>>>>> calling
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see
> >> writes
> >> > > > >>>> from
> >> > > > >>>>>> the
> >> > > > >>>>>>>>>> other
> >> > > > >>>>>>>>>>>>>>>>>>>>>> transaction
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> until
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> >> > > > >>>>>> READ_UNCOMMITTED, all
> >> > > > >>>>>>>>>>>>>>>> bets are
> >> > > > >>>>>>>>>>>>>>>>>>>> off,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe
> >> > > > >> written
> >> > > > >>>>>> records
> >> > > > >>>>>>>> at
> >> > > > >>>>>>>>>>>>>>>> any
> >> > > > >>>>>>>>>>>>>>>>>>>> time,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> which is
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That
> >> > > > >> said,
> >> > > > >>>>>>>> StateStores
> >> > > > >>>>>>>>>>>>>>>> are
> >> > > > >>>>>>>>>>>>>>>>>> free
> >> > > > >>>>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implement
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these guarantees however they can,
> >> > > > >> which is
> >> > > > >>>> a
> >> > > > >>>>>> bit
> >> > > > >>>>>>>> more
> >> > > > >>>>>>>>>>>>>>>>>> relaxed
> >> > > > >>>>>>>>>>>>>>>>>>>>>> than
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dictating "you must use
> >> transactions".
> >> > > > >> For
> >> > > > >>>>>> example,
> >> > > > >>>>>>>>>>>> with
> >> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> >> > > > >>>>>>>>>>>>>>>>>>>>>> we
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED
> ==
> >> > > > >>>> WBWI-based
> >> > > > >>>>>>>>>>>>>>>>>> "transactions",
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes
> to
> >> the
> >> > > > >>>>>> database.
> >> > > > >>>>>>>> But
> >> > > > >>>>>>>>>>>>>>>> with
> >> > > > >>>>>>>>>>>>>>>>>>>> other
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> storage
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to
> >> > > > >> *always*
> >> > > > >>>> use
> >> > > > >>>>>>>>>>>>>>>> transactions,
> >> > > > >>>>>>>>>>>>>>>>>>>> even
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be
> storage
> >> > > > >> engines
> >> > > > >>>>>> that
> >> > > > >>>>>>>>>> don't
> >> > > > >>>>>>>>>>>>>>>>>> provide
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation
> >> > > > >> guarantees
> >> > > > >>>> can
> >> > > > >>>>>> be
> >> > > > >>>>>>>> met
> >> > > > >>>>>>>>>>>>>>>> using a
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> different
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> technique.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the
> >> > > > >> StateStore
> >> > > > >>>>>> interface
> >> > > > >>>>>>>> as
> >> > > > >>>>>>>>>>>>>>>>>> loosely
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> coupled
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as
> possible,
> >> to
> >> > > > >> give
> >> > > > >>>>>>>>>>>>>>>> implementers
> >> > > > >>>>>>>>>>>>>>>>>> more
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> freedom, and
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal
> >> knowledge
> >> > > > >>>>>> required.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> >> > > > >>>> "IsolationLevel"
> >> > > > >>>>>> might
> >> > > > >>>>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> right
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make
> >> it
> >> > > > >> much
> >> > > > >>>> more
> >> > > > >>>>>>>>>>>>>>>> explicit if
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> required, e.g.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7-8.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I can make these changes either
> later
> >> > > > >> today
> >> > > > >>>> or
> >> > > > >>>>>>>>>>>> tomorrow.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Small update:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and
> >> > > > >> fixed a
> >> > > > >>>>>> bunch of
> >> > > > >>>>>>>>>>>>>>>> issues
> >> > > > >>>>>>>>>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the
> tests
> >> > > > >> pass,
> >> > > > >>>>>> which is
> >> > > > >>>>>>>>>>>>>>>>>> promising,
> >> > > > >>>>>>>>>>>>>>>>>>>> but
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> it
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to undergo some performance
> >> > > > >> testing. I
> >> > > > >>>>>> haven't
> >> > > > >>>>>>>>>>>>>>>> (yet)
> >> > > > >>>>>>>>>>>>>>>>>>>> worked
> >> > > > >>>>>>>>>>>>>>>>>>>>>> on
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()`
> >> stuff,
> >> > > > >> but I
> >> > > > >>>>>> would
> >> > > > >>>>>>>>>>>>>>>> expect
> >> > > > >>>>>>>>>>>>>>>>>> that,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
> >> > > > >>>> difference. The
> >> > > > >>>>>>>>>> branch
> >> > > > >>>>>>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>>>> available
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> at
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> >> > > > >>>>>>>>>> if
> >> > > > >>>>>>>>>>>>>>>>>> anyone
> >> > > > >>>>>>>>>>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno
> >> > > > >> Cadonna
> >> > > > >>>> <
> >> > > > >>>>>>>>>>>>>>>>>>>> cadonna@apache.org>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was
> >> > > > >> actually
> >> > > > >>>> also
> >> > > > >>>>>> my
> >> > > > >>>>>>>>>>>>>>>> point. I
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> understood
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that John was proposing the
> >> ingestion
> >> > > > >> path
> >> > > > >>>> as
> >> > > > >>>>>> a way
> >> > > > >>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>> avoid
> >> > > > >>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> early
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I
> misinterpreted
> >> the
> >> > > > >>>> intent.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that
> >> actually
> >> > > > >> it is
> >> > > > >>>>>> public
> >> > > > >>>>>>>>>>>>>>>> API. My
> >> > > > >>>>>>>>>>>>>>>>>>>>>> question
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects
> >> normal
> >> > > > >>>>>> processing.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for
> the
> >> > > > >> size
> >> > > > >>>> of the
> >> > > > >>>>>>>>>>>>>>>> transaction
> >> > > > >>>>>>>>>>>>>>>>>>>>>> buffer
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit
> >> > > > >> affects
> >> > > > >>>> the
> >> > > > >>>>>> whole
> >> > > > >>>>>>>>>>>>>>>>>> processing
> >> > > > >>>>>>>>>>>>>>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a
> >> > > > >> state
> >> > > > >>>> store
> >> > > > >>>>>> is
> >> > > > >>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>>>>>> confined to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills
> over
> >> and
> >> > > > >>>>>> changes the
> >> > > > >>>>>>>>>>>>>>>> behavior
> >> > > > >>>>>>>>>>>>>>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with
> >> you
> >> > > > >> that
> >> > > > >>>> it
> >> > > > >>>>>> is a
> >> > > > >>>>>>>>>>>>>>>> decent
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compromise. I
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the
> downsides
> >> > > > >> and
> >> > > > >>>> list
> >> > > > >>>>>> the
> >> > > > >>>>>>>>>>>>>>>> options to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> overcome
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that
> all
> >> > > > >>>> options
> >> > > > >>>>>> seem
> >> > > > >>>>>>>>>>>> quite
> >> > > > >>>>>>>>>>>>>>>>>> heavy
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compared
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand
> >> > > > >> what you
> >> > > > >>>>>> mean
> >> > > > >>>>>>>> with
> >> > > > >>>>>>>>>>>>>>>> "less
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> predictable
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for users", though.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
> >> > > > >>>> alternatives
> >> > > > >>>>>>>> really
> >> > > > >>>>>>>>>>>>>>>>>>>>>> interesting.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> But I
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds
> >> good
> >> > > > >> and
> >> > > > >>>> we
> >> > > > >>>>>> should
> >> > > > >>>>>>>>>>>>>>>> continue
> >> > > > >>>>>>>>>>>>>>>>>>>> with
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> it!
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my
> >> > > > >> e-mail on
> >> > > > >>>>>> June
> >> > > > >>>>>>>>>> 20th:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the
> reasoning
> >> > > > >> behind
> >> > > > >>>>>> putting
> >> > > > >>>>>>>>>>>>>>>> isolation
> >> > > > >>>>>>>>>>>>>>>>>>>>>> level
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks!
> >> Should
> >> > > > >>>> that
> >> > > > >>>>>> also
> >> > > > >>>>>>>> be
> >> > > > >>>>>>>>>> a
> >> > > > >>>>>>>>>>>>>>>> way
> >> > > > >>>>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>> give
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to
> >> > > > >> decide
> >> > > > >>>>>> whether
> >> > > > >>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>> turn on
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more
> >> concerned
> >> > > > >> about
> >> > > > >>>>>> how do
> >> > > > >>>>>>>>>> you
> >> > > > >>>>>>>>>>>>>>>> know
> >> > > > >>>>>>>>>>>>>>>>>>>> if a
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be
> written
> >> > > > >> under
> >> > > > >>>> EOS,
> >> > > > >>>>>> if
> >> > > > >>>>>>>> you
> >> > > > >>>>>>>>>>>>>>>> do not
> >> > > > >>>>>>>>>>>>>>>>>>>>>> have a
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> way
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
> >> > > > >>>> transactional or
> >> > > > >>>>>> not.
> >> > > > >>>>>>>>>> If
> >> > > > >>>>>>>>>>>>>>>> a
> >> > > > >>>>>>>>>>>>>>>>>> state
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file
> >> can
> >> > > > >> be
> >> > > > >>>>>> written
> >> > > > >>>>>>>>>>>>>>>> during
> >> > > > >>>>>>>>>>>>>>>>>>>> normal
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state
> >> > > > >> store
> >> > > > >>>> is not
> >> > > > >>>>>>>>>>>>>>>>>> transactional,
> >> > > > >>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be
> written
> >> > > > >> under
> >> > > > >>>> EOS.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My point was about not only
> >> > > > >> considering the
> >> > > > >>>>>> bytes
> >> > > > >>>>>>>> in
> >> > > > >>>>>>>>>>>>>>>> memory
> >> > > > >>>>>>>>>>>>>>>>>> in
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> config
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes,
> >> but
> >> > > > >> also
> >> > > > >>>>>> bytes
> >> > > > >>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>> might
> >> > > > >>>>>>>>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> spilled
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was
> wondering
> >> > > > >>>> whether you
> >> > > > >>>>>>>>>> should
> >> > > > >>>>>>>>>>>>>>>>>> remove
> >> > > > >>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of
> >> memory
> >> > > > >>>> bytes to
> >> > > > >>>>>> be
> >> > > > >>>>>>>>>> used
> >> > > > >>>>>>>>>>>>>>>> to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store
> >> > > > >> records." My
> >> > > > >>>>>>>> thinking
> >> > > > >>>>>>>>>>>>>>>> was
> >> > > > >>>>>>>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>>>>>> even
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted
> >> bytes to
> >> > > > >>>> disk,
> >> > > > >>>>>>>>>> limiting
> >> > > > >>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>> overall
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> bytes
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about
> it
> >> > > > >> again
> >> > > > >>>> and
> >> > > > >>>>>>>>>>>>>>>> considering
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> recent
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too
> >> much
> >> > > > >>>> sense
> >> > > > >>>>>>>> anymore.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I like the name
> >> > > > >>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes that
> >> > > > >>>>>>>>>>>>>>>>>>>> you
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 8.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> >> > > > >>>>>> implementation
> >> > > > >>>>>>>>>>>>>>>> details) of
> >> > > > >>>>>>>>>>>>>>>>>>>> how
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of
> >> > > > >> changelog
> >> > > > >>>>>>>>>>>>>>>> transactions,
> >> > > > >>>>>>>>>>>>>>>>>>>> state
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing
> >> would be
> >> > > > >>>> great.
> >> > > > >>>>>>>> Would
> >> > > > >>>>>>>>>>>> be
> >> > > > >>>>>>>>>>>>>>>>>> great
> >> > > > >>>>>>>>>>>>>>>>>>>> if
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> you
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could also add some sentences
> about
> >> the
> >> > > > >>>>>> behavior in
> >> > > > >>>>>>>>>>>>>>>> case of
> >> > > > >>>>>>>>>>>>>>>>>> a
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> failure.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For instance how does a
> >> transactional
> >> > > > >> state
> >> > > > >>>>>> store
> >> > > > >>>>>>>>>>>>>>>> recover
> >> > > > >>>>>>>>>>>>>>>>>>>> after a
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
> >> > > > >>>> transaction
> >> > > > >>>>>>>> buffer,
> >> > > > >>>>>>>>>>>>>>>> etc.
> >> > > > >>>>>>>>>>>>>>>>>>>> (that
> >> > > > >>>>>>>>>>>>>>>>>>>>>> is
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> what
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point
> 9.)
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford
> >> wrote:
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue
> >> that
> >> > > > >>>>>>>>>>>>>>>> WriteBatchWithIndex
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or
> likely
> >> to
> >> > > > >>>> exceed)
> >> > > > >>>>>>>>>>>>>>>> configured
> >> > > > >>>>>>>>>>>>>>>>>>>> memory
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needs to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
> >> > > > >>>> Ultimately,
> >> > > > >>>>>> any
> >> > > > >>>>>>>>>>>>>>>> approach
> >> > > > >>>>>>>>>>>>>>>>>>>> based
> >> > > > >>>>>>>>>>>>>>>>>>>>>> on
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cracking
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using
> >> it in
> >> > > > >>>> ways
> >> > > > >>>>>> it's
> >> > > > >>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>> really
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> designed
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for is
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen
> >> > > > >> performance
> >> > > > >>>> or
> >> > > > >>>>>>>>>>>>>>>> consistency
> >> > > > >>>>>>>>>>>>>>>>>>>> issues.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for
> removing
> >> > > > >> these
> >> > > > >>>>>> early
> >> > > > >>>>>>>>>>>>>>>> commits?
> >> > > > >>>>>>>>>>>>>>>>>>>> While
> >> > > > >>>>>>>>>>>>>>>>>>>>>> not
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideal, I
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise
> >> to
> >> > > > >>>> ensure
> >> > > > >>>>>>>>>>>>>>>> consistency
> >> > > > >>>>>>>>>>>>>>>>>>>> whilst
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maintaining
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas
> seem
> >> > > > >> *very*
> >> > > > >>>>>>>>>>>>>>>> complicated, and
> >> > > > >>>>>>>>>>>>>>>>>>>> might
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable
> for
> >> > > > >> users
> >> > > > >>>> as a
> >> > > > >>>>>>>>>>>>>>>> consequence.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the
> scope
> >> of
> >> > > > >> this
> >> > > > >>>>>> KIP is
> >> > > > >>>>>>>>>>>>>>>> growing a
> >> > > > >>>>>>>>>>>>>>>>>>>> bit
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> out
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to
> discuss
> >> > > > >> ideas
> >> > > > >>>> for
> >> > > > >>>>>>>> future
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> improvements, I
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> >> > > > >>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the
> scope
> >> > > > >> down
> >> > > > >>>> to a
> >> > > > >>>>>>>> design
> >> > > > >>>>>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>>>>>> achieves
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> most
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant
> sized
> >> > > > >>>>>> restorations
> >> > > > >>>>>>>>>>>>>>>> during
> >> > > > >>>>>>>>>>>>>>>>>> dirty
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any
> >> design
> >> > > > >> that
> >> > > > >>>>>> this KIP
> >> > > > >>>>>>>>>>>>>>>> produces
> >> > > > >>>>>>>>>>>>>>>>>>>> can
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ultimately
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future,
> >> especially
> >> > > > >> if
> >> > > > >>>> the
> >> > > > >>>>>> bulk
> >> > > > >>>>>>>> of
> >> > > > >>>>>>>>>>>>>>>> it is
> >> > > > >>>>>>>>>>>>>>>>>>>>>> internal
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next
> >> week
> >> > > > >>>> trying
> >> > > > >>>>>> to
> >> > > > >>>>>>>>>>>>>>>> re-work
> >> > > > >>>>>>>>>>>>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to
> >> remove
> >> > > > >> the
> >> > > > >>>>>>>>>>>>>>>> newTransaction()
> >> > > > >>>>>>>>>>>>>>>>>>>>>> method,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> such
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation
> detail
> >> of
> >> > > > >>>>>>>> RocksDBStore.
> >> > > > >>>>>>>>>>>>>>>> That
> >> > > > >>>>>>>>>>>>>>>>>>>> way, if
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in
> the
> >> > > > >> future,
> >> > > > >>>>>> like
> >> > > > >>>>>>>> the
> >> > > > >>>>>>>>>>>>>>>> SST
> >> > > > >>>>>>>>>>>>>>>>>> file
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> management
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do
> so
> >> > > > >> with
> >> > > > >>>>>> little/no
> >> > > > >>>>>>>>>>>> API
> >> > > > >>>>>>>>>>>>>>>>>>>> changes.
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>>
> >> > > > >>>>>>>>>>>>
> >> > > > >>>>>>>>>>>
> >> > > > >>>>>>>>>>
> >> > > > >>>>>>>>>
> >> > > > >>>>>>>>
> >> > > > >>>>>>
> >> > > > >>>>>>
> >> > > > >>>>
> >> > > > >>
> >> > > > >
> >> > >
> >> >
> >>
> >
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi everyone,

After various off-list discussions, it's become clear there are still some
contentious points to this KIP.

1.
We agreed to change the "default.state.isolation.level" config property to
be a transactional state store feature toggle. What should we name this?
"enable.transactional.statestores"? What's the convention for this kind of
flag?

2.
The "atomic checkpointing" section has been broken out into KIP-1035 (with
one significant change, and otherwise just more explanation). This is now a
hard dependency of KIP-892.

Regards,
Nick

On Mon, 6 Nov 2023 at 17:00, Nick Telford <ni...@gmail.com> wrote:

> Hi everyone,
>
> Sorry for the delay.
>
> I've updated the KIP based on our discussion. I've also added some details
> on "Transactional support under READ_UNCOMMITTED" and  "Query-time
> Isolation Levels" in the "Rejected Alternatives" section, to highlight the
> potential to improve things in a future KIP.
>
> If there are no more requested changes or objections, I'll kick off the
> vote thread tomorrow, approximately 24 hours from now.
>
> While I'd love for this KIP to make it into 3.7.0, I can't commit to it
> being code-complete by the December 20th deadline. While the branch I have
> is broadly working, there are areas that will likely require improvement,
> and others that are untested:
>
> - Untested outside of unit tests: Windowed stores, Versioned stores,
> Global stores, IQv2. They should all work as expected, but the Position
> file logic might be a bit janky.
> - Requires improvement: Position data handling. Very clunky right now.
> - Requires implementation: The "feature flag", where we disable
> transactionality under EOS+READ_UNCOMMITTED. I've started this, but it's
> far from simple. Plus this will require fleshing out the test suite quite a
> bit.
>
> For the curious, my active development branch has been rebased against
> 3.6.0: https://github.com/nicktelford/kafka/tree/KIP-892-3.6.0
>
> Regards,
> Nick
>
> On Mon, 30 Oct 2023 at 23:32, Sophie Blee-Goldman <so...@responsive.dev>
> wrote:
>
>> Hey Nick, sounds like things are moving along here. I think you're already
>> aware, but just as a reminder to us all, the KIP freeze deadline for 3.7
>> is
>> approaching and is currently set for Nov 18. I think we can all agree that
>> it would be great to have this KIP accepted by then, and it sounds like
>> much of the implementation is already completed, in which case this might
>> be able to make it into 3.7.
>>
>> It sounds like this discussion is wrapping up so I just want to summarize
>> my understanding of the current plan regarding configs since it doesn't
>> seem like the KIP has been updated to include this yet.
>>
>> Basically we're all in agreement to go with option #1 that you presented
>> earlier, right? Assuming so, I just want to clear up some details around
>> the default behavior. What happens if:
>> 1. User sets EOS and sets READ_UNCOMMITTED: txn state stores will be
>> disabled/feature-flagged
>> 2. User sets EOS and does not set anything for the isolation level:
>> although the default is READ_UNCOMMITTED, enabling EOS will change the
>> default to READ_COMMITTED and txn state stores will be used
>>
>> As for the future, it sounds like when READ_UNCOMMITTED mode is
>> implemented, we will basically just remove this "feature flag" and txn
>> state stores will always be used for all EOS cases. EOS users will be able
>> to configure the isolation level independently, although it will still
>> default to READ_COMMITTED when EOS is enabled and it wasn't explicitly
>> configured.
>>
>> Is this understanding correct? I think this was the right choice
>> regardless, as it will give people a way to disable the txn stores in an
>> emergency -- as a project we went a long time with little pressure to
>> feature flag things, and our users paid the cost for that. Even if we
>> managed to ship something without bugs, it was often only after an intense
>> period of successive blocker bugs that delayed the entire released for
>> weeks. Other times, major bugs slipped through and some versions became
>> unusable for certain use cases. So having some way to disable the txn
>> state
>> stores when EOS is used feels like a good strategy, since you just never
>> know what might get through all the testing we do.
>>
>> If this ends up slipping to 4.0 and you manage to implement the
>> READ_UNCOMMITTED mode within the same release, I think it's worth
>> considering to add in an additional feature flag, even if it's just a
>> backdoor internal config (eg as we did in KIP-441 with the internal task
>> assignor config). But for now let's assume the first release this KIP
>> appears in will have the behavior as described above, with
>> READ_UNCOMMITTED
>> mode acting as a feature flag
>>
>> Nick -- please let us know when you've updated the KIP to clarify the
>> config behavior, and are ready for a vote!
>>
>>
>> On Sun, Oct 29, 2023 at 12:02 PM Colt McNealy <co...@littlehorse.io>
>> wrote:
>>
>> > Guozhang—I agree, I am in favor of moving forward with the KIP now that
>> the
>> > Transactional State Stores will be behind a feature flag.
>> >
>> > Nick—I just did a bit more light testing of your branch `KIP-892-3.5.0`
>> > with your most recent changes. I couldn't detect a performance
>> difference
>> > versus trunk (in the past there was a slight degradation of performance
>> on
>> > the restoration path, but that has been fixed). I don't believe that
>> your
>> > branch has the state updater thread enabled, so I didn't test that path
>> too
>> > heavily.
>> >
>> > As expected, however, our internal correctness tests failed due to the
>> IQ
>> > read-your-own-writes issue we discussed previously. The community as a
>> > whole would vastly benefit from this KIP getting over the finish line in
>> > 3.7.0, and so long as it is behind a feature flag so that we at
>> LittleHorse
>> > can still guarantee RYOW for our users, I think it's purely a win 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 <
>> guozhang.wang.us@gmail.com
>> > >
>> > wrote:
>> >
>> > > I'd agree with you guys that as long as we are in agreement about the
>> > > configuration semantics, that would be a big win to move forward for
>> > > this KIP. As for the TaskCorruptedException handling like wiping state
>> > > stores, we can discuss that in the PR rather than in the KIP.
>> > >
>> > > Just to clarify, I'm onboard with the latest proposal, and probably we
>> > > can move on for voting on this KIP now?
>> > >
>> > > Guozhang
>> > >
>> > > On Thu, Oct 19, 2023 at 5:33 AM Bruno Cadonna <ca...@apache.org>
>> > wrote:
>> > > >
>> > > > Hi Nick,
>> > > >
>> > > > What you and Lucas wrote about the different configurations of
>> ALOS/EOS
>> > > > and READ_COMMITTED/READ_UNCOMMITTED make sense to me. My earlier
>> > > > concerns about changelogs diverging from the content of the local
>> state
>> > > > stores turned out to not apply. So I think, we can move on with
>> those
>> > > > configurations.
>> > > >
>> > > > Regarding the TaskCorruptedException and wiping out the state stores
>> > > > under EOS, couldn't we abort the transaction on the state store and
>> > > > close the task dirty? If the Kafka transaction was indeed committed,
>> > the
>> > > > store would restore the missing part from the changelog topic. If
>> the
>> > > > Kafka transaction was not committed, changelog topic and state store
>> > are
>> > > > in-sync.
>> > > >
>> > > > In any case, IMO those are implementation details that we do not
>> need
>> > to
>> > > > discuss and solve in the KIP discussion. We can solve them on the
>> PR.
>> > > > The important thing is that the processing guarantees hold.
>> > > >
>> > > > Best,
>> > > > Bruno
>> > > >
>> > > > On 10/18/23 3:56 PM, Nick Telford wrote:
>> > > > > Hi Lucas,
>> > > > >
>> > > > > TaskCorruptedException is how Streams signals that the Task state
>> > > needs to
>> > > > > be wiped, so we can't retain that exception without also wiping
>> state
>> > > on
>> > > > > timeouts.
>> > > > >
>> > > > > Regards,
>> > > > > Nick
>> > > > >
>> > > > > On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <
>> lbrutschy@confluent.io
>> > > .invalid>
>> > > > > wrote:
>> > > > >
>> > > > >> Hi Nick,
>> > > > >>
>> > > > >> I think indeed the better behavior would be to retry
>> > commitTransaction
>> > > > >> until we risk running out of time to meet `max.poll.interval.ms
>> `.
>> > > > >>
>> > > > >> However, if it's handled as a `TaskCorruptedException` at the
>> > moment,
>> > > > >> I would do the same in this KIP, and leave exception handling
>> > > > >> improvements to future work. 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 <
>> > nick.telford@gmail.com>
>> > > > >> wrote:
>> > > > >>>
>> > > > >>> Hi Lucas,
>> > > > >>>
>> > > > >>> Yeah, this is pretty much the direction I'm thinking of going in
>> > > now. You
>> > > > >>> make an interesting point about committing on-error under
>> > > > >>> ALOS/READ_COMMITTED, although I haven't had a chance to think
>> > > through the
>> > > > >>> implications yet.
>> > > > >>>
>> > > > >>> Something that I ran into earlier this week is an issue with the
>> > new
>> > > > >>> handling of TimeoutException. Without TX stores,
>> TimeoutException
>> > > under
>> > > > >> EOS
>> > > > >>> throws a TaskCorruptedException, which wipes the stores.
>> However,
>> > > with TX
>> > > > >>> stores, TimeoutException is now just bubbled up and dealt with
>> as
>> > it
>> > > is
>> > > > >>> under ALOS. The problem arises when the
>> Producer#commitTransaction
>> > > call
>> > > > >>> times out: Streams attempts to ignore the error and continue
>> > > producing,
>> > > > >>> which causes the next call to Producer#send to throw
>> > > > >>> "IllegalStateException: Cannot attempt operation `send` because
>> the
>> > > > >>> previous call to `commitTransaction` timed out and must be
>> > retried".
>> > > > >>>
>> > > > >>> I'm not sure what we should do here: retrying the
>> commitTransaction
>> > > seems
>> > > > >>> logical, but what if it times out again? Where do we draw the
>> line
>> > > and
>> > > > >>> shutdown the instance?
>> > > > >>>
>> > > > >>> Regards,
>> > > > >>> Nick
>> > > > >>>
>> > > > >>> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <
>> > lbrutschy@confluent.io
>> > > > >> .invalid>
>> > > > >>> wrote:
>> > > > >>>
>> > > > >>>> Hi all,
>> > > > >>>>
>> > > > >>>> I think I liked your suggestion of allowing EOS with
>> > > READ_UNCOMMITTED,
>> > > > >>>> but keep wiping the state on error, and I'd vote for this
>> solution
>> > > > >>>> when introducing `default.state.isolation.level`. This way,
>> we'd
>> > > have
>> > > > >>>> the most low-risk roll-out of this feature (no behavior change
>> > > without
>> > > > >>>> reconfiguration), with the possibility of switching to the most
>> > > sane /
>> > > > >>>> battle-tested default settings in 4.0. Essentially, we'd have a
>> > > > >>>> feature flag but call it `default.state.isolation.level` and
>> don't
>> > > > >>>> have to deprecate it later.
>> > > > >>>>
>> > > > >>>> So the possible configurations would then be this:
>> > > > >>>>
>> > > > >>>> 1. ALOS/READ_UNCOMMITTED (default) = processing uses
>> direct-to-DB,
>> > > IQ
>> > > > >>>> reads from DB.
>> > > > >>>> 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads
>> from
>> > > > >>>> WriteBatch/DB. Flush on error (see note below).
>> > > > >>>> 3. EOS/READ_UNCOMMITTED (default) = processing uses
>> direct-to-DB,
>> > IQ
>> > > > >>>> reads from DB. Wipe state on error.
>> > > > >>>> 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads
>> from
>> > > > >>>> WriteBatch/DB.
>> > > > >>>>
>> > > > >>>> I believe the feature is important enough that we will see good
>> > > > >>>> adoption even without changing the default. In 4.0, when we
>> have
>> > > seen
>> > > > >>>> this being adopted and is battle-tested, we make READ_COMMITTED
>> > the
>> > > > >>>> default for EOS, or even READ_COMITTED always the default,
>> > depending
>> > > > >>>> on our experiences. And we could add a clever implementation of
>> > > > >>>> READ_UNCOMITTED with WriteBatches later.
>> > > > >>>>
>> > > > >>>> The only smell here is that `default.state.isolation.level`
>> > wouldn't
>> > > > >>>> be purely an IQ setting, but it would also (slightly) change
>> the
>> > > > >>>> behavior of the processing, but that seems unavoidable as long
>> as
>> > we
>> > > > >>>> haven't solve READ_UNCOMITTED IQ with WriteBatches.
>> > > > >>>>
>> > > > >>>> Minor: As for Bruno's point 4, I think if we are concerned
>> about
>> > > this
>> > > > >>>> 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).
>> > > > >>>>
>> > > > >>>> Cheers,
>> > > > >>>> Lucas
>> > > > >>>>
>> > > > >>>> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <
>> > > nick.telford@gmail.com>
>> > > > >>>> wrote:
>> > > > >>>>>
>> > > > >>>>> Hi Guozhang,
>> > > > >>>>>
>> > > > >>>>> The KIP as it stands introduces a new configuration,
>> > > > >>>>> default.state.isolation.level, which is independent of
>> > > > >> processing.mode.
>> > > > >>>>> It's intended that this new configuration be used to
>> configure a
>> > > > >> global
>> > > > >>>> IQ
>> > > > >>>>> isolation level in the short term, with a future KIP
>> introducing
>> > > the
>> > > > >>>>> capability to change the isolation level on a per-query basis,
>> > > > >> falling
>> > > > >>>> back
>> > > > >>>>> to the "default" defined by this config. That's why I called
>> it
>> > > > >>>> "default",
>> > > > >>>>> for future-proofing.
>> > > > >>>>>
>> > > > >>>>> However, it currently includes the caveat that
>> READ_UNCOMMITTED
>> > is
>> > > > >> not
>> > > > >>>>> available under EOS. I think this is the coupling you are
>> > alluding
>> > > > >> to?
>> > > > >>>>>
>> > > > >>>>> This isn't intended to be a restriction of the API, but is
>> > > currently
>> > > > >> a
>> > > > >>>>> technical limitation. However, after discussing with some
>> users
>> > > about
>> > > > >>>>> use-cases that would require READ_UNCOMMITTED under EOS, I'm
>> > > > >> inclined to
>> > > > >>>>> remove that clause and put in the necessary work to make that
>> > > > >> combination
>> > > > >>>>> possible now.
>> > > > >>>>>
>> > > > >>>>> I currently see two possible approaches:
>> > > > >>>>>
>> > > > >>>>>     1. Disable TX StateStores internally when the
>> IsolationLevel
>> > is
>> > > > >>>>>     READ_UNCOMMITTED and the processing.mode is EOS. This is
>> more
>> > > > >>>> difficult
>> > > > >>>>>     than it sounds, as there are many assumptions being made
>> > > > >> throughout
>> > > > >>>> the
>> > > > >>>>>     internals about the guarantees StateStores provide. It
>> would
>> > > > >>>> definitely add
>> > > > >>>>>     a lot of extra "if (read_uncommitted && eos)" branches,
>> > > > >> complicating
>> > > > >>>>>     maintenance and testing.
>> > > > >>>>>     2. Invest the time *now* to make READ_UNCOMMITTED of EOS
>> > > > >> StateStores
>> > > > >>>>>     possible. I have some ideas on how this could be achieved,
>> > but
>> > > > >> they
>> > > > >>>> would
>> > > > >>>>>     need testing and could introduce some additional issues.
>> The
>> > > > >> benefit
>> > > > >>>> of
>> > > > >>>>>     this approach is that it would make query-time
>> > IsolationLevels
>> > > > >> much
>> > > > >>>> simpler
>> > > > >>>>>     to implement in the future.
>> > > > >>>>>
>> > > > >>>>> Unfortunately, both will require considerable work that will
>> > > further
>> > > > >>>> delay
>> > > > >>>>> this KIP, which was the reason I placed the restriction in the
>> > KIP
>> > > > >> in the
>> > > > >>>>> first place.
>> > > > >>>>>
>> > > > >>>>> Regards,
>> > > > >>>>> Nick
>> > > > >>>>>
>> > > > >>>>> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
>> > > > >> guozhang.wang.us@gmail.com>
>> > > > >>>>> wrote:
>> > > > >>>>>
>> > > > >>>>>> Hello Nick,
>> > > > >>>>>>
>> > > > >>>>>> First of all, thanks a lot for the great effort you've put in
>> > > > >> driving
>> > > > >>>>>> this KIP! I really like it coming through finally, as many
>> > people
>> > > > >> in
>> > > > >>>>>> the community have raised this. At the same time I honestly
>> > feel a
>> > > > >> bit
>> > > > >>>>>> ashamed for not putting enough of my time supporting it and
>> > > > >> pushing it
>> > > > >>>>>> through the finish line (you raised this KIP almost a year
>> ago).
>> > > > >>>>>>
>> > > > >>>>>> I briefly passed through the DISCUSS thread so far, not sure
>> > I've
>> > > > >> 100
>> > > > >>>>>> percent digested all the bullet points. But with the goal of
>> > > > >> trying to
>> > > > >>>>>> help take it through the finish line in mind, I'd want to
>> throw
>> > > > >>>>>> thoughts on top of my head only on the point #4 above which I
>> > felt
>> > > > >> may
>> > > > >>>>>> be the main hurdle for the current KIP to drive to a
>> consensus
>> > > now.
>> > > > >>>>>>
>> > > > >>>>>> The general question I asked myself is, whether we want to
>> > couple
>> > > > >> "IQ
>> > > > >>>>>> reading mode" with "processing mode". While technically I
>> tend
>> > to
>> > > > >>>>>> agree with you that, it's feels like a bug if some single
>> user
>> > > > >> chose
>> > > > >>>>>> "EOS" for processing mode while choosing "read uncommitted"
>> for
>> > IQ
>> > > > >>>>>> reading mode, at the same time, I'm thinking if it's possible
>> > that
>> > > > >>>>>> there could be two different persons (or even two teams) that
>> > > > >> would be
>> > > > >>>>>> using the stream API to build the app, and the IQ API to
>> query
>> > the
>> > > > >>>>>> running state of the app. I know this is less of a technical
>> > thing
>> > > > >> but
>> > > > >>>>>> rather a more design stuff, but if it could be ever the case,
>> > I'm
>> > > > >>>>>> wondering if the personale using the IQ API knows about the
>> > risks
>> > > > >> of
>> > > > >>>>>> using read uncommitted but still chose so for the favor of
>> > > > >>>>>> performance, no matter if the underlying stream processing
>> mode
>> > > > >>>>>> configured by another personale is EOS or not. In that
>> regard,
>> > I'm
>> > > > >>>>>> leaning towards a "leaving the door open, and close it later
>> if
>> > we
>> > > > >>>>>> found it's a bad idea" aspect with a configuration 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 can still
>> enforce
>> > > > >>>>>> coupling them.
>> > > > >>>>>>
>> > > > >>>>>> Again, just my 2c here. Thanks again for the great patience
>> and
>> > > > >>>>>> diligence on this KIP.
>> > > > >>>>>>
>> > > > >>>>>>
>> > > > >>>>>> Guozhang
>> > > > >>>>>>
>> > > > >>>>>>
>> > > > >>>>>>
>> > > > >>>>>> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
>> > > > >> nick.telford@gmail.com>
>> > > > >>>>>> wrote:
>> > > > >>>>>>>
>> > > > >>>>>>> Hi Bruno,
>> > > > >>>>>>>
>> > > > >>>>>>> 4.
>> > > > >>>>>>> I'll hold off on making that change until we have a
>> consensus
>> > as
>> > > > >> to
>> > > > >>>> what
>> > > > >>>>>>> configuration to use to control all of this, as it'll be
>> > > > >> affected by
>> > > > >>>> the
>> > > > >>>>>>> decision on EOS isolation levels.
>> > > > >>>>>>>
>> > > > >>>>>>> 5.
>> > > > >>>>>>> Done. I've chosen "committedOffsets".
>> > > > >>>>>>>
>> > > > >>>>>>> Regards,
>> > > > >>>>>>> Nick
>> > > > >>>>>>>
>> > > > >>>>>>> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <
>> > cadonna@apache.org>
>> > > > >>>> wrote:
>> > > > >>>>>>>
>> > > > >>>>>>>> Hi Nick,
>> > > > >>>>>>>>
>> > > > >>>>>>>> 1.
>> > > > >>>>>>>> Yeah, you are probably right that it does not make too much
>> > > > >> sense.
>> > > > >>>>>>>> Thanks for the clarification!
>> > > > >>>>>>>>
>> > > > >>>>>>>>
>> > > > >>>>>>>> 4.
>> > > > >>>>>>>> Yes, sorry for the back and forth, but I think for the
>> sake of
>> > > > >> the
>> > > > >>>> KIP
>> > > > >>>>>>>> it is better to let the ALOS behavior as it is for now due
>> to
>> > > > >> the
>> > > > >>>>>>>> possible issues you would run into. Maybe we can find a
>> > > > >> solution
>> > > > >>>> in the
>> > > > >>>>>>>> future. Now the question returns to whether we really need
>> > > > >>>>>>>> default.state.isolation.level. Maybe the config could be
>> the
>> > > > >>>> feature
>> > > > >>>>>>>> flag Sophie requested.
>> > > > >>>>>>>>
>> > > > >>>>>>>>
>> > > > >>>>>>>> 5.
>> > > > >>>>>>>> There 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
>> committedOffsetFor().
>> > > > >>>>>>>>
>> > > > >>>>>>>>
>> > > > >>>>>>>> Best,
>> > > > >>>>>>>> Bruno
>> > > > >>>>>>>>
>> > > > >>>>>>>>
>> > > > >>>>>>>> On 10/13/23 3:21 PM, Nick Telford wrote:
>> > > > >>>>>>>>> Hi Bruno,
>> > > > >>>>>>>>>
>> > > > >>>>>>>>> Thanks for getting back to me.
>> > > > >>>>>>>>>
>> > > > >>>>>>>>> 1.
>> > > > >>>>>>>>> I think this should be possible. Are you thinking of the
>> > > > >>>> situation
>> > > > >>>>>> where
>> > > > >>>>>>>> a
>> > > > >>>>>>>>> user may downgrade to a previous version of Kafka
>> Streams? In
>> > > > >>>> that
>> > > > >>>>>> case,
>> > > > >>>>>>>>> sadly, the RocksDBStore would get wiped by the older
>> version
>> > > > >> of
>> > > > >>>> Kafka
>> > > > >>>>>>>>> Streams anyway, because that version wouldn't understand
>> the
>> > > > >>>> extra
>> > > > >>>>>> column
>> > > > >>>>>>>>> family (that holds offsets), so the missing Position file
>> > > > >> would
>> > > > >>>>>>>>> automatically get rebuilt when the store is rebuilt from
>> the
>> > > > >>>>>> changelog.
>> > > > >>>>>>>>> Are there other situations than downgrade where a
>> > > > >> transactional
>> > > > >>>> store
>> > > > >>>>>>>> could
>> > > > >>>>>>>>> be replaced by a non-transactional one? I can't think of
>> any.
>> > > > >>>>>>>>>
>> > > > >>>>>>>>> 2.
>> > > > >>>>>>>>> Ahh yes, the Test Plan - my Kryptonite! This section
>> > > > >> definitely
>> > > > >>>>>> needs to
>> > > > >>>>>>>> be
>> > > > >>>>>>>>> fleshed out. I'll work on that. How much detail do you
>> need?
>> > > > >>>>>>>>>
>> > > > >>>>>>>>> 3.
>> > > > >>>>>>>>> See my previous email discussing this.
>> > > > >>>>>>>>>
>> > > > >>>>>>>>> 4.
>> > > > >>>>>>>>> Hmm, this is an interesting point. Are you suggesting that
>> > > > >> under
>> > > > >>>> ALOS
>> > > > >>>>>>>>> READ_COMMITTED should not be supported?
>> > > > >>>>>>>>>
>> > > > >>>>>>>>> Regards,
>> > > > >>>>>>>>> Nick
>> > > > >>>>>>>>>
>> > > > >>>>>>>>> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
>> > > > >> cadonna@apache.org>
>> > > > >>>>>> wrote:
>> > > > >>>>>>>>>
>> > > > >>>>>>>>>> Hi Nick,
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>> I think the KIP is converging!
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>> 1.
>> > > > >>>>>>>>>> I am wondering whether it makes sense to write the
>> position
>> > > > >> file
>> > > > >>>>>> during
>> > > > >>>>>>>>>> close as we do for the checkpoint file, so that in case
>> the
>> > > > >>>> state
>> > > > >>>>>> store
>> > > > >>>>>>>>>> is replaced with a non-transactional state store the
>> > > > >>>>>> non-transactional
>> > > > >>>>>>>>>> state store finds the position file. I think, this is not
>> > > > >>>> strictly
>> > > > >>>>>>>>>> needed, but would be a nice behavior instead of just
>> > > > >> deleting
>> > > > >>>> the
>> > > > >>>>>>>>>> position file.
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>> 2.
>> > > > >>>>>>>>>> The test plan does not mention integration tests. Do you
>> not
>> > > > >>>> need to
>> > > > >>>>>>>>>> extend existing ones and add new ones. Also for upgrading
>> > > > >> and
>> > > > >>>>>>>>>> downgrading you might need integration and/or system
>> tests.
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>> 3.
>> > > > >>>>>>>>>> I think Sophie made a point. Although, IQ reading from
>> > > > >>>> uncommitted
>> > > > >>>>>> data
>> > > > >>>>>>>>>> under EOS might be considered a bug by some people. Thus,
>> > > > >> your
>> > > > >>>> KIP
>> > > > >>>>>> would
>> > > > >>>>>>>>>> fix a bug rather than changing the intended behavior.
>> > > > >> However, I
>> > > > >>>>>> also
>> > > > >>>>>>>>>> see that a feature flag would help users that rely on
>> this
>> > > > >> buggy
>> > > > >>>>>>>>>> behavior (at least until AK 4.0).
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>> 4.
>> > > > >>>>>>>>>> This is related to the previous point. I assume that the
>> > > > >>>> difference
>> > > > >>>>>>>>>> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is
>> > > > >> that in
>> > > > >>>> the
>> > > > >>>>>>>>>> former you enable transactions on the state store and in
>> the
>> > > > >>>> latter
>> > > > >>>>>> you
>> > > > >>>>>>>>>> disable them. If my assumption is correct, I think that
>> is
>> > > > >> an
>> > > > >>>> issue.
>> > > > >>>>>>>>>> Let's assume under ALOS Streams fails over a couple of
>> times
>> > > > >>>> more or
>> > > > >>>>>>>>>> less at the same step in processing after value 3 is
>> added
>> > > > >> to an
>> > > > >>>>>>>>>> aggregation but the offset of the corresponding input
>> record
>> > > > >>>> was not
>> > > > >>>>>>>>>> committed. Without transactions disabled, the aggregation
>> > > > >> value
>> > > > >>>>>> would
>> > > > >>>>>>>>>> increase by 3 for each failover. With transactions
>> enabled,
>> > > > >>>> value 3
>> > > > >>>>>>>>>> would only be added to the aggregation once when the
>> offset
>> > > > >> of
>> > > > >>>> the
>> > > > >>>>>> input
>> > > > >>>>>>>>>> record is committed and the transaction finally
>> completes.
>> > > > >> So
>> > > > >>>> the
>> > > > >>>>>>>>>> content of the state store would change depending on the
>> > > > >>>>>> configuration
>> > > > >>>>>>>>>> for IQ. IMO, the content of the state store should be
>> > > > >>>> independent
>> > > > >>>>>> from
>> > > > >>>>>>>>>> IQ. Given this issue, I propose to not use transactions
>> with
>> > > > >>>> ALOS at
>> > > > >>>>>>>>>> all. I was a big proponent of using transactions with
>> ALOS,
>> > > > >> but
>> > > > >>>> I
>> > > > >>>>>>>>>> realized that transactions with ALOS is not as easy as
>> > > > >> enabling
>> > > > >>>>>>>>>> transactions on state stores. Another aspect that is
>> > > > >>>> problematic is
>> > > > >>>>>> that
>> > > > >>>>>>>>>> the changelog topic which actually replicates the state
>> > > > >> store
>> > > > >>>> is not
>> > > > >>>>>>>>>> transactional under ALOS. Thus, it might happen that the
>> > > > >> state
>> > > > >>>>>> store and
>> > > > >>>>>>>>>> the changelog differ in their content. All of this is
>> maybe
>> > > > >>>> solvable
>> > > > >>>>>>>>>> somehow, but for the sake of this KIP, I would leave it
>> for
>> > > > >> the
>> > > > >>>>>> future.
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>> Best,
>> > > > >>>>>>>>>> Bruno
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
>> > > > >>>>>>>>>>> Hey Nick! First of all thanks for taking up this awesome
>> > > > >>>> feature,
>> > > > >>>>>> I'm
>> > > > >>>>>>>>>> sure
>> > > > >>>>>>>>>>> every single
>> > > > >>>>>>>>>>> Kafka Streams user and dev would agree that it is sorely
>> > > > >>>> needed.
>> > > > >>>>>>>>>>>
>> > > > >>>>>>>>>>> I've just been catching up on the KIP and surrounding
>> > > > >>>> discussion,
>> > > > >>>>>> so
>> > > > >>>>>>>>>> please
>> > > > >>>>>>>>>>> forgive me
>> > > > >>>>>>>>>>> for any misunderstandings or misinterpretations of the
>> > > > >> current
>> > > > >>>>>> plan and
>> > > > >>>>>>>>>>> don't hesitate to
>> > > > >>>>>>>>>>> correct me.
>> > > > >>>>>>>>>>>
>> > > > >>>>>>>>>>> Before I jump in, I just want to say that having seen
>> this
>> > > > >>>> drag on
>> > > > >>>>>> for
>> > > > >>>>>>>> so
>> > > > >>>>>>>>>>> long, my singular
>> > > > >>>>>>>>>>> goal in responding is to help this KIP past a perceived
>> > > > >>>> impasse so
>> > > > >>>>>> we
>> > > > >>>>>>>> can
>> > > > >>>>>>>>>>> finally move on
>> > > > >>>>>>>>>>> to voting and implementing it. Long discussions are to
>> be
>> > > > >>>> expected
>> > > > >>>>>> for
>> > > > >>>>>>>>>>> major features like
>> > > > >>>>>>>>>>> this but it's completely on us as the Streams devs to
>> make
>> > > > >> sure
>> > > > >>>>>> there
>> > > > >>>>>>>> is
>> > > > >>>>>>>>>> an
>> > > > >>>>>>>>>>> end in sight
>> > > > >>>>>>>>>>> for any ongoing discussion.
>> > > > >>>>>>>>>>>
>> > > > >>>>>>>>>>> With that said, it's my understanding that the KIP as
>> > > > >> currently
>> > > > >>>>>>>> proposed
>> > > > >>>>>>>>>> is
>> > > > >>>>>>>>>>> just not tenable
>> > > > >>>>>>>>>>> for Kafka Streams, and would prevent some EOS users from
>> > > > >>>> upgrading
>> > > > >>>>>> to
>> > > > >>>>>>>> the
>> > > > >>>>>>>>>>> version it
>> > > > >>>>>>>>>>> first appears in. Given that we can't predict or
>> guarantee
>> > > > >>>> whether
>> > > > >>>>>> any
>> > > > >>>>>>>> of
>> > > > >>>>>>>>>>> the followup KIPs
>> > > > >>>>>>>>>>> would be completed in the same release cycle as this
>> one,
>> > > > >> we
>> > > > >>>> need
>> > > > >>>>>> to
>> > > > >>>>>>>> make
>> > > > >>>>>>>>>>> sure that the
>> > > > >>>>>>>>>>> feature is either compatible with all current users or
>> else
>> > > > >>>>>>>>>> feature-flagged
>> > > > >>>>>>>>>>> so that they may
>> > > > >>>>>>>>>>> opt in/out.
>> > > > >>>>>>>>>>>
>> > > > >>>>>>>>>>> Therefore, IIUC we need to have either (or both) of
>> these
>> > > > >> as
>> > > > >>>>>>>>>>> fully-implemented config options:
>> > > > >>>>>>>>>>> 1. default.state.isolation.level
>> > > > >>>>>>>>>>> 2. enable.transactional.state.stores
>> > > > >>>>>>>>>>>
>> > > > >>>>>>>>>>> This way EOS users for whom read_committed semantics are
>> > > > >> not
>> > > > >>>>>> viable can
>> > > > >>>>>>>>>>> still upgrade,
>> > > > >>>>>>>>>>> and either use the isolation.level config to leverage
>> the
>> > > > >> new
>> > > > >>>> txn
>> > > > >>>>>> state
>> > > > >>>>>>>>>>> stores without sacrificing
>> > > > >>>>>>>>>>> their application semantics, or else simply keep the
>> > > > >>>> transactional
>> > > > >>>>>>>> state
>> > > > >>>>>>>>>>> stores disabled until we
>> > > > >>>>>>>>>>> are able to fully implement the isolation level
>> > > > >> configuration
>> > > > >>>> at
>> > > > >>>>>> either
>> > > > >>>>>>>>>> an
>> > > > >>>>>>>>>>> application or query level.
>> > > > >>>>>>>>>>>
>> > > > >>>>>>>>>>> Frankly you are the expert here and know much more about
>> > > > >> the
>> > > > >>>>>> tradeoffs
>> > > > >>>>>>>> in
>> > > > >>>>>>>>>>> both semantics and
>> > > > >>>>>>>>>>> effort level of implementing one of these configs vs the
>> > > > >>>> other. In
>> > > > >>>>>> my
>> > > > >>>>>>>>>>> opinion, either option would
>> > > > >>>>>>>>>>> be fine and I would leave the decision of which one to
>> > > > >> include
>> > > > >>>> in
>> > > > >>>>>> this
>> > > > >>>>>>>>>> KIP
>> > > > >>>>>>>>>>> completely up to you.
>> > > > >>>>>>>>>>> I just don't see a way for the KIP to proceed without
>> some
>> > > > >>>>>> variation of
>> > > > >>>>>>>>>> the
>> > > > >>>>>>>>>>> above that would allow
>> > > > >>>>>>>>>>> EOS users to opt-out of read_committed.
>> > > > >>>>>>>>>>>
>> > > > >>>>>>>>>>> (If it's all the same to you, I would recommend 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 that gets released.
>> > > > >>>>>>>>>>> So personally, my choice would be to add the feature
>> flag
>> > > > >> and
>> > > > >>>>>> leave it
>> > > > >>>>>>>>>> off
>> > > > >>>>>>>>>>> by default. If all 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
>> > > > >>>>>>>>>>>
>> > > > >>>>>>>>>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
>> > > > >>>>>> nick.telford@gmail.com>
>> > > > >>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>
>> > > > >>>>>>>>>>>> Hi Bruno,
>> > > > >>>>>>>>>>>>
>> > > > >>>>>>>>>>>> Agreed, I can live with that for now.
>> > > > >>>>>>>>>>>>
>> > > > >>>>>>>>>>>> In an effort to keep the scope of this KIP from
>> > > > >> expanding, I'm
>> > > > >>>>>> leaning
>> > > > >>>>>>>>>>>> towards just providing a configurable
>> > > > >>>>>> default.state.isolation.level
>> > > > >>>>>>>> and
>> > > > >>>>>>>>>>>> removing IsolationLevel from the StateStoreContext.
>> This
>> > > > >>>> would be
>> > > > >>>>>>>>>>>> compatible with adding support for query-time
>> > > > >> IsolationLevels
>> > > > >>>> in
>> > > > >>>>>> the
>> > > > >>>>>>>>>>>> future, whilst providing a way for users to select an
>> > > > >>>> isolation
>> > > > >>>>>> level
>> > > > >>>>>>>>>> now.
>> > > > >>>>>>>>>>>>
>> > > > >>>>>>>>>>>> The big problem with this, however, is that if a user
>> > > > >> selects
>> > > > >>>>>>>>>>>> processing.mode
>> > > > >>>>>>>>>>>> = "exactly-once(-v2|-beta)", and
>> > > > >>>> default.state.isolation.level =
>> > > > >>>>>>>>>>>> "READ_UNCOMMITTED", we need to guarantee that the data
>> > > > >> isn't
>> > > > >>>>>> written
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>> disk until commit() is called, but we also need to
>> permit
>> > > > >> IQ
>> > > > >>>>>> threads
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>> read from the ongoing transaction.
>> > > > >>>>>>>>>>>>
>> > > > >>>>>>>>>>>> A simple solution would be to (temporarily) forbid this
>> > > > >>>>>> combination of
>> > > > >>>>>>>>>>>> configuration, and have default.state.isolation.level
>> > > > >>>>>> automatically
>> > > > >>>>>>>>>> switch
>> > > > >>>>>>>>>>>> to READ_COMMITTED when processing.mode is anything
>> other
>> > > > >> than
>> > > > >>>>>>>>>>>> at-least-once. Do you think this would be acceptable?
>> > > > >>>>>>>>>>>>
>> > > > >>>>>>>>>>>> In a later KIP, we can add support for query-time
>> > > > >> isolation
>> > > > >>>>>> levels and
>> > > > >>>>>>>>>>>> solve this particular problem there, which would relax
>> > > > >> this
>> > > > >>>>>>>> restriction.
>> > > > >>>>>>>>>>>>
>> > > > >>>>>>>>>>>> Regards,
>> > > > >>>>>>>>>>>> Nick
>> > > > >>>>>>>>>>>>
>> > > > >>>>>>>>>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
>> > > > >>>> cadonna@apache.org>
>> > > > >>>>>>>>>> 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,
>> > > > >>>>>>>>>>>>> Bruno
>> > > > >>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
>> > > > >>>>>>>>>>>>>> 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 <
>> > > > >>>>>> nick.telford@gmail.com
>> > > > >>>>>>>>>
>> > > > >>>>>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> Hi everyone,
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> I agree that having IsolationLevel be determined at
>> > > > >>>> query-time
>> > > > >>>>>> is
>> > > > >>>>>>>> the
>> > > > >>>>>>>>>>>>>>> ideal design, but there are a few sticking points:
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> 1.
>> > > > >>>>>>>>>>>>>>> There needs to be some way to communicate the
>> > > > >>>> IsolationLevel
>> > > > >>>>>> down
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>> RocksDBStore itself, so that the query can respect
>> it.
>> > > > >>>> Since
>> > > > >>>>>> stores
>> > > > >>>>>>>>>>>> are
>> > > > >>>>>>>>>>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
>> > > > >>>>>> MeteredStore,
>> > > > >>>>>>>>>>>>> etc.),
>> > > > >>>>>>>>>>>>>>> we need some way to deliver that information to the
>> > > > >> bottom
>> > > > >>>>>> layer.
>> > > > >>>>>>>> For
>> > > > >>>>>>>>>>>>> IQv2,
>> > > > >>>>>>>>>>>>>>> we can use the existing State#query() method, but
>> IQv1
>> > > > >> has
>> > > > >>>> no
>> > > > >>>>>> way
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>> do
>> > > > >>>>>>>>>>>>>>> this.
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> A simple approach, which would potentially open up
>> > > > >> other
>> > > > >>>>>> options,
>> > > > >>>>>>>>>>>> would
>> > > > >>>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
>> > > > >>>>>>>>>>>>>>> readOnlyView(IsolationLevel isolationLevel) to
>> > > > >>>>>>>> ReadOnlyKeyValueStore
>> > > > >>>>>>>>>>>>> (and
>> > > > >>>>>>>>>>>>>>> similar to ReadOnlyWindowStore,
>> ReadOnlySessionStore,
>> > > > >>>> etc.).
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> 2.
>> > > > >>>>>>>>>>>>>>> As mentioned above, RocksDB WriteBatches are not
>> > > > >>>> thread-safe,
>> > > > >>>>>> which
>> > > > >>>>>>>>>>>>> causes
>> > > > >>>>>>>>>>>>>>> a problem if we want to provide READ_UNCOMMITTED
>> > > > >>>> Iterators. I
>> > > > >>>>>> also
>> > > > >>>>>>>>>>>> had a
>> > > > >>>>>>>>>>>>>>> look at RocksDB Transactions[1], but they solve a
>> very
>> > > > >>>>>> different
>> > > > >>>>>>>>>>>>> problem,
>> > > > >>>>>>>>>>>>>>> and have the same thread-safety issue.
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> One possible approach that I mentioned is chaining
>> > > > >>>>>> WriteBatches:
>> > > > >>>>>>>>>> every
>> > > > >>>>>>>>>>>>>>> time a new Interactive Query is received (i.e.
>> > > > >>>> readOnlyView,
>> > > > >>>>>> see
>> > > > >>>>>>>>>>>> above,
>> > > > >>>>>>>>>>>>>>> is called) we "freeze" the existing WriteBatch, and
>> > > > >> start a
>> > > > >>>>>> new one
>> > > > >>>>>>>>>>>> for
>> > > > >>>>>>>>>>>>> new
>> > > > >>>>>>>>>>>>>>> writes. The Interactive Query queries the "chain" of
>> > > > >>>> previous
>> > > > >>>>>>>>>>>>> WriteBatches
>> > > > >>>>>>>>>>>>>>> + the underlying database; while the StreamThread
>> > > > >> starts
>> > > > >>>>>> writing to
>> > > > >>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>> *new* WriteBatch. On-commit, the StreamThread would
>> > > > >> write
>> > > > >>>> *all*
>> > > > >>>>>>>>>>>>>>> WriteBatches in the chain to the database (that have
>> > > > >> not
>> > > > >>>> yet
>> > > > >>>>>> been
>> > > > >>>>>>>>>>>>> written).
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> WriteBatches would be closed/freed only when they
>> have
>> > > > >> been
>> > > > >>>>>> both
>> > > > >>>>>>>>>>>>>>> committed, and all open Interactive Queries on them
>> > > > >> have
>> > > > >>>> been
>> > > > >>>>>>>> closed.
>> > > > >>>>>>>>>>>>> This
>> > > > >>>>>>>>>>>>>>> would require some reference counting.
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> Obviously a drawback of this approach is the
>> potential
>> > > > >> for
>> > > > >>>>>>>> increased
>> > > > >>>>>>>>>>>>>>> memory usage: if an Interactive Query is long-lived,
>> > > > >> for
>> > > > >>>>>> example by
>> > > > >>>>>>>>>>>>> doing a
>> > > > >>>>>>>>>>>>>>> full scan over a large database, or even just
>> pausing
>> > > > >> in
>> > > > >>>> the
>> > > > >>>>>> middle
>> > > > >>>>>>>>>> of
>> > > > >>>>>>>>>>>>> an
>> > > > >>>>>>>>>>>>>>> iteration, then the existing chain of WriteBatches
>> > > > >> could be
>> > > > >>>>>> kept
>> > > > >>>>>>>>>>>> around
>> > > > >>>>>>>>>>>>> for
>> > > > >>>>>>>>>>>>>>> a long time, potentially forever.
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> --
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> A.
>> > > > >>>>>>>>>>>>>>> Going off on a tangent, it looks like in addition to
>> > > > >>>> supporting
>> > > > >>>>>>>>>>>>>>> READ_COMMITTED queries, we could go further and
>> support
>> > > > >>>>>>>>>>>> REPEATABLE_READ
>> > > > >>>>>>>>>>>>>>> queries (i.e. where subsequent reads to the same key
>> > > > >> in the
>> > > > >>>>>> same
>> > > > >>>>>>>>>>>>>>> Interactive Query are guaranteed to yield the same
>> > > > >> value)
>> > > > >>>> by
>> > > > >>>>>> making
>> > > > >>>>>>>>>>>> use
>> > > > >>>>>>>>>>>>> of
>> > > > >>>>>>>>>>>>>>> RocksDB Snapshots[2]. These are fairly lightweight,
>> so
>> > > > >> the
>> > > > >>>>>>>>>> performance
>> > > > >>>>>>>>>>>>>>> impact is likely to be negligible, but they do
>> require
>> > > > >>>> that the
>> > > > >>>>>>>>>>>>> Interactive
>> > > > >>>>>>>>>>>>>>> Query session can be explicitly closed.
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> This could be achieved if we made the above
>> > > > >> readOnlyView
>> > > > >>>>>> interface
>> > > > >>>>>>>>>>>> look
>> > > > >>>>>>>>>>>>>>> more like:
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueView<K, V> implements
>> > > > >>>>>>>>>>>> ReadOnlyKeyValueStore<K,
>> > > > >>>>>>>>>>>>>>> V>, AutoCloseable {}
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
>> > > > >>>>>>>>>>>>>>>         ...
>> > > > >>>>>>>>>>>>>>>         ReadOnlyKeyValueView<K, V>
>> > > > >>>> readOnlyView(IsolationLevel
>> > > > >>>>>>>>>>>>> isolationLevel);
>> > > > >>>>>>>>>>>>>>> }
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> But this would be a breaking change, as existing
>> IQv1
>> > > > >>>> queries
>> > > > >>>>>> are
>> > > > >>>>>>>>>>>>>>> guaranteed to never call store.close(), and
>> therefore
>> > > > >> these
>> > > > >>>>>> would
>> > > > >>>>>>>>>> leak
>> > > > >>>>>>>>>>>>>>> memory under REPEATABLE_READ.
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> B.
>> > > > >>>>>>>>>>>>>>> One thing that's notable: MyRocks states that they
>> > > > >> support
>> > > > >>>>>>>>>>>>> READ_COMMITTED
>> > > > >>>>>>>>>>>>>>> and REPEATABLE_READ, but they make no mention of
>> > > > >>>>>>>>>>>> READ_UNCOMMITTED[3][4].
>> > > > >>>>>>>>>>>>>>> This could be because doing so is technically
>> > > > >>>>>> difficult/impossible
>> > > > >>>>>>>>>>>> using
>> > > > >>>>>>>>>>>>>>> the primitives available in RocksDB.
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> --
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> Lucas, to address your points:
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> U1.
>> > > > >>>>>>>>>>>>>>> It's only "SHOULD" to permit alternative (i.e.
>> > > > >> non-RocksDB)
>> > > > >>>>>>>>>>>>>>> implementations of StateStore that do not support
>> > > > >> atomic
>> > > > >>>>>> writes.
>> > > > >>>>>>>>>>>>> Obviously
>> > > > >>>>>>>>>>>>>>> in those cases, the guarantees Kafka Streams
>> > > > >>>> provides/expects
>> > > > >>>>>> would
>> > > > >>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>> relaxed. Do you think we should require all
>> > > > >>>> implementations to
>> > > > >>>>>>>>>> support
>> > > > >>>>>>>>>>>>>>> atomic writes?
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> U2.
>> > > > >>>>>>>>>>>>>>> Stores can support multiple IsolationLevels. As
>> we've
>> > > > >>>> discussed
>> > > > >>>>>>>>>> above,
>> > > > >>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>> ideal scenario would be to specify the
>> IsolationLevel
>> > > > >> at
>> > > > >>>>>>>> query-time.
>> > > > >>>>>>>>>>>>>>> Failing that, I think the second-best approach is to
>> > > > >>>> define the
>> > > > >>>>>>>>>>>>>>> IsolationLevel for *all* queries based on the
>> > > > >>>> processing.mode,
>> > > > >>>>>>>> which
>> > > > >>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>> what the default StateStoreContext#isolationLevel()
>> > > > >>>> achieves.
>> > > > >>>>>> Would
>> > > > >>>>>>>>>>>> you
>> > > > >>>>>>>>>>>>>>> prefer an alternative?
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> While the existing implementation is equivalent to
>> > > > >>>>>>>> READ_UNCOMMITTED,
>> > > > >>>>>>>>>>>>> this
>> > > > >>>>>>>>>>>>>>> can yield unexpected results/errors under EOS, if a
>> > > > >>>>>> transaction is
>> > > > >>>>>>>>>>>>> rolled
>> > > > >>>>>>>>>>>>>>> back. While this would be a change in behaviour for
>> > > > >> users,
>> > > > >>>> it
>> > > > >>>>>> would
>> > > > >>>>>>>>>>>> look
>> > > > >>>>>>>>>>>>>>> more like a bug fix than a breaking change. That
>> said,
>> > > > >> we
>> > > > >>>>>> *could*
>> > > > >>>>>>>>>> make
>> > > > >>>>>>>>>>>>> it
>> > > > >>>>>>>>>>>>>>> configurable, and default to the existing behaviour
>> > > > >>>>>>>>>> (READ_UNCOMMITTED)
>> > > > >>>>>>>>>>>>>>> instead of inferring it from the processing.mode?
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> N1, N2.
>> > > > >>>>>>>>>>>>>>> These were only primitives to avoid boxing costs,
>> but
>> > > > >> since
>> > > > >>>>>> this is
>> > > > >>>>>>>>>>>> not
>> > > > >>>>>>>>>>>>> a
>> > > > >>>>>>>>>>>>>>> performance sensitive area, it should be fine to
>> > > > >> change if
>> > > > >>>>>> that's
>> > > > >>>>>>>>>>>>> desirable.
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> N3.
>> > > > >>>>>>>>>>>>>>> It's because the store "manages its own offsets",
>> which
>> > > > >>>>>> includes
>> > > > >>>>>>>> both
>> > > > >>>>>>>>>>>>>>> committing the offset, *and providing it* via
>> > > > >>>>>> getCommittedOffset().
>> > > > >>>>>>>>>>>>>>> Personally, I think "managesOffsets" conveys this
>> best,
>> > > > >>>> but I
>> > > > >>>>>> don't
>> > > > >>>>>>>>>>>> mind
>> > > > >>>>>>>>>>>>>>> changing it if the nomenclature is unclear.
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> Sorry for the massive emails/essays!
>> > > > >>>>>>>>>>>>>>> --
>> > > > >>>>>>>>>>>>>>> Nick
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> 1:
>> > > > >> https://github.com/facebook/rocksdb/wiki/Transactions
>> > > > >>>>>>>>>>>>>>> 2:
>> https://github.com/facebook/rocksdb/wiki/Snapshot
>> > > > >>>>>>>>>>>>>>> 3:
>> > > > >>>>>>>>
>> > > > >> https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
>> > > > >>>>>>>>>>>>>>> 4:
>> > > > >>>> https://mariadb.com/kb/en/myrocks-transactional-isolation/
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
>> > > > >>>>>>>>>>>>>>> <lb...@confluent.io.invalid> wrote:
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>> Hi Nick,
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>> since I last read it in April, the KIP has become
>> much
>> > > > >>>>>> cleaner and
>> > > > >>>>>>>>>>>>>>>> easier to read. Great work!
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>> It feels to me the last big open point is whether
>> we
>> > > > >> can
>> > > > >>>>>> implement
>> > > > >>>>>>>>>>>>>>>> isolation level as a query parameter. I understand
>> > > > >> that
>> > > > >>>> there
>> > > > >>>>>> are
>> > > > >>>>>>>>>>>>>>>> implementation concerns, but as Colt says, it would
>> > > > >> be a
>> > > > >>>> great
>> > > > >>>>>>>>>>>>>>>> addition, and would also simplify the migration
>> path
>> > > > >> for
>> > > > >>>> this
>> > > > >>>>>>>>>> change.
>> > > > >>>>>>>>>>>>>>>> Is the implementation problem you mentioned caused
>> by
>> > > > >> the
>> > > > >>>>>>>> WriteBatch
>> > > > >>>>>>>>>>>>>>>> not having a notion of a snapshot, as the
>> underlying
>> > > > >> DB
>> > > > >>>>>> iterator
>> > > > >>>>>>>>>>>> does?
>> > > > >>>>>>>>>>>>>>>> In that case, I am not sure a chain of WriteBatches
>> > > > >> as you
>> > > > >>>>>> propose
>> > > > >>>>>>>>>>>>>>>> would fully solve the problem, but maybe I didn't
>> dig
>> > > > >>>> enough
>> > > > >>>>>> into
>> > > > >>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>> details to fully understand it.
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>> If it's not possible to implement it now, would it
>> be
>> > > > >> an
>> > > > >>>>>> option to
>> > > > >>>>>>>>>>>>>>>> make sure in this KIP that we do not fully close
>> the
>> > > > >> door
>> > > > >>>> on
>> > > > >>>>>>>>>>>> per-query
>> > > > >>>>>>>>>>>>>>>> isolation levels in the interface, as it may be
>> > > > >> possible
>> > > > >>>> to
>> > > > >>>>>>>>>> implement
>> > > > >>>>>>>>>>>>>>>> the missing primitives in RocksDB or Speedb in the
>> > > > >> future.
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>> Understanding:
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets
>> to
>> > > > >> be
>> > > > >>>>>> persisted
>> > > > >>>>>>>>>>>>>>>> atomically with the records?
>> > > > >>>>>>>>>>>>>>>> * U2) Don't understand the default implementation
>> of
>> > > > >>>>>>>>>>>> `isolationLevel`.
>> > > > >>>>>>>>>>>>>>>> The isolation level should be a property of the
>> > > > >> underlying
>> > > > >>>>>> store,
>> > > > >>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>> not be defined by the default config? Existing
>> stores
>> > > > >>>> probably
>> > > > >>>>>>>> don't
>> > > > >>>>>>>>>>>>>>>> guarantee READ_COMMITTED, so the default should be
>> to
>> > > > >>>> return
>> > > > >>>>>>>>>>>>>>>> READ_UNCOMMITTED.
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>> Nits:
>> > > > >>>>>>>>>>>>>>>> * N1) Could `getComittedOffset` use an
>> `OptionalLong`
>> > > > >>>> return
>> > > > >>>>>> type,
>> > > > >>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>> avoid the `null`?
>> > > > >>>>>>>>>>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
>> > > > >>>>>> `OptionalLong`
>> > > > >>>>>>>>>>>>>>>> return type, to avoid the `-1`?
>> > > > >>>>>>>>>>>>>>>> * N3) I don't understand why `managesOffsets` uses
>> the
>> > > > >>>>>> 'manage'
>> > > > >>>>>>>>>> verb,
>> > > > >>>>>>>>>>>>>>>> whereas all other methods use the "commits" verb.
>> I'd
>> > > > >>>> suggest
>> > > > >>>>>>>>>>>>>>>> `commitsOffsets`.
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>> Either way, it feels this KIP is very close to the
>> > > > >> finish
>> > > > >>>>>> line,
>> > > > >>>>>>>> I'm
>> > > > >>>>>>>>>>>>>>>> looking forward to seeing this in production!
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>> Cheers,
>> > > > >>>>>>>>>>>>>>>> Lucas
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
>> > > > >>>>>> colt@littlehorse.io
>> > > > >>>>>>>>>
>> > > > >>>>>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>> Making IsolationLevel a query-time constraint,
>> > > > >> rather
>> > > > >>>> than
>> > > > >>>>>>>> linking
>> > > > >>>>>>>>>>>> it
>> > > > >>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>> the processing.guarantee.
>> > > > >>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>> As I understand it, would this allow even a user
>> of
>> > > > >> EOS
>> > > > >>>> to
>> > > > >>>>>>>> control
>> > > > >>>>>>>>>>>>>>>> whether
>> > > > >>>>>>>>>>>>>>>>> reading committed or uncommitted records? If so,
>> I am
>> > > > >>>> highly
>> > > > >>>>>> in
>> > > > >>>>>>>>>>>> favor
>> > > > >>>>>>>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>> this.
>> > > > >>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>> I know that I was one of the early people to point
>> > > > >> out
>> > > > >>>> the
>> > > > >>>>>>>> current
>> > > > >>>>>>>>>>>>>>>>> shortcoming that IQ reads uncommitted records, but
>> > > > >> just
>> > > > >>>> this
>> > > > >>>>>>>>>>>> morning I
>> > > > >>>>>>>>>>>>>>>>> realized a pattern we use which means that (for
>> > > > >> certain
>> > > > >>>>>> queries)
>> > > > >>>>>>>>>> our
>> > > > >>>>>>>>>>>>>>>> system
>> > > > >>>>>>>>>>>>>>>>> needs to be able to read uncommitted records,
>> which
>> > > > >> is
>> > > > >>>> the
>> > > > >>>>>>>> current
>> > > > >>>>>>>>>>>>>>>> behavior
>> > > > >>>>>>>>>>>>>>>>> of Kafka Streams in EOS.***
>> > > > >>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>> If IsolationLevel being a query-time decision
>> allows
>> > > > >> for
>> > > > >>>>>> this,
>> > > > >>>>>>>> then
>> > > > >>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>> would be amazing. I would also vote that the
>> default
>> > > > >>>> behavior
>> > > > >>>>>>>>>> should
>> > > > >>>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>> for
>> > > > >>>>>>>>>>>>>>>>> reading uncommitted records, because it is totally
>> > > > >>>> possible
>> > > > >>>>>> for a
>> > > > >>>>>>>>>>>>> valid
>> > > > >>>>>>>>>>>>>>>>> application to depend on that behavior, and
>> breaking
>> > > > >> it
>> > > > >>>> in a
>> > > > >>>>>>>> minor
>> > > > >>>>>>>>>>>>>>>> release
>> > > > >>>>>>>>>>>>>>>>> might be a bit strong.
>> > > > >>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>> *** (Note, for the curious reader....) Our
>> > > > >> use-case/query
>> > > > >>>>>> pattern
>> > > > >>>>>>>>>>>> is a
>> > > > >>>>>>>>>>>>>>>> bit
>> > > > >>>>>>>>>>>>>>>>> complex, but reading "uncommitted" records is
>> > > > >> actually
>> > > > >>>> safe
>> > > > >>>>>> in
>> > > > >>>>>>>> our
>> > > > >>>>>>>>>>>>> case
>> > > > >>>>>>>>>>>>>>>>> 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 soon as the processor
>> processes
>> > > > >> the
>> > > > >>>>>> result.
>> > > > >>>>>>>> If
>> > > > >>>>>>>>>>>>> they
>> > > > >>>>>>>>>>>>>>>>> can't read uncommitted records, they may receive a
>> > > > >> "201 -
>> > > > >>>>>>>> Created"
>> > > > >>>>>>>>>>>>>>>>> response, immediately followed by a "404 - 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 <
>> > > > >>>>>>>>>>>> nick.telford@gmail.com>
>> > > > >>>>>>>>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>> Addendum:
>> > > > >>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>> I think we would also face the same problem with
>> the
>> > > > >>>>>> approach
>> > > > >>>>>>>> John
>> > > > >>>>>>>>>>>>>>>> outlined
>> > > > >>>>>>>>>>>>>>>>>> earlier (using the record cache as a transaction
>> > > > >> buffer
>> > > > >>>> and
>> > > > >>>>>>>>>>>> flushing
>> > > > >>>>>>>>>>>>>>>> it
>> > > > >>>>>>>>>>>>>>>>>> straight to SST files). This is because the
>> record
>> > > > >> cache
>> > > > >>>>>> (the
>> > > > >>>>>>>>>>>>>>>> ThreadCache
>> > > > >>>>>>>>>>>>>>>>>> class) is not thread-safe, so every commit would
>> > > > >>>> invalidate
>> > > > >>>>>> open
>> > > > >>>>>>>>>> IQ
>> > > > >>>>>>>>>>>>>>>>>> Iterators in the same way that RocksDB
>> WriteBatches
>> > > > >> do.
>> > > > >>>>>>>>>>>>>>>>>> --
>> > > > >>>>>>>>>>>>>>>>>> Nick
>> > > > >>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
>> > > > >>>>>>>>>>>> nick.telford@gmail.com>
>> > > > >>>>>>>>>>>>>>>>>> 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.
>> > > > >>>>>>>>>>>>>>>>>>> 2. Making IsolationLevel a query-time
>> constraint,
>> > > > >>>> rather
>> > > > >>>>>> than
>> > > > >>>>>>>>>>>>>>>> linking it
>> > > > >>>>>>>>>>>>>>>>>>> to the processing.guarantee.
>> > > > >>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>> There's a wrinkle that makes this a challenge:
>> > > > >>>> Interactive
>> > > > >>>>>>>>>> Queries
>> > > > >>>>>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>>> open an Iterator, when using transactions and
>> > > > >>>>>> READ_UNCOMMITTED.
>> > > > >>>>>>>>>>>>>>>>>>> The problem is that under READ_UNCOMMITTED,
>> queries
>> > > > >>>> need
>> > > > >>>>>> to be
>> > > > >>>>>>>>>>>> able
>> > > > >>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>> read records from the currently uncommitted
>> > > > >> transaction
>> > > > >>>>>> buffer
>> > > > >>>>>>>>>>>>>>>>>>> (WriteBatch). This includes for Iterators, which
>> > > > >> should
>> > > > >>>>>> iterate
>> > > > >>>>>>>>>>>>>>>> both the
>> > > > >>>>>>>>>>>>>>>>>>> transaction buffer and underlying database
>> (using
>> > > > >>>>>>>>>>>>>>>>>>> WriteBatch#iteratorWithBase()).
>> > > > >>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>> The issue is that when the StreamThread
>> commits, it
>> > > > >>>> writes
>> > > > >>>>>> the
>> > > > >>>>>>>>>>>>>>>> current
>> > > > >>>>>>>>>>>>>>>>>>> WriteBatch to RocksDB *and then clears the
>> > > > >> WriteBatch*.
>> > > > >>>>>>>> Clearing
>> > > > >>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>> WriteBatch while an Interactive Query holds an
>> open
>> > > > >>>>>> Iterator on
>> > > > >>>>>>>>>> it
>> > > > >>>>>>>>>>>>>>>> will
>> > > > >>>>>>>>>>>>>>>>>>> invalidate the Iterator. Worse, it turns out
>> that
>> > > > >>>> Iterators
>> > > > >>>>>>>> over
>> > > > >>>>>>>>>> a
>> > > > >>>>>>>>>>>>>>>>>>> WriteBatch become invalidated not just when the
>> > > > >>>> WriteBatch
>> > > > >>>>>> is
>> > > > >>>>>>>>>>>>>>>> cleared,
>> > > > >>>>>>>>>>>>>>>>>> but
>> > > > >>>>>>>>>>>>>>>>>>> also when the Iterators' current key receives a
>> new
>> > > > >>>> write.
>> > > > >>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>> Now that I'm writing this, I remember that this
>> is
>> > > > >> the
>> > > > >>>>>> major
>> > > > >>>>>>>>>>>> reason
>> > > > >>>>>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>> I
>> > > > >>>>>>>>>>>>>>>>>>> switched the original design from having a
>> > > > >> query-time
>> > > > >>>>>>>>>>>>>>>> IsolationLevel to
>> > > > >>>>>>>>>>>>>>>>>>> having the IsolationLevel linked to the
>> > > > >>>> transactionality
>> > > > >>>>>> of the
>> > > > >>>>>>>>>>>>>>>> stores
>> > > > >>>>>>>>>>>>>>>>>>> themselves.
>> > > > >>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>> It *might* be possible to resolve this, by
>> having a
>> > > > >>>>>> "chain" of
>> > > > >>>>>>>>>>>>>>>>>>> WriteBatches, with the StreamThread switching
>> to a
>> > > > >> new
>> > > > >>>>>>>> WriteBatch
>> > > > >>>>>>>>>>>>>>>>>> whenever
>> > > > >>>>>>>>>>>>>>>>>>> a new Interactive Query attempts to read from
>> the
>> > > > >>>>>> database, but
>> > > > >>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>> could
>> > > > >>>>>>>>>>>>>>>>>>> cause some performance problems/memory pressure
>> > > > >> when
>> > > > >>>>>> subjected
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>> a
>> > > > >>>>>>>>>>>>>>>> high
>> > > > >>>>>>>>>>>>>>>>>>> Interactive Query load. It would also reduce the
>> > > > >>>>>> efficiency of
>> > > > >>>>>>>>>>>>>>>>>> WriteBatches
>> > > > >>>>>>>>>>>>>>>>>>> on-commit, as we'd have to write N WriteBatches,
>> > > > >> where
>> > > > >>>> N
>> > > > >>>>>> is the
>> > > > >>>>>>>>>>>>>>>> number of
>> > > > >>>>>>>>>>>>>>>>>>> Interactive Queries since the last commit.
>> > > > >>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>> I realise this is getting into the weeds of the
>> > > > >>>>>> implementation,
>> > > > >>>>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>> you'd
>> > > > >>>>>>>>>>>>>>>>>>> rather we focus on the API for now, but I think
>> > > > >> it's
>> > > > >>>>>> important
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>> consider
>> > > > >>>>>>>>>>>>>>>>>>> how to implement the desired API, in case we
>> come
>> > > > >> up
>> > > > >>>> with
>> > > > >>>>>> an
>> > > > >>>>>>>> API
>> > > > >>>>>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>>> cannot be implemented efficiently, or even at
>> all!
>> > > > >>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>> Thoughts?
>> > > > >>>>>>>>>>>>>>>>>>> --
>> > > > >>>>>>>>>>>>>>>>>>> Nick
>> > > > >>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
>> > > > >>>>>>>> cadonna@apache.org
>> > > > >>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
>> > > > >>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>> 6.
>> > > > >>>>>>>>>>>>>>>>>>>> Of course, you are right! My bad!
>> > > > >>>>>>>>>>>>>>>>>>>> Wiping out the state in the downgrading case is
>> > > > >> fine.
>> > > > >>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>> 3a.
>> > > > >>>>>>>>>>>>>>>>>>>> Focus on the public facing changes for the
>> KIP. We
>> > > > >>>> will
>> > > > >>>>>> manage
>> > > > >>>>>>>>>> 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.
>> > > > >>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>> Best,
>> > > > >>>>>>>>>>>>>>>>>>>> Bruno
>> > > > >>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>> Hi Bruno,
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> Thanks for getting back to me!
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> 2.
>> > > > >>>>>>>>>>>>>>>>>>>>> The fact that implementations can always track
>> > > > >>>> estimated
>> > > > >>>>>>>> memory
>> > > > >>>>>>>>>>>>>>>> usage
>> > > > >>>>>>>>>>>>>>>>>> in
>> > > > >>>>>>>>>>>>>>>>>>>>> the wrapper is a good point. I can remove -1
>> as
>> > > > >> an
>> > > > >>>>>> option,
>> > > > >>>>>>>> and
>> > > > >>>>>>>>>>>>>>>> I'll
>> > > > >>>>>>>>>>>>>>>>>>>> clarify
>> > > > >>>>>>>>>>>>>>>>>>>>> the JavaDoc that 0 is not just for
>> > > > >> non-transactional
>> > > > >>>>>> stores,
>> > > > >>>>>>>>>>>>>>>> which is
>> > > > >>>>>>>>>>>>>>>>>>>>> currently misleading.
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> 6.
>> > > > >>>>>>>>>>>>>>>>>>>>> The problem with catching the exception in the
>> > > > >>>> downgrade
>> > > > >>>>>>>>>> process
>> > > > >>>>>>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>>>>> would require new code in the Kafka version
>> being
>> > > > >>>>>> downgraded
>> > > > >>>>>>>>>> to.
>> > > > >>>>>>>>>>>>>>>> Since
>> > > > >>>>>>>>>>>>>>>>>>>>> users could conceivably downgrade to almost
>> *any*
>> > > > >>>> older
>> > > > >>>>>>>> version
>> > > > >>>>>>>>>>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>>> Kafka
>> > > > >>>>>>>>>>>>>>>>>>>>> Streams, I'm not sure how we could add that
>> code?
>> > > > >>>>>>>>>>>>>>>>>>>>> The only way I can think of doing it would be
>> to
>> > > > >>>> provide
>> > > > >>>>>> a
>> > > > >>>>>>>>>>>>>>>> dedicated
>> > > > >>>>>>>>>>>>>>>>>>>>> downgrade tool, that goes through every local
>> > > > >> store
>> > > > >>>> and
>> > > > >>>>>>>> removes
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>> offsets column families. But that seems like
>> an
>> > > > >>>>>> unnecessary
>> > > > >>>>>>>>>>>>>>>> amount of
>> > > > >>>>>>>>>>>>>>>>>>>> extra
>> > > > >>>>>>>>>>>>>>>>>>>>> code to maintain just to handle a somewhat
>> niche
>> > > > >>>>>> situation,
>> > > > >>>>>>>>>> when
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>> alternative (automatically wipe and restore
>> > > > >> stores)
>> > > > >>>>>> should be
>> > > > >>>>>>>>>>>>>>>>>>>> acceptable.
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've
>> > > > >>>> requested.
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> 3a.
>> > > > >>>>>>>>>>>>>>>>>>>>> I agree that IsolationLevel makes more sense
>> at
>> > > > >>>>>> query-time,
>> > > > >>>>>>>> and
>> > > > >>>>>>>>>>>> I
>> > > > >>>>>>>>>>>>>>>>>>>> actually
>> > > > >>>>>>>>>>>>>>>>>>>>> initially attempted to place the
>> IsolationLevel
>> > > > >> at
>> > > > >>>>>>>> query-time,
>> > > > >>>>>>>>>>>>>>>> but I
>> > > > >>>>>>>>>>>>>>>>>> ran
>> > > > >>>>>>>>>>>>>>>>>>>>> into some problems:
>> > > > >>>>>>>>>>>>>>>>>>>>> - The key issue is that, under ALOS we're not
>> > > > >> staging
>> > > > >>>>>> writes
>> > > > >>>>>>>> in
>> > > > >>>>>>>>>>>>>>>>>>>>> transactions, so can't perform writes at the
>> > > > >>>>>> READ_COMMITTED
>> > > > >>>>>>>>>>>>>>>> isolation
>> > > > >>>>>>>>>>>>>>>>>>>>> level. However, this may be addressed if we
>> > > > >> decide to
>> > > > >>>>>>>> *always*
>> > > > >>>>>>>>>>>>>>>> use
>> > > > >>>>>>>>>>>>>>>>>>>>> transactions as discussed under 3b.
>> > > > >>>>>>>>>>>>>>>>>>>>> - IQv1 and IQv2 have quite different
>> > > > >>>> implementations. I
>> > > > >>>>>>>>>> remember
>> > > > >>>>>>>>>>>>>>>>>> having
>> > > > >>>>>>>>>>>>>>>>>>>>> some difficulty understanding the IQv1
>> internals,
>> > > > >>>> which
>> > > > >>>>>> made
>> > > > >>>>>>>> it
>> > > > >>>>>>>>>>>>>>>>>>>> difficult
>> > > > >>>>>>>>>>>>>>>>>>>>> to determine what needed to be changed.
>> However,
>> > > > >> I
>> > > > >>>>>> *think*
>> > > > >>>>>>>> this
>> > > > >>>>>>>>>>>>>>>> can be
>> > > > >>>>>>>>>>>>>>>>>>>>> addressed for both implementations by wrapping
>> > > > >> the
>> > > > >>>>>>>> RocksDBStore
>> > > > >>>>>>>>>>>>>>>> in an
>> > > > >>>>>>>>>>>>>>>>>>>>> IsolationLevel-dependent wrapper, that
>> overrides
>> > > > >> read
>> > > > >>>>>> methods
>> > > > >>>>>>>>>>>>>>>> (get,
>> > > > >>>>>>>>>>>>>>>>>>>> etc.)
>> > > > >>>>>>>>>>>>>>>>>>>>> to either read directly from the database or
>> > > > >> from the
>> > > > >>>>>> ongoing
>> > > > >>>>>>>>>>>>>>>>>>>> transaction.
>> > > > >>>>>>>>>>>>>>>>>>>>> But IQv1 might still be difficult.
>> > > > >>>>>>>>>>>>>>>>>>>>> - If IsolationLevel becomes a query
>> constraint,
>> > > > >> then
>> > > > >>>> all
>> > > > >>>>>>>> other
>> > > > >>>>>>>>>>>>>>>>>>>> StateStores
>> > > > >>>>>>>>>>>>>>>>>>>>> will need to respect it, including the
>> in-memory
>> > > > >>>> stores.
>> > > > >>>>>> This
>> > > > >>>>>>>>>>>>>>>> would
>> > > > >>>>>>>>>>>>>>>>>>>> require
>> > > > >>>>>>>>>>>>>>>>>>>>> us to adapt in-memory stores to stage their
>> > > > >> writes so
>> > > > >>>>>> they
>> > > > >>>>>>>> can
>> > > > >>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>>>> isolated
>> > > > >>>>>>>>>>>>>>>>>>>>> from READ_COMMITTTED queries. It would also
>> > > > >> become an
>> > > > >>>>>>>> important
>> > > > >>>>>>>>>>>>>>>>>>>>> consideration for third-party stores on
>> upgrade,
>> > > > >> as
>> > > > >>>>>> without
>> > > > >>>>>>>>>>>>>>>> changes,
>> > > > >>>>>>>>>>>>>>>>>>>> they
>> > > > >>>>>>>>>>>>>>>>>>>>> would not support READ_COMMITTED queries
>> > > > >> correctly.
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> Ultimately, I may need some help making the
>> > > > >> necessary
>> > > > >>>>>> change
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>>>>>> IQv1
>> > > > >>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>> support this, but I don't think it's
>> > > > >> fundamentally
>> > > > >>>>>>>> impossible,
>> > > > >>>>>>>>>>>>>>>> if we
>> > > > >>>>>>>>>>>>>>>>>>>> want
>> > > > >>>>>>>>>>>>>>>>>>>>> to pursue this route.
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> 3b.
>> > > > >>>>>>>>>>>>>>>>>>>>> The main reason I chose to keep ALOS
>> > > > >> un-transactional
>> > > > >>>>>> was to
>> > > > >>>>>>>>>>>>>>>> minimize
>> > > > >>>>>>>>>>>>>>>>>>>>> behavioural change for most users (I believe
>> most
>> > > > >>>> Streams
>> > > > >>>>>>>> users
>> > > > >>>>>>>>>>>>>>>> use
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>> default configuration, which is ALOS). That
>> said,
>> > > > >>>> it's
>> > > > >>>>>> clear
>> > > > >>>>>>>>>>>>>>>> that if
>> > > > >>>>>>>>>>>>>>>>>>>> ALOS
>> > > > >>>>>>>>>>>>>>>>>>>>> also used transactional stores, the only
>> change
>> > > > >> in
>> > > > >>>>>> behaviour
>> > > > >>>>>>>>>>>>>>>> would be
>> > > > >>>>>>>>>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>>>>> it would become *more correct*, which could be
>> > > > >>>>>> considered a
>> > > > >>>>>>>>>> "bug
>> > > > >>>>>>>>>>>>>>>> fix"
>> > > > >>>>>>>>>>>>>>>>>> by
>> > > > >>>>>>>>>>>>>>>>>>>>> users, rather than a change they need to
>> handle.
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> I believe that performance using transactions
>> > > > >> (aka.
>> > > > >>>>>> RocksDB
>> > > > >>>>>>>>>>>>>>>>>>>> WriteBatches)
>> > > > >>>>>>>>>>>>>>>>>>>>> should actually be *better* than the
>> un-batched
>> > > > >>>>>> write-path
>> > > > >>>>>>>> that
>> > > > >>>>>>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>>> currently used[1]. The only "performance"
>> > > > >>>> consideration
>> > > > >>>>>> will
>> > > > >>>>>>>> be
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>> increased memory usage that transactions
>> require.
>> > > > >>>> Given
>> > > > >>>>>> the
>> > > > >>>>>>>>>>>>>>>>>> mitigations
>> > > > >>>>>>>>>>>>>>>>>>>> for
>> > > > >>>>>>>>>>>>>>>>>>>>> this memory that we have in place, I would
>> expect
>> > > > >>>> that
>> > > > >>>>>> this
>> > > > >>>>>>>> is
>> > > > >>>>>>>>>>>>>>>> not a
>> > > > >>>>>>>>>>>>>>>>>>>>> problem for most users.
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> If we're happy to do so, we can make ALOS also
>> > > > >> use
>> > > > >>>>>>>>>> transactions.
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> Regards,
>> > > > >>>>>>>>>>>>>>>>>>>>> Nick
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> Link 1:
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>
>> > > > >>>>>>>>>>
>> > > > >>>>>>
>> > > > >>>>
>> > > > >>
>> > >
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
>> > > > >>>>>>>>>>>> cadonna@apache.org
>> > > > >>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> Hi Nick,
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> Thanks for the updates and sorry for the
>> delay
>> > > > >> on my
>> > > > >>>>>> side!
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> 1.
>> > > > >>>>>>>>>>>>>>>>>>>>>> Making the default implementation for
>> flush() a
>> > > > >>>> no-op
>> > > > >>>>>> sounds
>> > > > >>>>>>>>>>>>>>>> good to
>> > > > >>>>>>>>>>>>>>>>>>>> me.
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> 2.
>> > > > >>>>>>>>>>>>>>>>>>>>>> I think what was bugging me here is that a
>> > > > >>>> third-party
>> > > > >>>>>> state
>> > > > >>>>>>>>>>>>>>>> store
>> > > > >>>>>>>>>>>>>>>>>>>> needs
>> > > > >>>>>>>>>>>>>>>>>>>>>> to implement the state store interface. That
>> > > > >> means
>> > > > >>>> they
>> > > > >>>>>> need
>> > > > >>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>> implement a wrapper around the actual state
>> > > > >> store
>> > > > >>>> as we
>> > > > >>>>>> do
>> > > > >>>>>>>> for
>> > > > >>>>>>>>>>>>>>>>>> RocksDB
>> > > > >>>>>>>>>>>>>>>>>>>>>> with RocksDBStore. So, a third-party state
>> > > > >> store can
>> > > > >>>>>> always
>> > > > >>>>>>>>>>>>>>>> estimate
>> > > > >>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> uncommitted bytes, if it wants, because the
>> > > > >> wrapper
>> > > > >>>> can
>> > > > >>>>>>>> record
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>> added
>> > > > >>>>>>>>>>>>>>>>>>>>>> bytes.
>> > > > >>>>>>>>>>>>>>>>>>>>>> One case I can think of where returning -1
>> makes
>> > > > >>>> sense
>> > > > >>>>>> is
>> > > > >>>>>>>> when
>> > > > >>>>>>>>>>>>>>>>>> Streams
>> > > > >>>>>>>>>>>>>>>>>>>>>> does not need to estimate the size of the
>> write
>> > > > >>>> batch
>> > > > >>>>>> and
>> > > > >>>>>>>>>>>>>>>> trigger
>> > > > >>>>>>>>>>>>>>>>>>>>>> extraordinary commits, because the
>> third-party
>> > > > >> state
>> > > > >>>>>> store
>> > > > >>>>>>>>>>>>>>>> takes care
>> > > > >>>>>>>>>>>>>>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>>>>>>> memory. But in that case the method could
>> also
>> > > > >> just
>> > > > >>>>>> return
>> > > > >>>>>>>> 0.
>> > > > >>>>>>>>>>>>>>>> Even
>> > > > >>>>>>>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>>>>>> case would be better solved with a method
>> that
>> > > > >>>> returns
>> > > > >>>>>>>> whether
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>> state
>> > > > >>>>>>>>>>>>>>>>>>>>>> store manages itself the memory used for
>> > > > >> uncommitted
>> > > > >>>>>> bytes
>> > > > >>>>>>>> or
>> > > > >>>>>>>>>>>>>>>> not.
>> > > > >>>>>>>>>>>>>>>>>>>>>> Said that, I am fine with keeping the -1
>> return
>> > > > >>>> value,
>> > > > >>>>>> I was
>> > > > >>>>>>>>>>>>>>>> just
>> > > > >>>>>>>>>>>>>>>>>>>>>> wondering when and if it will be used.
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> Regarding returning 0 for transactional state
>> > > > >> stores
>> > > > >>>>>> when
>> > > > >>>>>>>> the
>> > > > >>>>>>>>>>>>>>>> batch
>> > > > >>>>>>>>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>>>> empty, I was just wondering because you
>> > > > >> explicitly
>> > > > >>>>>> stated
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> "or {@code 0} if this StateStore does not
>> > > > >> support
>> > > > >>>>>>>>>>>> transactions."
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> So it seemed to me returning 0 could only
>> > > > >> happen for
>> > > > >>>>>>>>>>>>>>>>>> non-transactional
>> > > > >>>>>>>>>>>>>>>>>>>>>> state stores.
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> 3.
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> a) What do you think if we move the isolation
>> > > > >> level
>> > > > >>>> to
>> > > > >>>>>> IQ
>> > > > >>>>>>>> (v1
>> > > > >>>>>>>>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>> v2)?
>> > > > >>>>>>>>>>>>>>>>>>>>>> In the end this is the only component that
>> > > > >> really
>> > > > >>>> needs
>> > > > >>>>>> to
>> > > > >>>>>>>>>>>>>>>> specify
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> isolation level. It is similar to the Kafka
>> > > > >> consumer
>> > > > >>>>>> that
>> > > > >>>>>>>> can
>> > > > >>>>>>>>>>>>>>>> choose
>> > > > >>>>>>>>>>>>>>>>>>>>>> with what isolation level to read the input
>> > > > >> topic.
>> > > > >>>>>>>>>>>>>>>>>>>>>> For IQv1 the isolation level should go into
>> > > > >>>>>>>>>>>>>>>> StoreQueryParameters. For
>> > > > >>>>>>>>>>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> b) Point a) raises the question what should
>> > > > >> happen
>> > > > >>>>>> during
>> > > > >>>>>>>>>>>>>>>>>> at-least-once
>> > > > >>>>>>>>>>>>>>>>>>>>>> processing when the state store does not use
>> > > > >>>>>> transactions?
>> > > > >>>>>>>>>> John
>> > > > >>>>>>>>>>>>>>>> in
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> past proposed to also use transactions on
>> state
>> > > > >>>> stores
>> > > > >>>>>> for
>> > > > >>>>>>>>>>>>>>>>>>>>>> at-least-once. I like that idea, because it
>> > > > >> avoids
>> > > > >>>>>>>> aggregating
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>> same
>> > > > >>>>>>>>>>>>>>>>>>>>>> records over and over again in the case of a
>> > > > >>>> failure. We
>> > > > >>>>>>>> had a
>> > > > >>>>>>>>>>>>>>>> case
>> > > > >>>>>>>>>>>>>>>>>> in
>> > > > >>>>>>>>>>>>>>>>>>>>>> the past where a Streams applications in
>> > > > >>>> at-least-once
>> > > > >>>>>> mode
>> > > > >>>>>>>>>> was
>> > > > >>>>>>>>>>>>>>>>>> failing
>> > > > >>>>>>>>>>>>>>>>>>>>>> continuously for some reasons I do not
>> remember
>> > > > >>>> before
>> > > > >>>>>>>>>>>>>>>> committing the
>> > > > >>>>>>>>>>>>>>>>>>>>>> offsets. After each failover, the app
>> aggregated
>> > > > >>>> again
>> > > > >>>>>> and
>> > > > >>>>>>>>>>>>>>>> again the
>> > > > >>>>>>>>>>>>>>>>>>>>>> same records. Of course the aggregate
>> increased
>> > > > >> to
>> > > > >>>> very
>> > > > >>>>>>>> wrong
>> > > > >>>>>>>>>>>>>>>> values
>> > > > >>>>>>>>>>>>>>>>>>>>>> just because of the failover. With
>> transactions
>> > > > >> on
>> > > > >>>> the
>> > > > >>>>>> state
>> > > > >>>>>>>>>>>>>>>> stores
>> > > > >>>>>>>>>>>>>>>>>> we
>> > > > >>>>>>>>>>>>>>>>>>>>>> could have avoided this. The app would have
>> > > > >> output
>> > > > >>>> the
>> > > > >>>>>> same
>> > > > >>>>>>>>>>>>>>>> aggregate
>> > > > >>>>>>>>>>>>>>>>>>>>>> multiple times (i.e., after each failover)
>> but
>> > > > >> at
>> > > > >>>> least
>> > > > >>>>>> the
>> > > > >>>>>>>>>>>>>>>> value of
>> > > > >>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> aggregate would not depend on the number of
>> > > > >>>> failovers.
>> > > > >>>>>>>>>>>>>>>> Outputting the
>> > > > >>>>>>>>>>>>>>>>>>>>>> same aggregate multiple times would be
>> incorrect
>> > > > >>>> under
>> > > > >>>>>>>>>>>>>>>> exactly-once
>> > > > >>>>>>>>>>>>>>>>>> but
>> > > > >>>>>>>>>>>>>>>>>>>>>> it is OK for at-least-once.
>> > > > >>>>>>>>>>>>>>>>>>>>>> If it makes sense to add a config to turn on
>> > > > >> and off
>> > > > >>>>>>>>>>>>>>>> transactions on
>> > > > >>>>>>>>>>>>>>>>>>>>>> state stores under at-least-once or just use
>> > > > >>>>>> transactions in
>> > > > >>>>>>>>>>>>>>>> any case
>> > > > >>>>>>>>>>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>>>> a question we should also discuss in this
>> KIP.
>> > > > >> It
>> > > > >>>>>> depends a
>> > > > >>>>>>>>>> bit
>> > > > >>>>>>>>>>>>>>>> on
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I
>> would
>> > > > >>>> add a
>> > > > >>>>>>>> config.
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> 4.
>> > > > >>>>>>>>>>>>>>>>>>>>>> Your points are all valid. I tend to say to
>> > > > >> keep the
>> > > > >>>>>> metrics
>> > > > >>>>>>>>>>>>>>>> around
>> > > > >>>>>>>>>>>>>>>>>>>>>> flush() until we remove flush() completely
>> from
>> > > > >> the
>> > > > >>>>>>>> interface.
>> > > > >>>>>>>>>>>>>>>> Calls
>> > > > >>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>> flush() might still exist since existing
>> > > > >> processors
>> > > > >>>>>> might
>> > > > >>>>>>>>>> still
>> > > > >>>>>>>>>>>>>>>> call
>> > > > >>>>>>>>>>>>>>>>>>>>>> flush() explicitly as you mentioned in 1).
>> For
>> > > > >>>> sure, we
>> > > > >>>>>> need
>> > > > >>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>> document
>> > > > >>>>>>>>>>>>>>>>>>>>>> how the metrics change due to the
>> transactions
>> > > > >> in
>> > > > >>>> the
>> > > > >>>>>>>> upgrade
>> > > > >>>>>>>>>>>>>>>> notes.
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> 5.
>> > > > >>>>>>>>>>>>>>>>>>>>>> I see. Then you should describe how the
>> > > > >> .position
>> > > > >>>> files
>> > > > >>>>>> are
>> > > > >>>>>>>>>>>>>>>> handled
>> > > > >>>>>>>>>>>>>>>>>> in
>> > > > >>>>>>>>>>>>>>>>>>>>>> a dedicated section of the KIP or incorporate
>> > > > >> the
>> > > > >>>>>>>> description
>> > > > >>>>>>>>>>>>>>>> in the
>> > > > >>>>>>>>>>>>>>>>>>>>>> "Atomic Checkpointing" section instead of
>> only
>> > > > >>>>>> mentioning it
>> > > > >>>>>>>>>> in
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> "Compatibility, 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:
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>
>> > > > >>>>>>
>> > > > >>>>
>> > > > >>
>> > >
>> >
>> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> It is a bit brittle, but it works.
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> Best,
>> > > > >>>>>>>>>>>>>>>>>>>>>> Bruno
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for taking the time to review the
>> KIP.
>> > > > >> I'm
>> > > > >>>> back
>> > > > >>>>>> from
>> > > > >>>>>>>>>>>>>>>> leave
>> > > > >>>>>>>>>>>>>>>>>> now
>> > > > >>>>>>>>>>>>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>>>>>>> intend to move this forwards as quickly as I
>> > > > >> can.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Addressing your points:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Because flush() is part of the StateStore
>> API,
>> > > > >> it's
>> > > > >>>>>> exposed
>> > > > >>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>> custom
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors, which might be making calls to
>> > > > >> flush().
>> > > > >>>>>> This
>> > > > >>>>>>>> was
>> > > > >>>>>>>>>>>>>>>>>> actually
>> > > > >>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>> case in a few integration tests.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> To maintain as much compatibility as
>> possible,
>> > > > >> I'd
>> > > > >>>>>> prefer
>> > > > >>>>>>>> not
>> > > > >>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>> make
>> > > > >>>>>>>>>>>>>>>>>>>>>> this
>> > > > >>>>>>>>>>>>>>>>>>>>>>> an UnsupportedOperationException, as it will
>> > > > >> cause
>> > > > >>>>>>>> previously
>> > > > >>>>>>>>>>>>>>>>>> working
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors to start throwing exceptions at
>> > > > >> runtime.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that it doesn't make sense for it to
>> > > > >> proxy
>> > > > >>>>>>>> commit(),
>> > > > >>>>>>>>>>>>>>>> though,
>> > > > >>>>>>>>>>>>>>>>>>>> as
>> > > > >>>>>>>>>>>>>>>>>>>>>>> that would cause it to violate the
>> "StateStores
>> > > > >>>> commit
>> > > > >>>>>> only
>> > > > >>>>>>>>>>>>>>>> when the
>> > > > >>>>>>>>>>>>>>>>>>>> Task
>> > > > >>>>>>>>>>>>>>>>>>>>>>> commits" rule.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Instead, I think we should make this a
>> no-op.
>> > > > >> That
>> > > > >>>> way,
>> > > > >>>>>>>>>>>>>>>> existing
>> > > > >>>>>>>>>>>>>>>>>> user
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors will continue to work as-before,
>> > > > >> without
>> > > > >>>>>>>> violation
>> > > > >>>>>>>>>>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>>> store
>> > > > >>>>>>>>>>>>>>>>>>>>>>> consistency that would be caused by
>> premature
>> > > > >>>>>> flush/commit
>> > > > >>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>>>>> StateStore
>> > > > >>>>>>>>>>>>>>>>>>>>>>> data to disk.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> What do you think?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
>> > > > >>>>>> implementation
>> > > > >>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, but is unable to estimate the
>> > > > >>>>>> uncommitted
>> > > > >>>>>>>>>>>> memory
>> > > > >>>>>>>>>>>>>>>>>> usage,
>> > > > >>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>> method will return -1.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> The intention here is to permit third-party
>> > > > >>>>>> implementations
>> > > > >>>>>>>>>>>>>>>> that may
>> > > > >>>>>>>>>>>>>>>>>>>> not
>> > > > >>>>>>>>>>>>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>>>>>>> able to estimate memory usage.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been
>> > > > >> written to
>> > > > >>>> the
>> > > > >>>>>>>> store
>> > > > >>>>>>>>>>>>>>>> yet. I
>> > > > >>>>>>>>>>>>>>>>>>>>>> thought
>> > > > >>>>>>>>>>>>>>>>>>>>>>> that was implied by "This method will
>> return an
>> > > > >>>>>>>> approximation
>> > > > >>>>>>>>>>>>>>>> of the
>> > > > >>>>>>>>>>>>>>>>>>>>>> memory
>> > > > >>>>>>>>>>>>>>>>>>>>>>> would be freed by the next call to {@link
>> > > > >>>>>> #commit(Map)}"
>> > > > >>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>> "@return
>> > > > >>>>>>>>>>>>>>>>>>>> The
>> > > > >>>>>>>>>>>>>>>>>>>>>>> approximate size of all records awaiting
>> {@link
>> > > > >>>>>>>>>>>> #commit(Map)}",
>> > > > >>>>>>>>>>>>>>>>>>>> however,
>> > > > >>>>>>>>>>>>>>>>>>>>>> I
>> > > > >>>>>>>>>>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you
>> > > > >> think
>> > > > >>>> this
>> > > > >>>>>> is
>> > > > >>>>>>>>>>>>>>>> unclear?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> I realise this is probably the most
>> contentious
>> > > > >>>> point
>> > > > >>>>>> in my
>> > > > >>>>>>>>>>>>>>>> design,
>> > > > >>>>>>>>>>>>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>>>>>> I'm
>> > > > >>>>>>>>>>>>>>>>>>>>>>> open to changing it if I'm unable to
>> convince
>> > > > >> you
>> > > > >>>> of
>> > > > >>>>>> the
>> > > > >>>>>>>>>>>>>>>> benefits.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Nevertheless, here's my argument:
>> > > > >>>>>>>>>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are
>> directly
>> > > > >>>> provided
>> > > > >>>>>>>>>>>>>>>> StateStores
>> > > > >>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>>> query, and it may be important for users to
>> > > > >>>>>>>> programmatically
>> > > > >>>>>>>>>>>>>>>> know
>> > > > >>>>>>>>>>>>>>>>>>>> which
>> > > > >>>>>>>>>>>>>>>>>>>>>>> mode the StateStore is operating under. If
>> we
>> > > > >>>> simply
>> > > > >>>>>>>> provide
>> > > > >>>>>>>>>>>> an
>> > > > >>>>>>>>>>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the
>> > > > >>>> internal
>> > > > >>>>>>>> streams
>> > > > >>>>>>>>>>>>>>>>>>>> engine), or
>> > > > >>>>>>>>>>>>>>>>>>>>>>> similar, then users will need to understand
>> the
>> > > > >>>>>> operation
>> > > > >>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>>>>>> consequences
>> > > > >>>>>>>>>>>>>>>>>>>>>>> of each available processing mode and how it
>> > > > >>>> pertains
>> > > > >>>>>> to
>> > > > >>>>>>>>>> their
>> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Interactive Query users aren't the only
>> people
>> > > > >> that
>> > > > >>>>>> care
>> > > > >>>>>>>>>> about
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>> processing.mode/IsolationLevel of a
>> StateStore:
>> > > > >>>>>>>> implementers
>> > > > >>>>>>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>>> custom
>> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStores also need to understand the
>> > > > >> behaviour
>> > > > >>>>>> expected
>> > > > >>>>>>>> of
>> > > > >>>>>>>>>>>>>>>> their
>> > > > >>>>>>>>>>>>>>>>>>>>>>> implementation. KIP-892 introduces some
>> > > > >> assumptions
>> > > > >>>>>> into
>> > > > >>>>>>>> the
>> > > > >>>>>>>>>>>>>>>> Streams
>> > > > >>>>>>>>>>>>>>>>>>>>>> Engine
>> > > > >>>>>>>>>>>>>>>>>>>>>>> about how StateStores operate under each
>> > > > >> processing
>> > > > >>>>>> mode,
>> > > > >>>>>>>> and
>> > > > >>>>>>>>>>>>>>>> it's
>> > > > >>>>>>>>>>>>>>>>>>>>>>> important that custom implementations
>> adhere to
>> > > > >>>> those
>> > > > >>>>>>>>>>>>>>>> assumptions in
>> > > > >>>>>>>>>>>>>>>>>>>>>> order
>> > > > >>>>>>>>>>>>>>>>>>>>>>> to maintain the consistency guarantees.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> IsolationLevels provide a high-level
>> contract
>> > > > >> on
>> > > > >>>> the
>> > > > >>>>>>>>>> behaviour
>> > > > >>>>>>>>>>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStore: a user knows that under
>> > > > >> READ_COMMITTED,
>> > > > >>>>>> they
>> > > > >>>>>>>> will
>> > > > >>>>>>>>>>>>>>>> see
>> > > > >>>>>>>>>>>>>>>>>>>> writes
>> > > > >>>>>>>>>>>>>>>>>>>>>>> only after the Task has committed, and under
>> > > > >>>>>>>> READ_UNCOMMITTED
>> > > > >>>>>>>>>>>>>>>> they
>> > > > >>>>>>>>>>>>>>>>>>>> will
>> > > > >>>>>>>>>>>>>>>>>>>>>> see
>> > > > >>>>>>>>>>>>>>>>>>>>>>> writes immediately. No understanding of the
>> > > > >>>> details of
>> > > > >>>>>> each
>> > > > >>>>>>>>>>>>>>>>>>>>>> processing.mode
>> > > > >>>>>>>>>>>>>>>>>>>>>>> is required, either for IQ users or
>> StateStore
>> > > > >>>>>>>> implementers.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> An argument can be made that these
>> contractual
>> > > > >>>>>> guarantees
>> > > > >>>>>>>> can
>> > > > >>>>>>>>>>>>>>>> simply
>> > > > >>>>>>>>>>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>>>>>>> documented for the processing.mode (i.e.
>> that
>> > > > >>>>>> exactly-once
>> > > > >>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED
>> and
>> > > > >>>>>>>> at-least-once
>> > > > >>>>>>>>>>>>>>>> behaves
>> > > > >>>>>>>>>>>>>>>>>>>> like
>> > > > >>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several
>> small
>> > > > >>>> issues
>> > > > >>>>>> with
>> > > > >>>>>>>>>>>>>>>> this I'd
>> > > > >>>>>>>>>>>>>>>>>>>>>> prefer
>> > > > >>>>>>>>>>>>>>>>>>>>>>> to avoid:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          - Where would we document these
>> > > > >> contracts,
>> > > > >>>> in
>> > > > >>>>>> a way
>> > > > >>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>> difficult
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          for users/implementers to
>> miss/ignore?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It's not clear to users that the
>> > > > >>>> processing
>> > > > >>>>>> mode
>> > > > >>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>> communicating
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          an expectation of read isolation,
>> > > > >> unless
>> > > > >>>> they
>> > > > >>>>>> read
>> > > > >>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> documentation. Users
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          rarely consult documentation unless
>> > > > >> they
>> > > > >>>> feel
>> > > > >>>>>> they
>> > > > >>>>>>>>>> need
>> > > > >>>>>>>>>>>>>>>> to, so
>> > > > >>>>>>>>>>>>>>>>>>>> it's
>> > > > >>>>>>>>>>>>>>>>>>>>>> likely
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          this detail would get missed by
>> many
>> > > > >> users.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It tightly couples processing
>> modes
>> > > > >> to
>> > > > >>>> read
>> > > > >>>>>>>>>> isolation.
>> > > > >>>>>>>>>>>>>>>> Adding
>> > > > >>>>>>>>>>>>>>>>>>>> new
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          processing modes, or changing the
>> read
>> > > > >>>>>> isolation of
>> > > > >>>>>>>>>>>>>>>> existing
>> > > > >>>>>>>>>>>>>>>>>>>>>> processing
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          modes would be
>> difficult/impossible.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Ultimately, the cost of introducing
>> > > > >>>> IsolationLevels is
>> > > > >>>>>>>> just a
>> > > > >>>>>>>>>>>>>>>> single
>> > > > >>>>>>>>>>>>>>>>>>>>>>> method, since we re-use the existing
>> > > > >> IsolationLevel
>> > > > >>>>>> enum
>> > > > >>>>>>>> from
>> > > > >>>>>>>>>>>>>>>> Kafka.
>> > > > >>>>>>>>>>>>>>>>>>>> This
>> > > > >>>>>>>>>>>>>>>>>>>>>>> gives us a clear place to document the
>> > > > >> contractual
>> > > > >>>>>>>> guarantees
>> > > > >>>>>>>>>>>>>>>>>> expected
>> > > > >>>>>>>>>>>>>>>>>>>>>>> of/provided by StateStores, that is
>> accessible
>> > > > >>>> both by
>> > > > >>>>>> the
>> > > > >>>>>>>>>>>>>>>>>> StateStore
>> > > > >>>>>>>>>>>>>>>>>>>>>>> itself, and by IQ users.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> (Writing this I've just realised that the
>> > > > >>>> StateStore
>> > > > >>>>>> and IQ
>> > > > >>>>>>>>>>>>>>>> APIs
>> > > > >>>>>>>>>>>>>>>>>>>> actually
>> > > > >>>>>>>>>>>>>>>>>>>>>>> don't provide access to StateStoreContext
>> that
>> > > > >> IQ
>> > > > >>>> users
>> > > > >>>>>>>> would
>> > > > >>>>>>>>>>>>>>>> have
>> > > > >>>>>>>>>>>>>>>>>>>> direct
>> > > > >>>>>>>>>>>>>>>>>>>>>>> access to... Perhaps StateStore should
>> expose
>> > > > >>>>>>>>>> isolationLevel()
>> > > > >>>>>>>>>>>>>>>>>> itself
>> > > > >>>>>>>>>>>>>>>>>>>>>> too?)
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> 4.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the
>> metrics
>> > > > >>>> in-place
>> > > > >>>>>>>>>>>>>>>> either, as
>> > > > >>>>>>>>>>>>>>>>>>>> it's a
>> > > > >>>>>>>>>>>>>>>>>>>>>>> backwards incompatible change. My concern is
>> > > > >> that,
>> > > > >>>> if
>> > > > >>>>>> we
>> > > > >>>>>>>>>> leave
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> existing
>> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics in place, they will be
>> > > > >> confusing to
>> > > > >>>>>> users.
>> > > > >>>>>>>>>>>>>>>> Right
>> > > > >>>>>>>>>>>>>>>>>> now,
>> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics record explicit flushes to
>> > > > >> disk,
>> > > > >>>> but
>> > > > >>>>>> under
>> > > > >>>>>>>>>>>>>>>> KIP-892,
>> > > > >>>>>>>>>>>>>>>>>>>> even
>> > > > >>>>>>>>>>>>>>>>>>>>>> a
>> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() will not explicitly flush data to
>> > > > >> disk -
>> > > > >>>>>> RocksDB
>> > > > >>>>>>>>>> will
>> > > > >>>>>>>>>>>>>>>>>> decide
>> > > > >>>>>>>>>>>>>>>>>>>> on
>> > > > >>>>>>>>>>>>>>>>>>>>>>> when to flush memtables to disk itself.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> If we keep the existing "flush" metrics,
>> we'd
>> > > > >> have
>> > > > >>>> two
>> > > > >>>>>>>>>>>> options,
>> > > > >>>>>>>>>>>>>>>>>> which
>> > > > >>>>>>>>>>>>>>>>>>>>>> both
>> > > > >>>>>>>>>>>>>>>>>>>>>>> seem pretty bad to me:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          1. Have them record calls to
>> commit(),
>> > > > >>>> which
>> > > > >>>>>> would
>> > > > >>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>>>> misleading, as
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          data is no longer explicitly
>> "flushed"
>> > > > >> to
>> > > > >>>> disk
>> > > > >>>>>> by
>> > > > >>>>>>>> this
>> > > > >>>>>>>>>>>>>>>> call.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          2. Have them record nothing at all,
>> > > > >> which
>> > > > >>>> is
>> > > > >>>>>>>>>> equivalent
>> > > > >>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>> removing
>> > > > >>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          metrics, except that users will see
>> > the
>> > > > >>>> metric
>> > > > >>>>>>>> still
>> > > > >>>>>>>>>>>>>>>> exists and
>> > > > >>>>>>>>>>>>>>>>>>>> so
>> > > > >>>>>>>>>>>>>>>>>>>>>> assume
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          that the metric is correct, and
>> that
>> > > > >>>> there's a
>> > > > >>>>>>>> problem
>> > > > >>>>>>>>>>>>>>>> with
>> > > > >>>>>>>>>>>>>>>>>> their
>> > > > >>>>>>>>>>>>>>>>>>>>>> system
>> > > > >>>>>>>>>>>>>>>>>>>>>>>          when there isn't.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that removing them is also a bad
>> > > > >> solution,
>> > > > >>>> and
>> > > > >>>>>> I'd
>> > > > >>>>>>>>>>>>>>>> like some
>> > > > >>>>>>>>>>>>>>>>>>>>>>> guidance on the best path forward here.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> 5.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Position files are updated on every write
>> to a
>> > > > >>>>>> StateStore.
>> > > > >>>>>>>>>>>>>>>> Since our
>> > > > >>>>>>>>>>>>>>>>>>>>>> writes
>> > > > >>>>>>>>>>>>>>>>>>>>>>> are now buffered until commit(), we can't
>> > > > >> update
>> > > > >>>> the
>> > > > >>>>>>>> Position
>> > > > >>>>>>>>>>>>>>>> file
>> > > > >>>>>>>>>>>>>>>>>>>> until
>> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() has been called, otherwise it
>> would be
>> > > > >>>>>>>> inconsistent
>> > > > >>>>>>>>>>>>>>>> with
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> data
>> > > > >>>>>>>>>>>>>>>>>>>>>>> in the event of a rollback. Consequently, we
>> > > > >> need
>> > > > >>>> to
>> > > > >>>>>> manage
>> > > > >>>>>>>>>>>>>>>> these
>> > > > >>>>>>>>>>>>>>>>>>>> offsets
>> > > > >>>>>>>>>>>>>>>>>>>>>>> the same way we manage the checkpoint
>> offsets,
>> > > > >> and
>> > > > >>>>>> ensure
>> > > > >>>>>>>>>>>>>>>> they're
>> > > > >>>>>>>>>>>>>>>>>> only
>> > > > >>>>>>>>>>>>>>>>>>>>>>> written on commit().
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> 6.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet
>> what
>> > > > >>>> tests to
>> > > > >>>>>>>>>> write.
>> > > > >>>>>>>>>>>>>>>> How
>> > > > >>>>>>>>>>>>>>>>>>>>>> explicit
>> > > > >>>>>>>>>>>>>>>>>>>>>>> do we need to be here in the KIP?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is
>> designed
>> > > > >> to be
>> > > > >>>>>>>> seamless,
>> > > > >>>>>>>>>>>>>>>> and we
>> > > > >>>>>>>>>>>>>>>>>>>>>> should
>> > > > >>>>>>>>>>>>>>>>>>>>>>> definitely add some tests around that.
>> > > > >> Downgrade,
>> > > > >>>> it
>> > > > >>>>>>>>>>>>>>>> transpires,
>> > > > >>>>>>>>>>>>>>>>>> isn't
>> > > > >>>>>>>>>>>>>>>>>>>>>>> currently possible, as the extra column
>> family
>> > > > >> for
>> > > > >>>>>> offset
>> > > > >>>>>>>>>>>>>>>> storage is
>> > > > >>>>>>>>>>>>>>>>>>>>>>> incompatible with the pre-KIP-892
>> > > > >> implementation:
>> > > > >>>> when
>> > > > >>>>>> you
>> > > > >>>>>>>>>>>>>>>> open a
>> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
>> > > > >>>>>>>>>>>>>>>>>>>>>>> database, you must open all available column
>> > > > >>>> families
>> > > > >>>>>> or
>> > > > >>>>>>>>>>>>>>>> receive an
>> > > > >>>>>>>>>>>>>>>>>>>>>> error.
>> > > > >>>>>>>>>>>>>>>>>>>>>>> What currently happens on downgrade is that
>> it
>> > > > >>>>>> attempts to
>> > > > >>>>>>>>>>>>>>>> open the
>> > > > >>>>>>>>>>>>>>>>>>>>>> store,
>> > > > >>>>>>>>>>>>>>>>>>>>>>> throws an error about the offsets column
>> > > > >> family not
>> > > > >>>>>> being
>> > > > >>>>>>>>>>>>>>>> opened,
>> > > > >>>>>>>>>>>>>>>>>>>> which
>> > > > >>>>>>>>>>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task.
>> Given
>> > > > >> that
>> > > > >>>>>>>>>> downgrades
>> > > > >>>>>>>>>>>>>>>>>> should
>> > > > >>>>>>>>>>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>>>>>>> uncommon, I think this is acceptable
>> > > > >> behaviour, as
>> > > > >>>> the
>> > > > >>>>>>>>>>>>>>>> end-state is
>> > > > >>>>>>>>>>>>>>>>>>>>>>> consistent, even if it results in an
>> > > > >> undesirable
>> > > > >>>> state
>> > > > >>>>>>>>>>>> restore.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Should I document the upgrade/downgrade
>> > > > >> behaviour
>> > > > >>>>>>>> explicitly
>> > > > >>>>>>>>>>>>>>>> in the
>> > > > >>>>>>>>>>>>>>>>>>>> KIP?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> --
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Regards,
>> > > > >>>>>>>>>>>>>>>>>>>>>>> Nick
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna
>> <
>> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
>> > > > >>>>>>>>>>>>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Nick!
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates!
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> Why does StateStore#flush() default to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> Since calls to flush() will not exist
>> anymore
>> > > > >>>> after
>> > > > >>>>>> this
>> > > > >>>>>>>> KIP
>> > > > >>>>>>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> released, I would rather throw an
>> unsupported
>> > > > >>>>>> operation
>> > > > >>>>>>>>>>>>>>>> exception
>> > > > >>>>>>>>>>>>>>>>>> by
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> default.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> When would a state store return -1 from
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
>> > > > >> while
>> > > > >>>>>> being
>> > > > >>>>>>>>>>>>>>>>>>>> transactional?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> Wouldn't
>> > > > >>>> StateStore#approximateNumUncommittedBytes()
>> > > > >>>>>> also
>> > > > >>>>>>>>>>>>>>>> return 0
>> > > > >>>>>>>>>>>>>>>>>> if
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> the state store is transactional but
>> nothing
>> > > > >> has
>> > > > >>>> been
>> > > > >>>>>>>>>> written
>> > > > >>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> state store yet?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this
>> > > > >> KIP
>> > > > >>>> really
>> > > > >>>>>>>> need
>> > > > >>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>> introduce
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
>> > > > >>>> StateStoreContext
>> > > > >>>>>> has
>> > > > >>>>>>>>>>>>>>>> already
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> appConfigs() which basically exposes the
>> same
>> > > > >>>>>> information,
>> > > > >>>>>>>>>>>>>>>> i.e., if
>> > > > >>>>>>>>>>>>>>>>>>>> EOS
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> is enabled or not.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore
>> > > > >>>> interface
>> > > > >>>>>> as
>> > > > >>>>>>>>>>>>>>>> loosely
>> > > > >>>>>>>>>>>>>>>>>>>> coupled
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to
>> give
>> > > > >>>>>> implementers
>> > > > >>>>>>>>>>>> more
>> > > > >>>>>>>>>>>>>>>>>>>> freedom,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> and reduce the amount of internal knowledge
>> > > > >>>> required."
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> While I understand the intent, I doubt
>> that it
>> > > > >>>>>> decreases
>> > > > >>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>> coupling of
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> a StateStore interface and the Streams
>> engine.
>> > > > >>>>>>>>>> READ_COMMITTED
>> > > > >>>>>>>>>>>>>>>> only
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> applies to IQ but not to reads by
>> processors.
>> > > > >>>> Thus,
>> > > > >>>>>>>>>>>>>>>> implementers
>> > > > >>>>>>>>>>>>>>>>>>>> need to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand how Streams accesses the state
>> > > > >> stores.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> I would like to hear what others think
>> about
>> > > > >> this.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> 4.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> Great exposing new metrics for
>> transactional
>> > > > >> state
>> > > > >>>>>> stores!
>> > > > >>>>>>>>>>>>>>>>>> However, I
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> would prefer to add new metrics and
>> deprecate
>> > > > >> (in
>> > > > >>>> the
>> > > > >>>>>>>> docs)
>> > > > >>>>>>>>>>>>>>>> the 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 also need to adapt/add
>> > > > >> integration
>> > > > >>>>>> tests
>> > > > >>>>>>>>>>>>>>>> besides
>> > > > >>>>>>>>>>>>>>>>>> unit
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> tests. Additionally, we probably need
>> > > > >> integration
>> > > > >>>> or
>> > > > >>>>>>>> system
>> > > > >>>>>>>>>>>>>>>> tests
>> > > > >>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> verify that upgrades and downgrades between
>> > > > >>>>>> transactional
>> > > > >>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> non-transactional state stores work as
>> > > > >> expected.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> Best,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> Bruno
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion
>> in
>> > > > >> the
>> > > > >>>> KIP,
>> > > > >>>>>>>> under
>> > > > >>>>>>>>>>>>>>>>>>>> "Rejected
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea
>> > > > >> worth
>> > > > >>>>>> pursuing,
>> > > > >>>>>>>>>>>>>>>> but I
>> > > > >>>>>>>>>>>>>>>>>>>> believe
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>> that it's out of the scope of this KIP,
>> > > > >> because
>> > > > >>>> it
>> > > > >>>>>>>> solves a
>> > > > >>>>>>>>>>>>>>>>>>>> different
>> > > > >>>>>>>>>>>>>>>>>>>>>> set
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of
>> > > > >> this
>> > > > >>>> one
>> > > > >>>>>> has
>> > > > >>>>>>>>>>>>>>>> already
>> > > > >>>>>>>>>>>>>>>>>> grown
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> quite
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>> large!
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick
>> Telford <
>> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP (
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>
>> > > > >>>>>>
>> > > > >>>>
>> > > > >>
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> )
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing
>> > > > >> back
>> > > > >>>>>> "Atomic
>> > > > >>>>>>>>>>>>>>>>>>>> Checkpointing"
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> (for
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think
>> > > > >> the one
>> > > > >>>>>> thing
>> > > > >>>>>>>>>>>>>>>> missing is
>> > > > >>>>>>>>>>>>>>>>>>>> some
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> changes to metrics (notably the store
>> > > > >> "flush"
>> > > > >>>>>> metrics
>> > > > >>>>>>>> will
>> > > > >>>>>>>>>>>>>>>> need
>> > > > >>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> renamed to "commit").
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> The reason I brought back Atomic
>> > > > >> Checkpointing
>> > > > >>>> was
>> > > > >>>>>> to
>> > > > >>>>>>>>>>>>>>>> decouple
>> > > > >>>>>>>>>>>>>>>>>>>> store
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> flush
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from store commit. This is important,
>> > > > >> because
>> > > > >>>> with
>> > > > >>>>>>>>>>>>>>>> Transactional
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush"
>> on
>> > > > >>>> *every*
>> > > > >>>>>> Task
>> > > > >>>>>>>>>>>>>>>> commit,
>> > > > >>>>>>>>>>>>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>>>>>> not
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> just when the StateStore is closing,
>> > > > >> otherwise
>> > > > >>>> our
>> > > > >>>>>>>>>>>>>>>> transaction
>> > > > >>>>>>>>>>>>>>>>>>>> buffer
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> will
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> never be written and persisted, instead
>> > > > >> growing
>> > > > >>>>>>>> unbounded!
>> > > > >>>>>>>>>>>> I
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> experimented
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with some simple solutions, like forcing
>> a
>> > > > >> store
>> > > > >>>>>> flush
>> > > > >>>>>>>>>>>>>>>> whenever
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed
>> its
>> > > > >>>>>> configured
>> > > > >>>>>>>>>>>>>>>> size, but
>> > > > >>>>>>>>>>>>>>>>>>>> this
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> was
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> brittle: it prevented the transaction
>> buffer
>> > > > >>>> from
>> > > > >>>>>> being
>> > > > >>>>>>>>>>>>>>>>>> configured
>> > > > >>>>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> unbounded, and it still would have
>> required
>> > > > >>>> explicit
>> > > > >>>>>>>>>>>>>>>> flushes of
>> > > > >>>>>>>>>>>>>>>>>>>>>> RocksDB,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yielding sub-optimal performance and
>> memory
>> > > > >>>>>> utilization.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the
>> > > > >> "right"
>> > > > >>>> way
>> > > > >>>>>> to
>> > > > >>>>>>>>>>>>>>>> resolve
>> > > > >>>>>>>>>>>>>>>>>> this
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog
>> > > > >> offsets
>> > > > >>>> that
>> > > > >>>>>>>>>>>>>>>> correspond
>> > > > >>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> most
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> recently written records are always
>> > > > >> atomically
>> > > > >>>>>> written
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (by writing them to the same transaction
>> > > > >>>> buffer),
>> > > > >>>>>> we can
>> > > > >>>>>>>>>>>>>>>> avoid
>> > > > >>>>>>>>>>>>>>>>>>>>>> forcibly
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk,
>> > > > >> letting
>> > > > >>>>>> RocksDB
>> > > > >>>>>>>>>>>>>>>> flush
>> > > > >>>>>>>>>>>>>>>>>> them
>> > > > >>>>>>>>>>>>>>>>>>>>>> only
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when necessary, without losing any of our
>> > > > >>>>>> consistency
>> > > > >>>>>>>>>>>>>>>> guarantees.
>> > > > >>>>>>>>>>>>>>>>>>>> See
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> updated KIP for more info.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I have fully implemented these changes,
>> > > > >>>> although I'm
>> > > > >>>>>>>> still
>> > > > >>>>>>>>>>>>>>>> not
>> > > > >>>>>>>>>>>>>>>>>>>>>> entirely
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> happy with the implementation for
>> segmented
>> > > > >>>>>> StateStores,
>> > > > >>>>>>>>>> so
>> > > > >>>>>>>>>>>>>>>> I
>> > > > >>>>>>>>>>>>>>>>>> plan
>> > > > >>>>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests
>> > > > >> pass. If
>> > > > >>>>>> you'd
>> > > > >>>>>>>> like
>> > > > >>>>>>>>>>>>>>>> to try
>> > > > >>>>>>>>>>>>>>>>>>>> out
>> > > > >>>>>>>>>>>>>>>>>>>>>> or
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> review this highly experimental and
>> > > > >> incomplete
>> > > > >>>>>> branch,
>> > > > >>>>>>>>>> it's
>> > > > >>>>>>>>>>>>>>>>>>>> available
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> here:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
>> > > > >>>>>>>> .
>> > > > >>>>>>>>>>>>>>>> Note:
>> > > > >>>>>>>>>>>>>>>>>>>> it's
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> built
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a
>> stable
>> > > > >> base
>> > > > >>>> to
>> > > > >>>>>> build
>> > > > >>>>>>>>>>>>>>>> and test
>> > > > >>>>>>>>>>>>>>>>>>>> it
>> > > > >>>>>>>>>>>>>>>>>>>>>> on,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples
>> > > > >> comparisons
>> > > > >>>> in a
>> > > > >>>>>>>> live
>> > > > >>>>>>>>>>>>>>>>>>>>>> environment. I
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's
>> > > > >> nearer
>> > > > >>>>>>>>>> completion
>> > > > >>>>>>>>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>> has
>> > > > >>>>>>>>>>>>>>>>>>>>>> been
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> proven on our main application.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I would really appreciate help in
>> reviewing
>> > > > >> and
>> > > > >>>>>> testing:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and
>> Window)
>> > > > >>>> stores
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Global stores
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> As I do not currently use either of
>> these,
>> > > > >> so my
>> > > > >>>>>> primary
>> > > > >>>>>>>>>>>>>>>> test
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> environment
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> doesn't test these areas.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next
>> > > > >> week
>> > > > >>>> for
>> > > > >>>>>> a few
>> > > > >>>>>>>>>>>>>>>> weeks,
>> > > > >>>>>>>>>>>>>>>>>> so
>> > > > >>>>>>>>>>>>>>>>>>>>>> will
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> not have time to move this forward until
>> > > > >> late
>> > > > >>>>>> August.
>> > > > >>>>>>>> That
>> > > > >>>>>>>>>>>>>>>> said,
>> > > > >>>>>>>>>>>>>>>>>>>> your
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I
>> just
>> > > > >>>> won't be
>> > > > >>>>>>>> able
>> > > > >>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>> respond
>> > > > >>>>>>>>>>>>>>>>>>>>>> as
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> quickly as usual.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick
>> Telford <
>> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact
>> > > > >> on IQ
>> > > > >>>> is
>> > > > >>>>>> not
>> > > > >>>>>>>>>>>>>>>> something
>> > > > >>>>>>>>>>>>>>>>>> I
>> > > > >>>>>>>>>>>>>>>>>>>> had
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> considered.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the state
>> > > > >> store
>> > > > >>>>>> from the
>> > > > >>>>>>>>>>>>>>>>>>>> transaction
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and
>> writing
>> > > > >> the
>> > > > >>>>>>>> checkpoint
>> > > > >>>>>>>>>>>>>>>> (thus,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount
>> of
>> > > > >> data
>> > > > >>>>>> and/or
>> > > > >>>>>>>>>>>>>>>> number of
>> > > > >>>>>>>>>>>>>>>>>>>>>> commit
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you
>> > > > >> suggesting
>> > > > >>>>>> that we
>> > > > >>>>>>>>>>>>>>>> add an
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> additional
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> config for the max number of commit
>> > > > >> intervals
>> > > > >>>>>> between
>> > > > >>>>>>>>>>>>>>>>>> checkpoints?
>> > > > >>>>>>>>>>>>>>>>>>>>>> That
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when
>> the
>> > > > >>>>>> transaction
>> > > > >>>>>>>>>>>>>>>> buffers
>> > > > >>>>>>>>>>>>>>>>>> are
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> nearly
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of
>> > > > >> commit
>> > > > >>>>>>>> intervals
>> > > > >>>>>>>>>>>>>>>> have
>> > > > >>>>>>>>>>>>>>>>>>>>>> elapsed,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> whichever comes first?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> That certainly seems reasonable,
>> although
>> > > > >> this
>> > > > >>>>>>>> re-ignites
>> > > > >>>>>>>>>>>>>>>> an
>> > > > >>>>>>>>>>>>>>>>>>>> earlier
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> debate about whether a config should be
>> > > > >>>> measured in
>> > > > >>>>>>>>>>>>>>>> "number of
>> > > > >>>>>>>>>>>>>>>>>>>> commit
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute
>> > > > >> time.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the
>> > > > >> reason
>> > > > >>>> I
>> > > > >>>>>> was
>> > > > >>>>>>>>>>>>>>>> pursuing
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> Atomic
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable
>> > > > >> flush
>> > > > >>>> from
>> > > > >>>>>>>>>>>>>>>> checkpointing,
>> > > > >>>>>>>>>>>>>>>>>>>>>> which
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every
>> > > > >> commit
>> > > > >>>>>> without
>> > > > >>>>>>>> any
>> > > > >>>>>>>>>>>>>>>>>>>> performance
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is
>> definitely
>> > > > >> the
>> > > > >>>>>> "best"
>> > > > >>>>>>>>>>>>>>>> solution,
>> > > > >>>>>>>>>>>>>>>>>>>> but
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm not
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back
>> > > > >> into
>> > > > >>>> this
>> > > > >>>>>> KIP.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
>> > > > >>>>>> transactional
>> > > > >>>>>>>>>>>> logic
>> > > > >>>>>>>>>>>>>>>>>>>> directly
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> into
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away
>> with
>> > > > >> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore#newTransaction
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> method, and reduces the number of new
>> > > > >> classes
>> > > > >>>>>>>> introduced,
>> > > > >>>>>>>>>>>>>>>>>>>>>> significantly
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reducing the complexity. If it works,
>> and
>> > > > >> the
>> > > > >>>>>>>> complexity
>> > > > >>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>>>> drastically
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic
>> > > > >>>> Checkpoints
>> > > > >>>>>>>> into
>> > > > >>>>>>>>>>>>>>>> this
>> > > > >>>>>>>>>>>>>>>>>> KIP.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno
>> Cadonna
>> > > > >> <
>> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
>> > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very
>> interesting!
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> As far as I understand, you want to
>> > > > >> atomically
>> > > > >>>>>> update
>> > > > >>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>> state
>> > > > >>>>>>>>>>>>>>>>>>>>>> store
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the
>> > > > >>>> memtable
>> > > > >>>>>> of a
>> > > > >>>>>>>>>>>> state
>> > > > >>>>>>>>>>>>>>>>>> store
>> > > > >>>>>>>>>>>>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write the checkpoint not after the
>> commit
>> > > > >> time
>> > > > >>>>>> elapsed
>> > > > >>>>>>>>>>>> but
>> > > > >>>>>>>>>>>>>>>>>> after
>> > > > >>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that
>> > > > >> would
>> > > > >>>> lead
>> > > > >>>>>> to
>> > > > >>>>>>>>>>>>>>>> exceeding
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
>> > > > >>>> before the
>> > > > >>>>>>>> next
>> > > > >>>>>>>>>>>>>>>> commit
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> interval
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> ends.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would
>> > > > >> commit
>> > > > >>>>>> every
>> > > > >>>>>>>>>>>>>>>> commit
>> > > > >>>>>>>>>>>>>>>>>>>> interval
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> but
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store will only be atomically
>> > > > >>>> updated
>> > > > >>>>>>>> roughly
>> > > > >>>>>>>>>>>>>>>> every
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> statestore.transaction.buffer.max.bytes of
>> > > > >>>> data.
>> > > > >>>>>> Also
>> > > > >>>>>>>> IQ
>> > > > >>>>>>>>>>>>>>>> would
>> > > > >>>>>>>>>>>>>>>>>>>> then
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> only
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see new data roughly every
>> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> After a failure the state store needs
>> to
>> > > > >>>> restore
>> > > > >>>>>> up to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> statestore.transaction.buffer.max.bytes.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Is this correct?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the
>> state
>> > > > >> store
>> > > > >>>>>> from
>> > > > >>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>> transaction
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and
>> writing
>> > > > >> the
>> > > > >>>>>>>> checkpoint
>> > > > >>>>>>>>>>>>>>>> (thus,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount
>> of
>> > > > >> data
>> > > > >>>>>> and/or
>> > > > >>>>>>>>>>>>>>>> number of
>> > > > >>>>>>>>>>>>>>>>>>>>>> commit
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have
>> > > > >> the
>> > > > >>>> same
>> > > > >>>>>> delay
>> > > > >>>>>>>>>>>> for
>> > > > >>>>>>>>>>>>>>>>>>>> records
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ
>> because
>> > > > >> both
>> > > > >>>>>> would
>> > > > >>>>>>>>>>>>>>>> appear
>> > > > >>>>>>>>>>>>>>>>>> when
>> > > > >>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka transaction is committed.
>> However,
>> > > > >>>> after a
>> > > > >>>>>>>> failure
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>> state
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> still needs to restore up to
>> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it might restore data that is already
>> in
>> > > > >> the
>> > > > >>>> state
>> > > > >>>>>>>> store
>> > > > >>>>>>>>>>>>>>>>>> because
>> > > > >>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable
>> > > > >> offset
>> > > > >>>>>> (i.e.
>> > > > >>>>>>>> the
>> > > > >>>>>>>>>>>>>>>> last
>> > > > >>>>>>>>>>>>>>>>>>>>>> committed
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset) of the changelog topics.
>> Restoring
>> > > > >>>> data
>> > > > >>>>>> that
>> > > > >>>>>>>> is
>> > > > >>>>>>>>>>>>>>>> already
>> > > > >>>>>>>>>>>>>>>>>>>> in
>> > > > >>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos
>> should
>> > > > >> not
>> > > > >>>>>> violated.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> This solution needs at least one new
>> > > > >> config to
>> > > > >>>>>> specify
>> > > > >>>>>>>>>>>>>>>> when a
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpoint
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> should be written.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> A small correction to your previous
>> e-mail
>> > > > >>>> that
>> > > > >>>>>> does
>> > > > >>>>>>>> not
>> > > > >>>>>>>>>>>>>>>> change
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> anything
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit
>> > > > >>>> interval
>> > > > >>>>>> is 30
>> > > > >>>>>>>>>>>>>>>> seconds,
>> > > > >>>>>>>>>>>>>>>>>>>> not
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> five
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> seconds.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've begun performance testing my
>> branch
>> > > > >> on
>> > > > >>>> our
>> > > > >>>>>>>> staging
>> > > > >>>>>>>>>>>>>>>>>>>>>> environment,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
>> > > > >>>> non-trivial
>> > > > >>>>>>>>>>>>>>>> application.
>> > > > >>>>>>>>>>>>>>>>>> I'm
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> already
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing the same increased flush
>> rate
>> > > > >> that
>> > > > >>>> we
>> > > > >>>>>> saw
>> > > > >>>>>>>> the
>> > > > >>>>>>>>>>>>>>>> last
>> > > > >>>>>>>>>>>>>>>>>>>> time
>> > > > >>>>>>>>>>>>>>>>>>>>>> we
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> attempted to use a version of this
>> KIP,
>> > > > >> but
>> > > > >>>> this
>> > > > >>>>>>>> time,
>> > > > >>>>>>>>>> I
>> > > > >>>>>>>>>>>>>>>>>> think I
>> > > > >>>>>>>>>>>>>>>>>>>>>> know
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> why.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit,
>> > > > >> which is
>> > > > >>>>>> called
>> > > > >>>>>>>> at
>> > > > >>>>>>>>>>>>>>>> the end
>> > > > >>>>>>>>>>>>>>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Task
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit process, has the following
>> > > > >> behaviour:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under ALOS: checkpoint
>> the
>> > > > >> state
>> > > > >>>>>> stores.
>> > > > >>>>>>>>>> This
>> > > > >>>>>>>>>>>>>>>>>> includes
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            flushing memtables in
>> RocksDB.
>> > > > >>>> This is
>> > > > >>>>>>>>>>>> acceptable
>> > > > >>>>>>>>>>>>>>>>>>>> because the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> default
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit.interval.ms is 5
>> > > > >> seconds,
>> > > > >>>> so
>> > > > >>>>>>>> forcibly
>> > > > >>>>>>>>>>>>>>>> flushing
>> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> every 5
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            seconds is acceptable for
>> most
>> > > > >>>>>>>> applications.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under EOS: checkpointing
>> is
>> > > > >> not
>> > > > >>>> done,
>> > > > >>>>>>>>>> *unless*
>> > > > >>>>>>>>>>>>>>>> it's
>> > > > >>>>>>>>>>>>>>>>>>>> being
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> forced, due
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to e.g. the Task closing or
>> > > > >> being
>> > > > >>>>>> revoked.
>> > > > >>>>>>>>>> This
>> > > > >>>>>>>>>>>>>>>> means
>> > > > >>>>>>>>>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>>>>>> under
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> normal
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing conditions, the
>> > > > >> state
>> > > > >>>> stores
>> > > > >>>>>>>> will
>> > > > >>>>>>>>>> not
>> > > > >>>>>>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpointed,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            not have memtables flushed
>> at
>> > > > >> all ,
>> > > > >>>>>> unless
>> > > > >>>>>>>>>>>> RocksDB
>> > > > >>>>>>>>>>>>>>>>>>>> decides to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush them on
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            its own. Checkpointing
>> stores
>> > > > >> and
>> > > > >>>>>>>>>> force-flushing
>> > > > >>>>>>>>>>>>>>>> their
>> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            done when a Task is being
>> > > > >> closed.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint
>> > > > >>>> stores on
>> > > > >>>>>> at
>> > > > >>>>>>>>>>>> least
>> > > > >>>>>>>>>>>>>>>>>> *some*
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> normal
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the
>> > > > >> RocksDB
>> > > > >>>>>>>> transaction
>> > > > >>>>>>>>>>>>>>>>>> buffers
>> > > > >>>>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the
>> offsets
>> > > > >> are
>> > > > >>>>>> synced to
>> > > > >>>>>>>>>>>>>>>> disk to
>> > > > >>>>>>>>>>>>>>>>>>>>>> prevent
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
>> > > > >>>> Consequently,
>> > > > >>>>>> my
>> > > > >>>>>>>>>>>>>>>> current
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task
>> > > > >> commit,
>> > > > >>>>>> which
>> > > > >>>>>>>> is
>> > > > >>>>>>>>>>>>>>>> far too
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> frequent.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000
>> > > > >> records,
>> > > > >>>>>> which
>> > > > >>>>>>>> is
>> > > > >>>>>>>>>> a
>> > > > >>>>>>>>>>>>>>>>>> change
>> > > > >>>>>>>>>>>>>>>>>>>> in
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
>> > > > >> performance
>> > > > >>>>>> problems
>> > > > >>>>>>>> for
>> > > > >>>>>>>>>>>>>>>> some
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> applications.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions,
>> and
>> > > > >> I'm
>> > > > >>>>>>>> currently
>> > > > >>>>>>>>>>>>>>>> leaning
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> towards
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> using the
>> > > > >>>> statestore.transaction.buffer.max.bytes
>> > > > >>>>>>>>>>>>>>>>>> configuration
>> > > > >>>>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to
>> > > > >>>> exceed it.
>> > > > >>>>>>>> This
>> > > > >>>>>>>>>>>>>>>> would
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> complement the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
>> > > > >> functionality
>> > > > >>>> that
>> > > > >>>>>> this
>> > > > >>>>>>>>>>>>>>>>>>>> configuration
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provides, in the following way:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Currently, we use
>> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
>> > > > >>>>>>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> force an
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            early Task commit if
>> > processing
>> > > > >>>> more
>> > > > >>>>>>>> records
>> > > > >>>>>>>>>>>> would
>> > > > >>>>>>>>>>>>>>>>>> cause
>> > > > >>>>>>>>>>>>>>>>>>>> our
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> state
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> store
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            transactions to exceed the
>> > > > >> memory
>> > > > >>>>>> assigned
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>>>>>> them.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - New functionality: when a
>> > > > >> Task
>> > > > >>>> *does*
>> > > > >>>>>>>>>> commit,
>> > > > >>>>>>>>>>>>>>>> we will
>> > > > >>>>>>>>>>>>>>>>>>>> not
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the stores (and hence flush
>> > the
>> > > > >>>>>> transaction
>> > > > >>>>>>>>>>>>>>>> buffers)
>> > > > >>>>>>>>>>>>>>>>>>>> unless
>> > > > >>>>>>>>>>>>>>>>>>>>>> we
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> expect to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            cross the
>> > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
>> > > > >>>>>>>>>>>>>>>>>>>> threshold
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> before
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the next
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick
>> > > > >> Telford <
>> > > > >>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I
>> > > > >> meant in
>> > > > >>>>>> terms of
>> > > > >>>>>>>>>>>>>>>>>>>> understanding
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance profile under various
>> > > > >>>>>> circumstances. The
>> > > > >>>>>>>>>>>>>>>> more
>> > > > >>>>>>>>>>>>>>>>>>>> complex
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would
>> > > > >> be for
>> > > > >>>>>> users
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>> understand
>> > > > >>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance they see. For example,
>> > > > >> spilling
>> > > > >>>>>> records
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>>>>>> disk
>> > > > >>>>>>>>>>>>>>>>>>>> when
>> > > > >>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a
>> threshold
>> > > > >>>> would, I
>> > > > >>>>>>>>>> expect,
>> > > > >>>>>>>>>>>>>>>>>> reduce
>> > > > >>>>>>>>>>>>>>>>>>>>>> write
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
>> > > > >>>> throughput
>> > > > >>>>>> could
>> > > > >>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>>>>>> unexpected,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> potentially difficult to
>> > > > >>>> diagnose/understand for
>> > > > >>>>>>>>>> users.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early
>> > > > >> commit"
>> > > > >>>>>> concept is
>> > > > >>>>>>>>>>>>>>>>>> relatively
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to
>> document,
>> > > > >> and
>> > > > >>>>>>>>>>>> conceptually
>> > > > >>>>>>>>>>>>>>>>>> fairly
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> obvious to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users. We could probably add a
>> metric to
>> > > > >>>> make it
>> > > > >>>>>>>>>> easier
>> > > > >>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it happens though.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially*
>> an
>> > > > >>>> indirect
>> > > > >>>>>> way
>> > > > >>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>>> telling
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
>> > > > >>>>>> READ_COMMITTED
>> > > > >>>>>>>>>>>>>>>>>>>> essentially
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> requires
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates
>> that
>> > > > >> two
>> > > > >>>>>> threads
>> > > > >>>>>>>>>>>>>>>> calling
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see
>> writes
>> > > > >>>> from
>> > > > >>>>>> the
>> > > > >>>>>>>>>> other
>> > > > >>>>>>>>>>>>>>>>>>>>>> transaction
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> until
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they have been committed. With
>> > > > >>>>>> READ_UNCOMMITTED, all
>> > > > >>>>>>>>>>>>>>>> bets are
>> > > > >>>>>>>>>>>>>>>>>>>> off,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe
>> > > > >> written
>> > > > >>>>>> records
>> > > > >>>>>>>> at
>> > > > >>>>>>>>>>>>>>>> any
>> > > > >>>>>>>>>>>>>>>>>>>> time,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> which is
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That
>> > > > >> said,
>> > > > >>>>>>>> StateStores
>> > > > >>>>>>>>>>>>>>>> are
>> > > > >>>>>>>>>>>>>>>>>> free
>> > > > >>>>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implement
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these guarantees however they can,
>> > > > >> which is
>> > > > >>>> a
>> > > > >>>>>> bit
>> > > > >>>>>>>> more
>> > > > >>>>>>>>>>>>>>>>>> relaxed
>> > > > >>>>>>>>>>>>>>>>>>>>>> than
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dictating "you must use
>> transactions".
>> > > > >> For
>> > > > >>>>>> example,
>> > > > >>>>>>>>>>>> with
>> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
>> > > > >>>>>>>>>>>>>>>>>>>>>> we
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED ==
>> > > > >>>> WBWI-based
>> > > > >>>>>>>>>>>>>>>>>> "transactions",
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to
>> the
>> > > > >>>>>> database.
>> > > > >>>>>>>> But
>> > > > >>>>>>>>>>>>>>>> with
>> > > > >>>>>>>>>>>>>>>>>>>> other
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> storage
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to
>> > > > >> *always*
>> > > > >>>> use
>> > > > >>>>>>>>>>>>>>>> transactions,
>> > > > >>>>>>>>>>>>>>>>>>>> even
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage
>> > > > >> engines
>> > > > >>>>>> that
>> > > > >>>>>>>>>> don't
>> > > > >>>>>>>>>>>>>>>>>> provide
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation
>> > > > >> guarantees
>> > > > >>>> can
>> > > > >>>>>> be
>> > > > >>>>>>>> met
>> > > > >>>>>>>>>>>>>>>> using a
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> different
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> technique.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the
>> > > > >> StateStore
>> > > > >>>>>> interface
>> > > > >>>>>>>> as
>> > > > >>>>>>>>>>>>>>>>>> loosely
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> coupled
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible,
>> to
>> > > > >> give
>> > > > >>>>>>>>>>>>>>>> implementers
>> > > > >>>>>>>>>>>>>>>>>> more
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> freedom, and
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal
>> knowledge
>> > > > >>>>>> required.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That said, I understand that
>> > > > >>>> "IsolationLevel"
>> > > > >>>>>> might
>> > > > >>>>>>>>>> not
>> > > > >>>>>>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> right
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make
>> it
>> > > > >> much
>> > > > >>>> more
>> > > > >>>>>>>>>>>>>>>> explicit if
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> required, e.g.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7-8.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I can make these changes either later
>> > > > >> today
>> > > > >>>> or
>> > > > >>>>>>>>>>>> tomorrow.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Small update:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and
>> > > > >> fixed a
>> > > > >>>>>> bunch of
>> > > > >>>>>>>>>>>>>>>> issues
>> > > > >>>>>>>>>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests
>> > > > >> pass,
>> > > > >>>>>> which is
>> > > > >>>>>>>>>>>>>>>>>> promising,
>> > > > >>>>>>>>>>>>>>>>>>>> but
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> it
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to undergo some performance
>> > > > >> testing. I
>> > > > >>>>>> haven't
>> > > > >>>>>>>>>>>>>>>> (yet)
>> > > > >>>>>>>>>>>>>>>>>>>> worked
>> > > > >>>>>>>>>>>>>>>>>>>>>> on
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()`
>> stuff,
>> > > > >> but I
>> > > > >>>>>> would
>> > > > >>>>>>>>>>>>>>>> expect
>> > > > >>>>>>>>>>>>>>>>>> that,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
>> > > > >>>> difference. The
>> > > > >>>>>>>>>> branch
>> > > > >>>>>>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>>>> available
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> at
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
>> > > > >>>>>>>>>> if
>> > > > >>>>>>>>>>>>>>>>>> anyone
>> > > > >>>>>>>>>>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno
>> > > > >> Cadonna
>> > > > >>>> <
>> > > > >>>>>>>>>>>>>>>>>>>> cadonna@apache.org>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was
>> > > > >> actually
>> > > > >>>> also
>> > > > >>>>>> my
>> > > > >>>>>>>>>>>>>>>> point. I
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> understood
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that John was proposing the
>> ingestion
>> > > > >> path
>> > > > >>>> as
>> > > > >>>>>> a way
>> > > > >>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>> avoid
>> > > > >>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> early
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted
>> the
>> > > > >>>> intent.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that
>> actually
>> > > > >> it is
>> > > > >>>>>> public
>> > > > >>>>>>>>>>>>>>>> API. My
>> > > > >>>>>>>>>>>>>>>>>>>>>> question
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects
>> normal
>> > > > >>>>>> processing.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the
>> > > > >> size
>> > > > >>>> of the
>> > > > >>>>>>>>>>>>>>>> transaction
>> > > > >>>>>>>>>>>>>>>>>>>>>> buffer
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit
>> > > > >> affects
>> > > > >>>> the
>> > > > >>>>>> whole
>> > > > >>>>>>>>>>>>>>>>>> processing
>> > > > >>>>>>>>>>>>>>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a
>> > > > >> state
>> > > > >>>> store
>> > > > >>>>>> is
>> > > > >>>>>>>> not
>> > > > >>>>>>>>>>>>>>>>>>>> confined to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over
>> and
>> > > > >>>>>> changes the
>> > > > >>>>>>>>>>>>>>>> behavior
>> > > > >>>>>>>>>>>>>>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> other
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with
>> you
>> > > > >> that
>> > > > >>>> it
>> > > > >>>>>> is a
>> > > > >>>>>>>>>>>>>>>> decent
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compromise. I
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides
>> > > > >> and
>> > > > >>>> list
>> > > > >>>>>> the
>> > > > >>>>>>>>>>>>>>>> options to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> overcome
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all
>> > > > >>>> options
>> > > > >>>>>> seem
>> > > > >>>>>>>>>>>> quite
>> > > > >>>>>>>>>>>>>>>>>> heavy
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compared
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand
>> > > > >> what you
>> > > > >>>>>> mean
>> > > > >>>>>>>> with
>> > > > >>>>>>>>>>>>>>>> "less
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> predictable
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for users", though.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
>> > > > >>>> alternatives
>> > > > >>>>>>>> really
>> > > > >>>>>>>>>>>>>>>>>>>>>> interesting.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> But I
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds
>> good
>> > > > >> and
>> > > > >>>> we
>> > > > >>>>>> should
>> > > > >>>>>>>>>>>>>>>> continue
>> > > > >>>>>>>>>>>>>>>>>>>> with
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> it!
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my
>> > > > >> e-mail on
>> > > > >>>>>> June
>> > > > >>>>>>>>>> 20th:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning
>> > > > >> behind
>> > > > >>>>>> putting
>> > > > >>>>>>>>>>>>>>>> isolation
>> > > > >>>>>>>>>>>>>>>>>>>>>> level
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> in
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks!
>> Should
>> > > > >>>> that
>> > > > >>>>>> also
>> > > > >>>>>>>> be
>> > > > >>>>>>>>>> a
>> > > > >>>>>>>>>>>>>>>> way
>> > > > >>>>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>> give
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to
>> > > > >> decide
>> > > > >>>>>> whether
>> > > > >>>>>>>> to
>> > > > >>>>>>>>>>>>>>>> turn on
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more
>> concerned
>> > > > >> about
>> > > > >>>>>> how do
>> > > > >>>>>>>>>> you
>> > > > >>>>>>>>>>>>>>>> know
>> > > > >>>>>>>>>>>>>>>>>>>> if a
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written
>> > > > >> under
>> > > > >>>> EOS,
>> > > > >>>>>> if
>> > > > >>>>>>>> you
>> > > > >>>>>>>>>>>>>>>> do not
>> > > > >>>>>>>>>>>>>>>>>>>>>> have a
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> way
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
>> > > > >>>> transactional or
>> > > > >>>>>> not.
>> > > > >>>>>>>>>> If
>> > > > >>>>>>>>>>>>>>>> a
>> > > > >>>>>>>>>>>>>>>>>> state
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file
>> can
>> > > > >> be
>> > > > >>>>>> written
>> > > > >>>>>>>>>>>>>>>> during
>> > > > >>>>>>>>>>>>>>>>>>>> normal
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state
>> > > > >> store
>> > > > >>>> is not
>> > > > >>>>>>>>>>>>>>>>>> transactional,
>> > > > >>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written
>> > > > >> under
>> > > > >>>> EOS.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My point was about not only
>> > > > >> considering the
>> > > > >>>>>> bytes
>> > > > >>>>>>>> in
>> > > > >>>>>>>>>>>>>>>> memory
>> > > > >>>>>>>>>>>>>>>>>> in
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> config
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes,
>> but
>> > > > >> also
>> > > > >>>>>> bytes
>> > > > >>>>>>>> that
>> > > > >>>>>>>>>>>>>>>> might
>> > > > >>>>>>>>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> spilled
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering
>> > > > >>>> whether you
>> > > > >>>>>>>>>> should
>> > > > >>>>>>>>>>>>>>>>>> remove
>> > > > >>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of
>> memory
>> > > > >>>> bytes to
>> > > > >>>>>> be
>> > > > >>>>>>>>>> used
>> > > > >>>>>>>>>>>>>>>> to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store
>> > > > >> records." My
>> > > > >>>>>>>> thinking
>> > > > >>>>>>>>>>>>>>>> was
>> > > > >>>>>>>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>>>>>> even
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted
>> bytes to
>> > > > >>>> disk,
>> > > > >>>>>>>>>> limiting
>> > > > >>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>> overall
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> bytes
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it
>> > > > >> again
>> > > > >>>> and
>> > > > >>>>>>>>>>>>>>>> considering
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> recent
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too
>> much
>> > > > >>>> sense
>> > > > >>>>>>>> anymore.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I like the name
>> > > > >>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes that
>> > > > >>>>>>>>>>>>>>>>>>>> you
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 8.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
>> > > > >>>>>> implementation
>> > > > >>>>>>>>>>>>>>>> details) of
>> > > > >>>>>>>>>>>>>>>>>>>> how
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of
>> > > > >> changelog
>> > > > >>>>>>>>>>>>>>>> transactions,
>> > > > >>>>>>>>>>>>>>>>>>>> state
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing
>> would be
>> > > > >>>> great.
>> > > > >>>>>>>> Would
>> > > > >>>>>>>>>>>> be
>> > > > >>>>>>>>>>>>>>>>>> great
>> > > > >>>>>>>>>>>>>>>>>>>> if
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> you
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about
>> the
>> > > > >>>>>> behavior in
>> > > > >>>>>>>>>>>>>>>> case of
>> > > > >>>>>>>>>>>>>>>>>> a
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> failure.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For instance how does a
>> transactional
>> > > > >> state
>> > > > >>>>>> store
>> > > > >>>>>>>>>>>>>>>> recover
>> > > > >>>>>>>>>>>>>>>>>>>> after a
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
>> > > > >>>> transaction
>> > > > >>>>>>>> buffer,
>> > > > >>>>>>>>>>>>>>>> etc.
>> > > > >>>>>>>>>>>>>>>>>>>> (that
>> > > > >>>>>>>>>>>>>>>>>>>>>> is
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> what
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford
>> wrote:
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue
>> that
>> > > > >>>>>>>>>>>>>>>> WriteBatchWithIndex
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely
>> to
>> > > > >>>> exceed)
>> > > > >>>>>>>>>>>>>>>> configured
>> > > > >>>>>>>>>>>>>>>>>>>> memory
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needs to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
>> > > > >>>> Ultimately,
>> > > > >>>>>> any
>> > > > >>>>>>>>>>>>>>>> approach
>> > > > >>>>>>>>>>>>>>>>>>>> based
>> > > > >>>>>>>>>>>>>>>>>>>>>> on
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cracking
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using
>> it in
>> > > > >>>> ways
>> > > > >>>>>> it's
>> > > > >>>>>>>> not
>> > > > >>>>>>>>>>>>>>>> really
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> designed
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for is
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen
>> > > > >> performance
>> > > > >>>> or
>> > > > >>>>>>>>>>>>>>>> consistency
>> > > > >>>>>>>>>>>>>>>>>>>> issues.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing
>> > > > >> these
>> > > > >>>>>> early
>> > > > >>>>>>>>>>>>>>>> commits?
>> > > > >>>>>>>>>>>>>>>>>>>> While
>> > > > >>>>>>>>>>>>>>>>>>>>>> not
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideal, I
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise
>> to
>> > > > >>>> ensure
>> > > > >>>>>>>>>>>>>>>> consistency
>> > > > >>>>>>>>>>>>>>>>>>>> whilst
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maintaining
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem
>> > > > >> *very*
>> > > > >>>>>>>>>>>>>>>> complicated, and
>> > > > >>>>>>>>>>>>>>>>>>>> might
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for
>> > > > >> users
>> > > > >>>> as a
>> > > > >>>>>>>>>>>>>>>> consequence.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope
>> of
>> > > > >> this
>> > > > >>>>>> KIP is
>> > > > >>>>>>>>>>>>>>>> growing a
>> > > > >>>>>>>>>>>>>>>>>>>> bit
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> out
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss
>> > > > >> ideas
>> > > > >>>> for
>> > > > >>>>>>>> future
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> improvements, I
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>> > > > >>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope
>> > > > >> down
>> > > > >>>> to a
>> > > > >>>>>>>> design
>> > > > >>>>>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>>>>>> achieves
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> most
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
>> > > > >>>>>> restorations
>> > > > >>>>>>>>>>>>>>>> during
>> > > > >>>>>>>>>>>>>>>>>> dirty
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any
>> design
>> > > > >> that
>> > > > >>>>>> this KIP
>> > > > >>>>>>>>>>>>>>>> produces
>> > > > >>>>>>>>>>>>>>>>>>>> can
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ultimately
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future,
>> especially
>> > > > >> if
>> > > > >>>> the
>> > > > >>>>>> bulk
>> > > > >>>>>>>> of
>> > > > >>>>>>>>>>>>>>>> it is
>> > > > >>>>>>>>>>>>>>>>>>>>>> internal
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next
>> week
>> > > > >>>> trying
>> > > > >>>>>> to
>> > > > >>>>>>>>>>>>>>>> re-work
>> > > > >>>>>>>>>>>>>>>>>> the
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to
>> remove
>> > > > >> the
>> > > > >>>>>>>>>>>>>>>> newTransaction()
>> > > > >>>>>>>>>>>>>>>>>>>>>> method,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> such
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail
>> of
>> > > > >>>>>>>> RocksDBStore.
>> > > > >>>>>>>>>>>>>>>> That
>> > > > >>>>>>>>>>>>>>>>>>>> way, if
>> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the
>> > > > >> future,
>> > > > >>>>>> like
>> > > > >>>>>>>> the
>> > > > >>>>>>>>>>>>>>>> SST
>> > > > >>>>>>>>>>>>>>>>>> file
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> management
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so
>> > > > >> with
>> > > > >>>>>> little/no
>> > > > >>>>>>>>>>>> API
>> > > > >>>>>>>>>>>>>>>>>>>> changes.
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>>
>> > > > >>>>>>>>>>>>
>> > > > >>>>>>>>>>>
>> > > > >>>>>>>>>>
>> > > > >>>>>>>>>
>> > > > >>>>>>>>
>> > > > >>>>>>
>> > > > >>>>>>
>> > > > >>>>
>> > > > >>
>> > > > >
>> > >
>> >
>>
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi everyone,

Sorry for the delay.

I've updated the KIP based on our discussion. I've also added some details
on "Transactional support under READ_UNCOMMITTED" and  "Query-time
Isolation Levels" in the "Rejected Alternatives" section, to highlight the
potential to improve things in a future KIP.

If there are no more requested changes or objections, I'll kick off the
vote thread tomorrow, approximately 24 hours from now.

While I'd love for this KIP to make it into 3.7.0, I can't commit to it
being code-complete by the December 20th deadline. While the branch I have
is broadly working, there are areas that will likely require improvement,
and others that are untested:

- Untested outside of unit tests: Windowed stores, Versioned stores,
Global stores, IQv2. They should all work as expected, but the Position
file logic might be a bit janky.
- Requires improvement: Position data handling. Very clunky right now.
- Requires implementation: The "feature flag", where we disable
transactionality under EOS+READ_UNCOMMITTED. I've started this, but it's
far from simple. Plus this will require fleshing out the test suite quite a
bit.

For the curious, my active development branch has been rebased against
3.6.0: https://github.com/nicktelford/kafka/tree/KIP-892-3.6.0

Regards,
Nick

On Mon, 30 Oct 2023 at 23:32, Sophie Blee-Goldman <so...@responsive.dev>
wrote:

> Hey Nick, sounds like things are moving along here. I think you're already
> aware, but just as a reminder to us all, the KIP freeze deadline for 3.7 is
> approaching and is currently set for Nov 18. I think we can all agree that
> it would be great to have this KIP accepted by then, and it sounds like
> much of the implementation is already completed, in which case this might
> be able to make it into 3.7.
>
> It sounds like this discussion is wrapping up so I just want to summarize
> my understanding of the current plan regarding configs since it doesn't
> seem like the KIP has been updated to include this yet.
>
> Basically we're all in agreement to go with option #1 that you presented
> earlier, right? Assuming so, I just want to clear up some details around
> the default behavior. What happens if:
> 1. User sets EOS and sets READ_UNCOMMITTED: txn state stores will be
> disabled/feature-flagged
> 2. User sets EOS and does not set anything for the isolation level:
> although the default is READ_UNCOMMITTED, enabling EOS will change the
> default to READ_COMMITTED and txn state stores will be used
>
> As for the future, it sounds like when READ_UNCOMMITTED mode is
> implemented, we will basically just remove this "feature flag" and txn
> state stores will always be used for all EOS cases. EOS users will be able
> to configure the isolation level independently, although it will still
> default to READ_COMMITTED when EOS is enabled and it wasn't explicitly
> configured.
>
> Is this understanding correct? I think this was the right choice
> regardless, as it will give people a way to disable the txn stores in an
> emergency -- as a project we went a long time with little pressure to
> feature flag things, and our users paid the cost for that. Even if we
> managed to ship something without bugs, it was often only after an intense
> period of successive blocker bugs that delayed the entire released for
> weeks. Other times, major bugs slipped through and some versions became
> unusable for certain use cases. So having some way to disable the txn state
> stores when EOS is used feels like a good strategy, since you just never
> know what might get through all the testing we do.
>
> If this ends up slipping to 4.0 and you manage to implement the
> READ_UNCOMMITTED mode within the same release, I think it's worth
> considering to add in an additional feature flag, even if it's just a
> backdoor internal config (eg as we did in KIP-441 with the internal task
> assignor config). But for now let's assume the first release this KIP
> appears in will have the behavior as described above, with READ_UNCOMMITTED
> mode acting as a feature flag
>
> Nick -- please let us know when you've updated the KIP to clarify the
> config behavior, and are ready for a vote!
>
>
> On Sun, Oct 29, 2023 at 12:02 PM Colt McNealy <co...@littlehorse.io> wrote:
>
> > Guozhang—I agree, I am in favor of moving forward with the KIP now that
> the
> > Transactional State Stores will be behind a feature flag.
> >
> > Nick—I just did a bit more light testing of your branch `KIP-892-3.5.0`
> > with your most recent changes. I couldn't detect a performance difference
> > versus trunk (in the past there was a slight degradation of performance
> on
> > the restoration path, but that has been fixed). I don't believe that your
> > branch has the state updater thread enabled, so I didn't test that path
> too
> > heavily.
> >
> > As expected, however, our internal correctness tests failed due to the IQ
> > read-your-own-writes issue we discussed previously. The community as a
> > whole would vastly benefit from this KIP getting over the finish line in
> > 3.7.0, and so long as it is behind a feature flag so that we at
> LittleHorse
> > can still guarantee RYOW for our users, I think it's purely a win 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 <
> guozhang.wang.us@gmail.com
> > >
> > wrote:
> >
> > > I'd agree with you guys that as long as we are in agreement about the
> > > configuration semantics, that would be a big win to move forward for
> > > this KIP. As for the TaskCorruptedException handling like wiping state
> > > stores, we can discuss that in the PR rather than in the KIP.
> > >
> > > Just to clarify, I'm onboard with the latest proposal, and probably we
> > > can move on for voting on this KIP now?
> > >
> > > Guozhang
> > >
> > > On Thu, Oct 19, 2023 at 5:33 AM Bruno Cadonna <ca...@apache.org>
> > wrote:
> > > >
> > > > Hi Nick,
> > > >
> > > > What you and Lucas wrote about the different configurations of
> ALOS/EOS
> > > > and READ_COMMITTED/READ_UNCOMMITTED make sense to me. My earlier
> > > > concerns about changelogs diverging from the content of the local
> state
> > > > stores turned out to not apply. So I think, we can move on with those
> > > > configurations.
> > > >
> > > > Regarding the TaskCorruptedException and wiping out the state stores
> > > > under EOS, couldn't we abort the transaction on the state store and
> > > > close the task dirty? If the Kafka transaction was indeed committed,
> > the
> > > > store would restore the missing part from the changelog topic. If the
> > > > Kafka transaction was not committed, changelog topic and state store
> > are
> > > > in-sync.
> > > >
> > > > In any case, IMO those are implementation details that we do not need
> > to
> > > > discuss and solve in the KIP discussion. We can solve them on the PR.
> > > > The important thing is that the processing guarantees hold.
> > > >
> > > > Best,
> > > > Bruno
> > > >
> > > > On 10/18/23 3:56 PM, Nick Telford wrote:
> > > > > Hi Lucas,
> > > > >
> > > > > TaskCorruptedException is how Streams signals that the Task state
> > > needs to
> > > > > be wiped, so we can't retain that exception without also wiping
> state
> > > on
> > > > > timeouts.
> > > > >
> > > > > Regards,
> > > > > Nick
> > > > >
> > > > > On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <
> lbrutschy@confluent.io
> > > .invalid>
> > > > > wrote:
> > > > >
> > > > >> Hi Nick,
> > > > >>
> > > > >> I think indeed the better behavior would be to retry
> > commitTransaction
> > > > >> until we risk running out of time to meet `max.poll.interval.ms`.
> > > > >>
> > > > >> However, if it's handled as a `TaskCorruptedException` at the
> > moment,
> > > > >> I would do the same in this KIP, and leave exception handling
> > > > >> improvements to future work. 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 <
> > nick.telford@gmail.com>
> > > > >> wrote:
> > > > >>>
> > > > >>> Hi Lucas,
> > > > >>>
> > > > >>> Yeah, this is pretty much the direction I'm thinking of going in
> > > now. You
> > > > >>> make an interesting point about committing on-error under
> > > > >>> ALOS/READ_COMMITTED, although I haven't had a chance to think
> > > through the
> > > > >>> implications yet.
> > > > >>>
> > > > >>> Something that I ran into earlier this week is an issue with the
> > new
> > > > >>> handling of TimeoutException. Without TX stores, TimeoutException
> > > under
> > > > >> EOS
> > > > >>> throws a TaskCorruptedException, which wipes the stores. However,
> > > with TX
> > > > >>> stores, TimeoutException is now just bubbled up and dealt with as
> > it
> > > is
> > > > >>> under ALOS. The problem arises when the
> Producer#commitTransaction
> > > call
> > > > >>> times out: Streams attempts to ignore the error and continue
> > > producing,
> > > > >>> which causes the next call to Producer#send to throw
> > > > >>> "IllegalStateException: Cannot attempt operation `send` because
> the
> > > > >>> previous call to `commitTransaction` timed out and must be
> > retried".
> > > > >>>
> > > > >>> I'm not sure what we should do here: retrying the
> commitTransaction
> > > seems
> > > > >>> logical, but what if it times out again? Where do we draw the
> line
> > > and
> > > > >>> shutdown the instance?
> > > > >>>
> > > > >>> Regards,
> > > > >>> Nick
> > > > >>>
> > > > >>> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <
> > lbrutschy@confluent.io
> > > > >> .invalid>
> > > > >>> wrote:
> > > > >>>
> > > > >>>> Hi all,
> > > > >>>>
> > > > >>>> I think I liked your suggestion of allowing EOS with
> > > READ_UNCOMMITTED,
> > > > >>>> but keep wiping the state on error, and I'd vote for this
> solution
> > > > >>>> when introducing `default.state.isolation.level`. This way, we'd
> > > have
> > > > >>>> the most low-risk roll-out of this feature (no behavior change
> > > without
> > > > >>>> reconfiguration), with the possibility of switching to the most
> > > sane /
> > > > >>>> battle-tested default settings in 4.0. Essentially, we'd have a
> > > > >>>> feature flag but call it `default.state.isolation.level` and
> don't
> > > > >>>> have to deprecate it later.
> > > > >>>>
> > > > >>>> So the possible configurations would then be this:
> > > > >>>>
> > > > >>>> 1. ALOS/READ_UNCOMMITTED (default) = processing uses
> direct-to-DB,
> > > IQ
> > > > >>>> reads from DB.
> > > > >>>> 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads
> from
> > > > >>>> WriteBatch/DB. Flush on error (see note below).
> > > > >>>> 3. EOS/READ_UNCOMMITTED (default) = processing uses
> direct-to-DB,
> > IQ
> > > > >>>> reads from DB. Wipe state on error.
> > > > >>>> 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads
> from
> > > > >>>> WriteBatch/DB.
> > > > >>>>
> > > > >>>> I believe the feature is important enough that we will see good
> > > > >>>> adoption even without changing the default. In 4.0, when we have
> > > seen
> > > > >>>> this being adopted and is battle-tested, we make READ_COMMITTED
> > the
> > > > >>>> default for EOS, or even READ_COMITTED always the default,
> > depending
> > > > >>>> on our experiences. And we could add a clever implementation of
> > > > >>>> READ_UNCOMITTED with WriteBatches later.
> > > > >>>>
> > > > >>>> The only smell here is that `default.state.isolation.level`
> > wouldn't
> > > > >>>> be purely an IQ setting, but it would also (slightly) change the
> > > > >>>> behavior of the processing, but that seems unavoidable as long
> as
> > we
> > > > >>>> haven't solve READ_UNCOMITTED IQ with WriteBatches.
> > > > >>>>
> > > > >>>> Minor: As for Bruno's point 4, I think if we are concerned about
> > > this
> > > > >>>> 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).
> > > > >>>>
> > > > >>>> Cheers,
> > > > >>>> Lucas
> > > > >>>>
> > > > >>>> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <
> > > nick.telford@gmail.com>
> > > > >>>> wrote:
> > > > >>>>>
> > > > >>>>> Hi Guozhang,
> > > > >>>>>
> > > > >>>>> The KIP as it stands introduces a new configuration,
> > > > >>>>> default.state.isolation.level, which is independent of
> > > > >> processing.mode.
> > > > >>>>> It's intended that this new configuration be used to configure
> a
> > > > >> global
> > > > >>>> IQ
> > > > >>>>> isolation level in the short term, with a future KIP
> introducing
> > > the
> > > > >>>>> capability to change the isolation level on a per-query basis,
> > > > >> falling
> > > > >>>> back
> > > > >>>>> to the "default" defined by this config. That's why I called it
> > > > >>>> "default",
> > > > >>>>> for future-proofing.
> > > > >>>>>
> > > > >>>>> However, it currently includes the caveat that READ_UNCOMMITTED
> > is
> > > > >> not
> > > > >>>>> available under EOS. I think this is the coupling you are
> > alluding
> > > > >> to?
> > > > >>>>>
> > > > >>>>> This isn't intended to be a restriction of the API, but is
> > > currently
> > > > >> a
> > > > >>>>> technical limitation. However, after discussing with some users
> > > about
> > > > >>>>> use-cases that would require READ_UNCOMMITTED under EOS, I'm
> > > > >> inclined to
> > > > >>>>> remove that clause and put in the necessary work to make that
> > > > >> combination
> > > > >>>>> possible now.
> > > > >>>>>
> > > > >>>>> I currently see two possible approaches:
> > > > >>>>>
> > > > >>>>>     1. Disable TX StateStores internally when the
> IsolationLevel
> > is
> > > > >>>>>     READ_UNCOMMITTED and the processing.mode is EOS. This is
> more
> > > > >>>> difficult
> > > > >>>>>     than it sounds, as there are many assumptions being made
> > > > >> throughout
> > > > >>>> the
> > > > >>>>>     internals about the guarantees StateStores provide. It
> would
> > > > >>>> definitely add
> > > > >>>>>     a lot of extra "if (read_uncommitted && eos)" branches,
> > > > >> complicating
> > > > >>>>>     maintenance and testing.
> > > > >>>>>     2. Invest the time *now* to make READ_UNCOMMITTED of EOS
> > > > >> StateStores
> > > > >>>>>     possible. I have some ideas on how this could be achieved,
> > but
> > > > >> they
> > > > >>>> would
> > > > >>>>>     need testing and could introduce some additional issues.
> The
> > > > >> benefit
> > > > >>>> of
> > > > >>>>>     this approach is that it would make query-time
> > IsolationLevels
> > > > >> much
> > > > >>>> simpler
> > > > >>>>>     to implement in the future.
> > > > >>>>>
> > > > >>>>> Unfortunately, both will require considerable work that will
> > > further
> > > > >>>> delay
> > > > >>>>> this KIP, which was the reason I placed the restriction in the
> > KIP
> > > > >> in the
> > > > >>>>> first place.
> > > > >>>>>
> > > > >>>>> Regards,
> > > > >>>>> Nick
> > > > >>>>>
> > > > >>>>> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
> > > > >> guozhang.wang.us@gmail.com>
> > > > >>>>> wrote:
> > > > >>>>>
> > > > >>>>>> Hello Nick,
> > > > >>>>>>
> > > > >>>>>> First of all, thanks a lot for the great effort you've put in
> > > > >> driving
> > > > >>>>>> this KIP! I really like it coming through finally, as many
> > people
> > > > >> in
> > > > >>>>>> the community have raised this. At the same time I honestly
> > feel a
> > > > >> bit
> > > > >>>>>> ashamed for not putting enough of my time supporting it and
> > > > >> pushing it
> > > > >>>>>> through the finish line (you raised this KIP almost a year
> ago).
> > > > >>>>>>
> > > > >>>>>> I briefly passed through the DISCUSS thread so far, not sure
> > I've
> > > > >> 100
> > > > >>>>>> percent digested all the bullet points. But with the goal of
> > > > >> trying to
> > > > >>>>>> help take it through the finish line in mind, I'd want to
> throw
> > > > >>>>>> thoughts on top of my head only on the point #4 above which I
> > felt
> > > > >> may
> > > > >>>>>> be the main hurdle for the current KIP to drive to a consensus
> > > now.
> > > > >>>>>>
> > > > >>>>>> The general question I asked myself is, whether we want to
> > couple
> > > > >> "IQ
> > > > >>>>>> reading mode" with "processing mode". While technically I tend
> > to
> > > > >>>>>> agree with you that, it's feels like a bug if some single user
> > > > >> chose
> > > > >>>>>> "EOS" for processing mode while choosing "read uncommitted"
> for
> > IQ
> > > > >>>>>> reading mode, at the same time, I'm thinking if it's possible
> > that
> > > > >>>>>> there could be two different persons (or even two teams) that
> > > > >> would be
> > > > >>>>>> using the stream API to build the app, and the IQ API to query
> > the
> > > > >>>>>> running state of the app. I know this is less of a technical
> > thing
> > > > >> but
> > > > >>>>>> rather a more design stuff, but if it could be ever the case,
> > I'm
> > > > >>>>>> wondering if the personale using the IQ API knows about the
> > risks
> > > > >> of
> > > > >>>>>> using read uncommitted but still chose so for the favor of
> > > > >>>>>> performance, no matter if the underlying stream processing
> mode
> > > > >>>>>> configured by another personale is EOS or not. In that regard,
> > I'm
> > > > >>>>>> leaning towards a "leaving the door open, and close it later
> if
> > we
> > > > >>>>>> found it's a bad idea" aspect with a configuration 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 can still
> enforce
> > > > >>>>>> coupling them.
> > > > >>>>>>
> > > > >>>>>> Again, just my 2c here. Thanks again for the great patience
> and
> > > > >>>>>> diligence on this KIP.
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>> Guozhang
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>>> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
> > > > >> nick.telford@gmail.com>
> > > > >>>>>> wrote:
> > > > >>>>>>>
> > > > >>>>>>> Hi Bruno,
> > > > >>>>>>>
> > > > >>>>>>> 4.
> > > > >>>>>>> I'll hold off on making that change until we have a consensus
> > as
> > > > >> to
> > > > >>>> what
> > > > >>>>>>> configuration to use to control all of this, as it'll be
> > > > >> affected by
> > > > >>>> the
> > > > >>>>>>> decision on EOS isolation levels.
> > > > >>>>>>>
> > > > >>>>>>> 5.
> > > > >>>>>>> Done. I've chosen "committedOffsets".
> > > > >>>>>>>
> > > > >>>>>>> Regards,
> > > > >>>>>>> Nick
> > > > >>>>>>>
> > > > >>>>>>> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <
> > cadonna@apache.org>
> > > > >>>> wrote:
> > > > >>>>>>>
> > > > >>>>>>>> Hi Nick,
> > > > >>>>>>>>
> > > > >>>>>>>> 1.
> > > > >>>>>>>> Yeah, you are probably right that it does not make too much
> > > > >> sense.
> > > > >>>>>>>> Thanks for the clarification!
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> 4.
> > > > >>>>>>>> Yes, sorry for the back and forth, but I think for the sake
> of
> > > > >> the
> > > > >>>> KIP
> > > > >>>>>>>> it is better to let the ALOS behavior as it is for now due
> to
> > > > >> the
> > > > >>>>>>>> possible issues you would run into. Maybe we can find a
> > > > >> solution
> > > > >>>> in the
> > > > >>>>>>>> future. Now the question returns to whether we really need
> > > > >>>>>>>> default.state.isolation.level. Maybe the config could be the
> > > > >>>> feature
> > > > >>>>>>>> flag Sophie requested.
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> 5.
> > > > >>>>>>>> There 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 committedOffsetFor().
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> Best,
> > > > >>>>>>>> Bruno
> > > > >>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>> On 10/13/23 3:21 PM, Nick Telford wrote:
> > > > >>>>>>>>> Hi Bruno,
> > > > >>>>>>>>>
> > > > >>>>>>>>> Thanks for getting back to me.
> > > > >>>>>>>>>
> > > > >>>>>>>>> 1.
> > > > >>>>>>>>> I think this should be possible. Are you thinking of the
> > > > >>>> situation
> > > > >>>>>> where
> > > > >>>>>>>> a
> > > > >>>>>>>>> user may downgrade to a previous version of Kafka Streams?
> In
> > > > >>>> that
> > > > >>>>>> case,
> > > > >>>>>>>>> sadly, the RocksDBStore would get wiped by the older
> version
> > > > >> of
> > > > >>>> Kafka
> > > > >>>>>>>>> Streams anyway, because that version wouldn't understand
> the
> > > > >>>> extra
> > > > >>>>>> column
> > > > >>>>>>>>> family (that holds offsets), so the missing Position file
> > > > >> would
> > > > >>>>>>>>> automatically get rebuilt when the store is rebuilt from
> the
> > > > >>>>>> changelog.
> > > > >>>>>>>>> Are there other situations than downgrade where a
> > > > >> transactional
> > > > >>>> store
> > > > >>>>>>>> could
> > > > >>>>>>>>> be replaced by a non-transactional one? I can't think of
> any.
> > > > >>>>>>>>>
> > > > >>>>>>>>> 2.
> > > > >>>>>>>>> Ahh yes, the Test Plan - my Kryptonite! This section
> > > > >> definitely
> > > > >>>>>> needs to
> > > > >>>>>>>> be
> > > > >>>>>>>>> fleshed out. I'll work on that. How much detail do you
> need?
> > > > >>>>>>>>>
> > > > >>>>>>>>> 3.
> > > > >>>>>>>>> See my previous email discussing this.
> > > > >>>>>>>>>
> > > > >>>>>>>>> 4.
> > > > >>>>>>>>> Hmm, this is an interesting point. Are you suggesting that
> > > > >> under
> > > > >>>> ALOS
> > > > >>>>>>>>> READ_COMMITTED should not be supported?
> > > > >>>>>>>>>
> > > > >>>>>>>>> Regards,
> > > > >>>>>>>>> Nick
> > > > >>>>>>>>>
> > > > >>>>>>>>> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
> > > > >> cadonna@apache.org>
> > > > >>>>>> wrote:
> > > > >>>>>>>>>
> > > > >>>>>>>>>> Hi Nick,
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> I think the KIP is converging!
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> 1.
> > > > >>>>>>>>>> I am wondering whether it makes sense to write the
> position
> > > > >> file
> > > > >>>>>> during
> > > > >>>>>>>>>> close as we do for the checkpoint file, so that in case
> the
> > > > >>>> state
> > > > >>>>>> store
> > > > >>>>>>>>>> is replaced with a non-transactional state store the
> > > > >>>>>> non-transactional
> > > > >>>>>>>>>> state store finds the position file. I think, this is not
> > > > >>>> strictly
> > > > >>>>>>>>>> needed, but would be a nice behavior instead of just
> > > > >> deleting
> > > > >>>> the
> > > > >>>>>>>>>> position file.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> 2.
> > > > >>>>>>>>>> The test plan does not mention integration tests. Do you
> not
> > > > >>>> need to
> > > > >>>>>>>>>> extend existing ones and add new ones. Also for upgrading
> > > > >> and
> > > > >>>>>>>>>> downgrading you might need integration and/or system
> tests.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> 3.
> > > > >>>>>>>>>> I think Sophie made a point. Although, IQ reading from
> > > > >>>> uncommitted
> > > > >>>>>> data
> > > > >>>>>>>>>> under EOS might be considered a bug by some people. Thus,
> > > > >> your
> > > > >>>> KIP
> > > > >>>>>> would
> > > > >>>>>>>>>> fix a bug rather than changing the intended behavior.
> > > > >> However, I
> > > > >>>>>> also
> > > > >>>>>>>>>> see that a feature flag would help users that rely on this
> > > > >> buggy
> > > > >>>>>>>>>> behavior (at least until AK 4.0).
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> 4.
> > > > >>>>>>>>>> This is related to the previous point. I assume that the
> > > > >>>> difference
> > > > >>>>>>>>>> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is
> > > > >> that in
> > > > >>>> the
> > > > >>>>>>>>>> former you enable transactions on the state store and in
> the
> > > > >>>> latter
> > > > >>>>>> you
> > > > >>>>>>>>>> disable them. If my assumption is correct, I think that is
> > > > >> an
> > > > >>>> issue.
> > > > >>>>>>>>>> Let's assume under ALOS Streams fails over a couple of
> times
> > > > >>>> more or
> > > > >>>>>>>>>> less at the same step in processing after value 3 is added
> > > > >> to an
> > > > >>>>>>>>>> aggregation but the offset of the corresponding input
> record
> > > > >>>> was not
> > > > >>>>>>>>>> committed. Without transactions disabled, the aggregation
> > > > >> value
> > > > >>>>>> would
> > > > >>>>>>>>>> increase by 3 for each failover. With transactions
> enabled,
> > > > >>>> value 3
> > > > >>>>>>>>>> would only be added to the aggregation once when the
> offset
> > > > >> of
> > > > >>>> the
> > > > >>>>>> input
> > > > >>>>>>>>>> record is committed and the transaction finally completes.
> > > > >> So
> > > > >>>> the
> > > > >>>>>>>>>> content of the state store would change depending on the
> > > > >>>>>> configuration
> > > > >>>>>>>>>> for IQ. IMO, the content of the state store should be
> > > > >>>> independent
> > > > >>>>>> from
> > > > >>>>>>>>>> IQ. Given this issue, I propose to not use transactions
> with
> > > > >>>> ALOS at
> > > > >>>>>>>>>> all. I was a big proponent of using transactions with
> ALOS,
> > > > >> but
> > > > >>>> I
> > > > >>>>>>>>>> realized that transactions with ALOS is not as easy as
> > > > >> enabling
> > > > >>>>>>>>>> transactions on state stores. Another aspect that is
> > > > >>>> problematic is
> > > > >>>>>> that
> > > > >>>>>>>>>> the changelog topic which actually replicates the state
> > > > >> store
> > > > >>>> is not
> > > > >>>>>>>>>> transactional under ALOS. Thus, it might happen that the
> > > > >> state
> > > > >>>>>> store and
> > > > >>>>>>>>>> the changelog differ in their content. All of this is
> maybe
> > > > >>>> solvable
> > > > >>>>>>>>>> somehow, but for the sake of this KIP, I would leave it
> for
> > > > >> the
> > > > >>>>>> future.
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> Best,
> > > > >>>>>>>>>> Bruno
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > > > >>>>>>>>>>> Hey Nick! First of all thanks for taking up this awesome
> > > > >>>> feature,
> > > > >>>>>> I'm
> > > > >>>>>>>>>> sure
> > > > >>>>>>>>>>> every single
> > > > >>>>>>>>>>> Kafka Streams user and dev would agree that it is sorely
> > > > >>>> needed.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> I've just been catching up on the KIP and surrounding
> > > > >>>> discussion,
> > > > >>>>>> so
> > > > >>>>>>>>>> please
> > > > >>>>>>>>>>> forgive me
> > > > >>>>>>>>>>> for any misunderstandings or misinterpretations of the
> > > > >> current
> > > > >>>>>> plan and
> > > > >>>>>>>>>>> don't hesitate to
> > > > >>>>>>>>>>> correct me.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Before I jump in, I just want to say that having seen
> this
> > > > >>>> drag on
> > > > >>>>>> for
> > > > >>>>>>>> so
> > > > >>>>>>>>>>> long, my singular
> > > > >>>>>>>>>>> goal in responding is to help this KIP past a perceived
> > > > >>>> impasse so
> > > > >>>>>> we
> > > > >>>>>>>> can
> > > > >>>>>>>>>>> finally move on
> > > > >>>>>>>>>>> to voting and implementing it. Long discussions are to be
> > > > >>>> expected
> > > > >>>>>> for
> > > > >>>>>>>>>>> major features like
> > > > >>>>>>>>>>> this but it's completely on us as the Streams devs to
> make
> > > > >> sure
> > > > >>>>>> there
> > > > >>>>>>>> is
> > > > >>>>>>>>>> an
> > > > >>>>>>>>>>> end in sight
> > > > >>>>>>>>>>> for any ongoing discussion.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> With that said, it's my understanding that the KIP as
> > > > >> currently
> > > > >>>>>>>> proposed
> > > > >>>>>>>>>> is
> > > > >>>>>>>>>>> just not tenable
> > > > >>>>>>>>>>> for Kafka Streams, and would prevent some EOS users from
> > > > >>>> upgrading
> > > > >>>>>> to
> > > > >>>>>>>> the
> > > > >>>>>>>>>>> version it
> > > > >>>>>>>>>>> first appears in. Given that we can't predict or
> guarantee
> > > > >>>> whether
> > > > >>>>>> any
> > > > >>>>>>>> of
> > > > >>>>>>>>>>> the followup KIPs
> > > > >>>>>>>>>>> would be completed in the same release cycle as this one,
> > > > >> we
> > > > >>>> need
> > > > >>>>>> to
> > > > >>>>>>>> make
> > > > >>>>>>>>>>> sure that the
> > > > >>>>>>>>>>> feature is either compatible with all current users or
> else
> > > > >>>>>>>>>> feature-flagged
> > > > >>>>>>>>>>> so that they may
> > > > >>>>>>>>>>> opt in/out.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Therefore, IIUC we need to have either (or both) of these
> > > > >> as
> > > > >>>>>>>>>>> fully-implemented config options:
> > > > >>>>>>>>>>> 1. default.state.isolation.level
> > > > >>>>>>>>>>> 2. enable.transactional.state.stores
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> This way EOS users for whom read_committed semantics are
> > > > >> not
> > > > >>>>>> viable can
> > > > >>>>>>>>>>> still upgrade,
> > > > >>>>>>>>>>> and either use the isolation.level config to leverage the
> > > > >> new
> > > > >>>> txn
> > > > >>>>>> state
> > > > >>>>>>>>>>> stores without sacrificing
> > > > >>>>>>>>>>> their application semantics, or else simply keep the
> > > > >>>> transactional
> > > > >>>>>>>> state
> > > > >>>>>>>>>>> stores disabled until we
> > > > >>>>>>>>>>> are able to fully implement the isolation level
> > > > >> configuration
> > > > >>>> at
> > > > >>>>>> either
> > > > >>>>>>>>>> an
> > > > >>>>>>>>>>> application or query level.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Frankly you are the expert here and know much more about
> > > > >> the
> > > > >>>>>> tradeoffs
> > > > >>>>>>>> in
> > > > >>>>>>>>>>> both semantics and
> > > > >>>>>>>>>>> effort level of implementing one of these configs vs the
> > > > >>>> other. In
> > > > >>>>>> my
> > > > >>>>>>>>>>> opinion, either option would
> > > > >>>>>>>>>>> be fine and I would leave the decision of which one to
> > > > >> include
> > > > >>>> in
> > > > >>>>>> this
> > > > >>>>>>>>>> KIP
> > > > >>>>>>>>>>> completely up to you.
> > > > >>>>>>>>>>> I just don't see a way for the KIP to proceed without
> some
> > > > >>>>>> variation of
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>> above that would allow
> > > > >>>>>>>>>>> EOS users to opt-out of read_committed.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> (If it's all the same to you, I would recommend 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 that gets released.
> > > > >>>>>>>>>>> So personally, my choice would be to add the feature flag
> > > > >> and
> > > > >>>>>> leave it
> > > > >>>>>>>>>> off
> > > > >>>>>>>>>>> by default. If all 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
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > > > >>>>>> nick.telford@gmail.com>
> > > > >>>>>>>>>> wrote:
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>> Hi Bruno,
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Agreed, I can live with that for now.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> In an effort to keep the scope of this KIP from
> > > > >> expanding, I'm
> > > > >>>>>> leaning
> > > > >>>>>>>>>>>> towards just providing a configurable
> > > > >>>>>> default.state.isolation.level
> > > > >>>>>>>> and
> > > > >>>>>>>>>>>> removing IsolationLevel from the StateStoreContext. This
> > > > >>>> would be
> > > > >>>>>>>>>>>> compatible with adding support for query-time
> > > > >> IsolationLevels
> > > > >>>> in
> > > > >>>>>> the
> > > > >>>>>>>>>>>> future, whilst providing a way for users to select an
> > > > >>>> isolation
> > > > >>>>>> level
> > > > >>>>>>>>>> now.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> The big problem with this, however, is that if a user
> > > > >> selects
> > > > >>>>>>>>>>>> processing.mode
> > > > >>>>>>>>>>>> = "exactly-once(-v2|-beta)", and
> > > > >>>> default.state.isolation.level =
> > > > >>>>>>>>>>>> "READ_UNCOMMITTED", we need to guarantee that the data
> > > > >> isn't
> > > > >>>>>> written
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>> disk until commit() is called, but we also need to
> permit
> > > > >> IQ
> > > > >>>>>> threads
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>> read from the ongoing transaction.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> A simple solution would be to (temporarily) forbid this
> > > > >>>>>> combination of
> > > > >>>>>>>>>>>> configuration, and have default.state.isolation.level
> > > > >>>>>> automatically
> > > > >>>>>>>>>> switch
> > > > >>>>>>>>>>>> to READ_COMMITTED when processing.mode is anything other
> > > > >> than
> > > > >>>>>>>>>>>> at-least-once. Do you think this would be acceptable?
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> In a later KIP, we can add support for query-time
> > > > >> isolation
> > > > >>>>>> levels and
> > > > >>>>>>>>>>>> solve this particular problem there, which would relax
> > > > >> this
> > > > >>>>>>>> restriction.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> > > > >>>> cadonna@apache.org>
> > > > >>>>>>>>>> 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,
> > > > >>>>>>>>>>>>> Bruno
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > > > >>>>>>>>>>>>>> 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 <
> > > > >>>>>> nick.telford@gmail.com
> > > > >>>>>>>>>
> > > > >>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Hi everyone,
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> I agree that having IsolationLevel be determined at
> > > > >>>> query-time
> > > > >>>>>> is
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>> ideal design, but there are a few sticking points:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> 1.
> > > > >>>>>>>>>>>>>>> There needs to be some way to communicate the
> > > > >>>> IsolationLevel
> > > > >>>>>> down
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> RocksDBStore itself, so that the query can respect
> it.
> > > > >>>> Since
> > > > >>>>>> stores
> > > > >>>>>>>>>>>> are
> > > > >>>>>>>>>>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
> > > > >>>>>> MeteredStore,
> > > > >>>>>>>>>>>>> etc.),
> > > > >>>>>>>>>>>>>>> we need some way to deliver that information to the
> > > > >> bottom
> > > > >>>>>> layer.
> > > > >>>>>>>> For
> > > > >>>>>>>>>>>>> IQv2,
> > > > >>>>>>>>>>>>>>> we can use the existing State#query() method, but
> IQv1
> > > > >> has
> > > > >>>> no
> > > > >>>>>> way
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>> do
> > > > >>>>>>>>>>>>>>> this.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> A simple approach, which would potentially open up
> > > > >> other
> > > > >>>>>> options,
> > > > >>>>>>>>>>>> would
> > > > >>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> > > > >>>>>>>>>>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > > > >>>>>>>> ReadOnlyKeyValueStore
> > > > >>>>>>>>>>>>> (and
> > > > >>>>>>>>>>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore,
> > > > >>>> etc.).
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>>> As mentioned above, RocksDB WriteBatches are not
> > > > >>>> thread-safe,
> > > > >>>>>> which
> > > > >>>>>>>>>>>>> causes
> > > > >>>>>>>>>>>>>>> a problem if we want to provide READ_UNCOMMITTED
> > > > >>>> Iterators. I
> > > > >>>>>> also
> > > > >>>>>>>>>>>> had a
> > > > >>>>>>>>>>>>>>> look at RocksDB Transactions[1], but they solve a
> very
> > > > >>>>>> different
> > > > >>>>>>>>>>>>> problem,
> > > > >>>>>>>>>>>>>>> and have the same thread-safety issue.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> One possible approach that I mentioned is chaining
> > > > >>>>>> WriteBatches:
> > > > >>>>>>>>>> every
> > > > >>>>>>>>>>>>>>> time a new Interactive Query is received (i.e.
> > > > >>>> readOnlyView,
> > > > >>>>>> see
> > > > >>>>>>>>>>>> above,
> > > > >>>>>>>>>>>>>>> is called) we "freeze" the existing WriteBatch, and
> > > > >> start a
> > > > >>>>>> new one
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>> writes. The Interactive Query queries the "chain" of
> > > > >>>> previous
> > > > >>>>>>>>>>>>> WriteBatches
> > > > >>>>>>>>>>>>>>> + the underlying database; while the StreamThread
> > > > >> starts
> > > > >>>>>> writing to
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> *new* WriteBatch. On-commit, the StreamThread would
> > > > >> write
> > > > >>>> *all*
> > > > >>>>>>>>>>>>>>> WriteBatches in the chain to the database (that have
> > > > >> not
> > > > >>>> yet
> > > > >>>>>> been
> > > > >>>>>>>>>>>>> written).
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> WriteBatches would be closed/freed only when they
> have
> > > > >> been
> > > > >>>>>> both
> > > > >>>>>>>>>>>>>>> committed, and all open Interactive Queries on them
> > > > >> have
> > > > >>>> been
> > > > >>>>>>>> closed.
> > > > >>>>>>>>>>>>> This
> > > > >>>>>>>>>>>>>>> would require some reference counting.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Obviously a drawback of this approach is the
> potential
> > > > >> for
> > > > >>>>>>>> increased
> > > > >>>>>>>>>>>>>>> memory usage: if an Interactive Query is long-lived,
> > > > >> for
> > > > >>>>>> example by
> > > > >>>>>>>>>>>>> doing a
> > > > >>>>>>>>>>>>>>> full scan over a large database, or even just pausing
> > > > >> in
> > > > >>>> the
> > > > >>>>>> middle
> > > > >>>>>>>>>> of
> > > > >>>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>> iteration, then the existing chain of WriteBatches
> > > > >> could be
> > > > >>>>>> kept
> > > > >>>>>>>>>>>> around
> > > > >>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>> a long time, potentially forever.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> A.
> > > > >>>>>>>>>>>>>>> Going off on a tangent, it looks like in addition to
> > > > >>>> supporting
> > > > >>>>>>>>>>>>>>> READ_COMMITTED queries, we could go further and
> support
> > > > >>>>>>>>>>>> REPEATABLE_READ
> > > > >>>>>>>>>>>>>>> queries (i.e. where subsequent reads to the same key
> > > > >> in the
> > > > >>>>>> same
> > > > >>>>>>>>>>>>>>> Interactive Query are guaranteed to yield the same
> > > > >> value)
> > > > >>>> by
> > > > >>>>>> making
> > > > >>>>>>>>>>>> use
> > > > >>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>> RocksDB Snapshots[2]. These are fairly lightweight,
> so
> > > > >> the
> > > > >>>>>>>>>> performance
> > > > >>>>>>>>>>>>>>> impact is likely to be negligible, but they do
> require
> > > > >>>> that the
> > > > >>>>>>>>>>>>> Interactive
> > > > >>>>>>>>>>>>>>> Query session can be explicitly closed.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> This could be achieved if we made the above
> > > > >> readOnlyView
> > > > >>>>>> interface
> > > > >>>>>>>>>>>> look
> > > > >>>>>>>>>>>>>>> more like:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> > > > >>>>>>>>>>>> ReadOnlyKeyValueStore<K,
> > > > >>>>>>>>>>>>>>> V>, AutoCloseable {}
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > > > >>>>>>>>>>>>>>>         ...
> > > > >>>>>>>>>>>>>>>         ReadOnlyKeyValueView<K, V>
> > > > >>>> readOnlyView(IsolationLevel
> > > > >>>>>>>>>>>>> isolationLevel);
> > > > >>>>>>>>>>>>>>> }
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> But this would be a breaking change, as existing IQv1
> > > > >>>> queries
> > > > >>>>>> are
> > > > >>>>>>>>>>>>>>> guaranteed to never call store.close(), and therefore
> > > > >> these
> > > > >>>>>> would
> > > > >>>>>>>>>> leak
> > > > >>>>>>>>>>>>>>> memory under REPEATABLE_READ.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> B.
> > > > >>>>>>>>>>>>>>> One thing that's notable: MyRocks states that they
> > > > >> support
> > > > >>>>>>>>>>>>> READ_COMMITTED
> > > > >>>>>>>>>>>>>>> and REPEATABLE_READ, but they make no mention of
> > > > >>>>>>>>>>>> READ_UNCOMMITTED[3][4].
> > > > >>>>>>>>>>>>>>> This could be because doing so is technically
> > > > >>>>>> difficult/impossible
> > > > >>>>>>>>>>>> using
> > > > >>>>>>>>>>>>>>> the primitives available in RocksDB.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Lucas, to address your points:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> U1.
> > > > >>>>>>>>>>>>>>> It's only "SHOULD" to permit alternative (i.e.
> > > > >> non-RocksDB)
> > > > >>>>>>>>>>>>>>> implementations of StateStore that do not support
> > > > >> atomic
> > > > >>>>>> writes.
> > > > >>>>>>>>>>>>> Obviously
> > > > >>>>>>>>>>>>>>> in those cases, the guarantees Kafka Streams
> > > > >>>> provides/expects
> > > > >>>>>> would
> > > > >>>>>>>>>> be
> > > > >>>>>>>>>>>>>>> relaxed. Do you think we should require all
> > > > >>>> implementations to
> > > > >>>>>>>>>> support
> > > > >>>>>>>>>>>>>>> atomic writes?
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> U2.
> > > > >>>>>>>>>>>>>>> Stores can support multiple IsolationLevels. As we've
> > > > >>>> discussed
> > > > >>>>>>>>>> above,
> > > > >>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> ideal scenario would be to specify the IsolationLevel
> > > > >> at
> > > > >>>>>>>> query-time.
> > > > >>>>>>>>>>>>>>> Failing that, I think the second-best approach is to
> > > > >>>> define the
> > > > >>>>>>>>>>>>>>> IsolationLevel for *all* queries based on the
> > > > >>>> processing.mode,
> > > > >>>>>>>> which
> > > > >>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>> what the default StateStoreContext#isolationLevel()
> > > > >>>> achieves.
> > > > >>>>>> Would
> > > > >>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>> prefer an alternative?
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> While the existing implementation is equivalent to
> > > > >>>>>>>> READ_UNCOMMITTED,
> > > > >>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>> can yield unexpected results/errors under EOS, if a
> > > > >>>>>> transaction is
> > > > >>>>>>>>>>>>> rolled
> > > > >>>>>>>>>>>>>>> back. While this would be a change in behaviour for
> > > > >> users,
> > > > >>>> it
> > > > >>>>>> would
> > > > >>>>>>>>>>>> look
> > > > >>>>>>>>>>>>>>> more like a bug fix than a breaking change. That
> said,
> > > > >> we
> > > > >>>>>> *could*
> > > > >>>>>>>>>> make
> > > > >>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>> configurable, and default to the existing behaviour
> > > > >>>>>>>>>> (READ_UNCOMMITTED)
> > > > >>>>>>>>>>>>>>> instead of inferring it from the processing.mode?
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> N1, N2.
> > > > >>>>>>>>>>>>>>> These were only primitives to avoid boxing costs, but
> > > > >> since
> > > > >>>>>> this is
> > > > >>>>>>>>>>>> not
> > > > >>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>> performance sensitive area, it should be fine to
> > > > >> change if
> > > > >>>>>> that's
> > > > >>>>>>>>>>>>> desirable.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> N3.
> > > > >>>>>>>>>>>>>>> It's because the store "manages its own offsets",
> which
> > > > >>>>>> includes
> > > > >>>>>>>> both
> > > > >>>>>>>>>>>>>>> committing the offset, *and providing it* via
> > > > >>>>>> getCommittedOffset().
> > > > >>>>>>>>>>>>>>> Personally, I think "managesOffsets" conveys this
> best,
> > > > >>>> but I
> > > > >>>>>> don't
> > > > >>>>>>>>>>>> mind
> > > > >>>>>>>>>>>>>>> changing it if the nomenclature is unclear.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Sorry for the massive emails/essays!
> > > > >>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> 1:
> > > > >> https://github.com/facebook/rocksdb/wiki/Transactions
> > > > >>>>>>>>>>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > > > >>>>>>>>>>>>>>> 3:
> > > > >>>>>>>>
> > > > >> https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > > >>>>>>>>>>>>>>> 4:
> > > > >>>> https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > > > >>>>>>>>>>>>>>> <lb...@confluent.io.invalid> wrote:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Hi Nick,
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> since I last read it in April, the KIP has become
> much
> > > > >>>>>> cleaner and
> > > > >>>>>>>>>>>>>>>> easier to read. Great work!
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> It feels to me the last big open point is whether we
> > > > >> can
> > > > >>>>>> implement
> > > > >>>>>>>>>>>>>>>> isolation level as a query parameter. I understand
> > > > >> that
> > > > >>>> there
> > > > >>>>>> are
> > > > >>>>>>>>>>>>>>>> implementation concerns, but as Colt says, it would
> > > > >> be a
> > > > >>>> great
> > > > >>>>>>>>>>>>>>>> addition, and would also simplify the migration path
> > > > >> for
> > > > >>>> this
> > > > >>>>>>>>>> change.
> > > > >>>>>>>>>>>>>>>> Is the implementation problem you mentioned caused
> by
> > > > >> the
> > > > >>>>>>>> WriteBatch
> > > > >>>>>>>>>>>>>>>> not having a notion of a snapshot, as the underlying
> > > > >> DB
> > > > >>>>>> iterator
> > > > >>>>>>>>>>>> does?
> > > > >>>>>>>>>>>>>>>> In that case, I am not sure a chain of WriteBatches
> > > > >> as you
> > > > >>>>>> propose
> > > > >>>>>>>>>>>>>>>> would fully solve the problem, but maybe I didn't
> dig
> > > > >>>> enough
> > > > >>>>>> into
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> details to fully understand it.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> If it's not possible to implement it now, would it
> be
> > > > >> an
> > > > >>>>>> option to
> > > > >>>>>>>>>>>>>>>> make sure in this KIP that we do not fully close the
> > > > >> door
> > > > >>>> on
> > > > >>>>>>>>>>>> per-query
> > > > >>>>>>>>>>>>>>>> isolation levels in the interface, as it may be
> > > > >> possible
> > > > >>>> to
> > > > >>>>>>>>>> implement
> > > > >>>>>>>>>>>>>>>> the missing primitives in RocksDB or Speedb in the
> > > > >> future.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Understanding:
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets
> to
> > > > >> be
> > > > >>>>>> persisted
> > > > >>>>>>>>>>>>>>>> atomically with the records?
> > > > >>>>>>>>>>>>>>>> * U2) Don't understand the default implementation of
> > > > >>>>>>>>>>>> `isolationLevel`.
> > > > >>>>>>>>>>>>>>>> The isolation level should be a property of the
> > > > >> underlying
> > > > >>>>>> store,
> > > > >>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>> not be defined by the default config? Existing
> stores
> > > > >>>> probably
> > > > >>>>>>>> don't
> > > > >>>>>>>>>>>>>>>> guarantee READ_COMMITTED, so the default should be
> to
> > > > >>>> return
> > > > >>>>>>>>>>>>>>>> READ_UNCOMMITTED.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Nits:
> > > > >>>>>>>>>>>>>>>> * N1) Could `getComittedOffset` use an
> `OptionalLong`
> > > > >>>> return
> > > > >>>>>> type,
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>> avoid the `null`?
> > > > >>>>>>>>>>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
> > > > >>>>>> `OptionalLong`
> > > > >>>>>>>>>>>>>>>> return type, to avoid the `-1`?
> > > > >>>>>>>>>>>>>>>> * N3) I don't understand why `managesOffsets` uses
> the
> > > > >>>>>> 'manage'
> > > > >>>>>>>>>> verb,
> > > > >>>>>>>>>>>>>>>> whereas all other methods use the "commits" verb.
> I'd
> > > > >>>> suggest
> > > > >>>>>>>>>>>>>>>> `commitsOffsets`.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Either way, it feels this KIP is very close to the
> > > > >> finish
> > > > >>>>>> line,
> > > > >>>>>>>> I'm
> > > > >>>>>>>>>>>>>>>> looking forward to seeing this in production!
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Cheers,
> > > > >>>>>>>>>>>>>>>> Lucas
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> > > > >>>>>> colt@littlehorse.io
> > > > >>>>>>>>>
> > > > >>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> Making IsolationLevel a query-time constraint,
> > > > >> rather
> > > > >>>> than
> > > > >>>>>>>> linking
> > > > >>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>> the processing.guarantee.
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> As I understand it, would this allow even a user of
> > > > >> EOS
> > > > >>>> to
> > > > >>>>>>>> control
> > > > >>>>>>>>>>>>>>>> whether
> > > > >>>>>>>>>>>>>>>>> reading committed or uncommitted records? If so, I
> am
> > > > >>>> highly
> > > > >>>>>> in
> > > > >>>>>>>>>>>> favor
> > > > >>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>> this.
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> I know that I was one of the early people to point
> > > > >> out
> > > > >>>> the
> > > > >>>>>>>> current
> > > > >>>>>>>>>>>>>>>>> shortcoming that IQ reads uncommitted records, but
> > > > >> just
> > > > >>>> this
> > > > >>>>>>>>>>>> morning I
> > > > >>>>>>>>>>>>>>>>> realized a pattern we use which means that (for
> > > > >> certain
> > > > >>>>>> queries)
> > > > >>>>>>>>>> our
> > > > >>>>>>>>>>>>>>>> system
> > > > >>>>>>>>>>>>>>>>> needs to be able to read uncommitted records, which
> > > > >> is
> > > > >>>> the
> > > > >>>>>>>> current
> > > > >>>>>>>>>>>>>>>> behavior
> > > > >>>>>>>>>>>>>>>>> of Kafka Streams in EOS.***
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> If IsolationLevel being a query-time decision
> allows
> > > > >> for
> > > > >>>>>> this,
> > > > >>>>>>>> then
> > > > >>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>> would be amazing. I would also vote that the
> default
> > > > >>>> behavior
> > > > >>>>>>>>>> should
> > > > >>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>> reading uncommitted records, because it is totally
> > > > >>>> possible
> > > > >>>>>> for a
> > > > >>>>>>>>>>>>> valid
> > > > >>>>>>>>>>>>>>>>> application to depend on that behavior, and
> breaking
> > > > >> it
> > > > >>>> in a
> > > > >>>>>>>> minor
> > > > >>>>>>>>>>>>>>>> release
> > > > >>>>>>>>>>>>>>>>> might be a bit strong.
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> *** (Note, for the curious reader....) Our
> > > > >> use-case/query
> > > > >>>>>> pattern
> > > > >>>>>>>>>>>> is a
> > > > >>>>>>>>>>>>>>>> bit
> > > > >>>>>>>>>>>>>>>>> complex, but reading "uncommitted" records is
> > > > >> actually
> > > > >>>> safe
> > > > >>>>>> in
> > > > >>>>>>>> our
> > > > >>>>>>>>>>>>> case
> > > > >>>>>>>>>>>>>>>>> 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 soon as the processor
> processes
> > > > >> the
> > > > >>>>>> result.
> > > > >>>>>>>> If
> > > > >>>>>>>>>>>>> they
> > > > >>>>>>>>>>>>>>>>> can't read uncommitted records, they may receive a
> > > > >> "201 -
> > > > >>>>>>>> Created"
> > > > >>>>>>>>>>>>>>>>> response, immediately followed by a "404 - 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 <
> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> > > > >>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> Addendum:
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> I think we would also face the same problem with
> the
> > > > >>>>>> approach
> > > > >>>>>>>> John
> > > > >>>>>>>>>>>>>>>> outlined
> > > > >>>>>>>>>>>>>>>>>> earlier (using the record cache as a transaction
> > > > >> buffer
> > > > >>>> and
> > > > >>>>>>>>>>>> flushing
> > > > >>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>> straight to SST files). This is because the record
> > > > >> cache
> > > > >>>>>> (the
> > > > >>>>>>>>>>>>>>>> ThreadCache
> > > > >>>>>>>>>>>>>>>>>> class) is not thread-safe, so every commit would
> > > > >>>> invalidate
> > > > >>>>>> open
> > > > >>>>>>>>>> IQ
> > > > >>>>>>>>>>>>>>>>>> Iterators in the same way that RocksDB
> WriteBatches
> > > > >> do.
> > > > >>>>>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > > > >>>>>>>>>>>> nick.telford@gmail.com>
> > > > >>>>>>>>>>>>>>>>>> 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.
> > > > >>>>>>>>>>>>>>>>>>> 2. Making IsolationLevel a query-time constraint,
> > > > >>>> rather
> > > > >>>>>> than
> > > > >>>>>>>>>>>>>>>> linking it
> > > > >>>>>>>>>>>>>>>>>>> to the processing.guarantee.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> There's a wrinkle that makes this a challenge:
> > > > >>>> Interactive
> > > > >>>>>>>>>> Queries
> > > > >>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>> open an Iterator, when using transactions and
> > > > >>>>>> READ_UNCOMMITTED.
> > > > >>>>>>>>>>>>>>>>>>> The problem is that under READ_UNCOMMITTED,
> queries
> > > > >>>> need
> > > > >>>>>> to be
> > > > >>>>>>>>>>>> able
> > > > >>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>> read records from the currently uncommitted
> > > > >> transaction
> > > > >>>>>> buffer
> > > > >>>>>>>>>>>>>>>>>>> (WriteBatch). This includes for Iterators, which
> > > > >> should
> > > > >>>>>> iterate
> > > > >>>>>>>>>>>>>>>> both the
> > > > >>>>>>>>>>>>>>>>>>> transaction buffer and underlying database (using
> > > > >>>>>>>>>>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> The issue is that when the StreamThread commits,
> it
> > > > >>>> writes
> > > > >>>>>> the
> > > > >>>>>>>>>>>>>>>> current
> > > > >>>>>>>>>>>>>>>>>>> WriteBatch to RocksDB *and then clears the
> > > > >> WriteBatch*.
> > > > >>>>>>>> Clearing
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>> WriteBatch while an Interactive Query holds an
> open
> > > > >>>>>> Iterator on
> > > > >>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>> invalidate the Iterator. Worse, it turns out that
> > > > >>>> Iterators
> > > > >>>>>>>> over
> > > > >>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>> WriteBatch become invalidated not just when the
> > > > >>>> WriteBatch
> > > > >>>>>> is
> > > > >>>>>>>>>>>>>>>> cleared,
> > > > >>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>> also when the Iterators' current key receives a
> new
> > > > >>>> write.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Now that I'm writing this, I remember that this
> is
> > > > >> the
> > > > >>>>>> major
> > > > >>>>>>>>>>>> reason
> > > > >>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>> switched the original design from having a
> > > > >> query-time
> > > > >>>>>>>>>>>>>>>> IsolationLevel to
> > > > >>>>>>>>>>>>>>>>>>> having the IsolationLevel linked to the
> > > > >>>> transactionality
> > > > >>>>>> of the
> > > > >>>>>>>>>>>>>>>> stores
> > > > >>>>>>>>>>>>>>>>>>> themselves.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> It *might* be possible to resolve this, by
> having a
> > > > >>>>>> "chain" of
> > > > >>>>>>>>>>>>>>>>>>> WriteBatches, with the StreamThread switching to
> a
> > > > >> new
> > > > >>>>>>>> WriteBatch
> > > > >>>>>>>>>>>>>>>>>> whenever
> > > > >>>>>>>>>>>>>>>>>>> a new Interactive Query attempts to read from the
> > > > >>>>>> database, but
> > > > >>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>> could
> > > > >>>>>>>>>>>>>>>>>>> cause some performance problems/memory pressure
> > > > >> when
> > > > >>>>>> subjected
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>> high
> > > > >>>>>>>>>>>>>>>>>>> Interactive Query load. It would also reduce the
> > > > >>>>>> efficiency of
> > > > >>>>>>>>>>>>>>>>>> WriteBatches
> > > > >>>>>>>>>>>>>>>>>>> on-commit, as we'd have to write N WriteBatches,
> > > > >> where
> > > > >>>> N
> > > > >>>>>> is the
> > > > >>>>>>>>>>>>>>>> number of
> > > > >>>>>>>>>>>>>>>>>>> Interactive Queries since the last commit.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> I realise this is getting into the weeds of the
> > > > >>>>>> implementation,
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>> you'd
> > > > >>>>>>>>>>>>>>>>>>> rather we focus on the API for now, but I think
> > > > >> it's
> > > > >>>>>> important
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>> consider
> > > > >>>>>>>>>>>>>>>>>>> how to implement the desired API, in case we come
> > > > >> up
> > > > >>>> with
> > > > >>>>>> an
> > > > >>>>>>>> API
> > > > >>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>> cannot be implemented efficiently, or even at
> all!
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Thoughts?
> > > > >>>>>>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> > > > >>>>>>>> cadonna@apache.org
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> 6.
> > > > >>>>>>>>>>>>>>>>>>>> Of course, you are right! My bad!
> > > > >>>>>>>>>>>>>>>>>>>> Wiping out the state in the downgrading case is
> > > > >> fine.
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> 3a.
> > > > >>>>>>>>>>>>>>>>>>>> Focus on the public facing changes for the KIP.
> We
> > > > >>>> will
> > > > >>>>>> manage
> > > > >>>>>>>>>> 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.
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>>>> Bruno
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > > > >>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> Thanks for getting back to me!
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>>>>>>>>> The fact that implementations can always track
> > > > >>>> estimated
> > > > >>>>>>>> memory
> > > > >>>>>>>>>>>>>>>> usage
> > > > >>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as
> > > > >> an
> > > > >>>>>> option,
> > > > >>>>>>>> and
> > > > >>>>>>>>>>>>>>>> I'll
> > > > >>>>>>>>>>>>>>>>>>>> clarify
> > > > >>>>>>>>>>>>>>>>>>>>> the JavaDoc that 0 is not just for
> > > > >> non-transactional
> > > > >>>>>> stores,
> > > > >>>>>>>>>>>>>>>> which is
> > > > >>>>>>>>>>>>>>>>>>>>> currently misleading.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> 6.
> > > > >>>>>>>>>>>>>>>>>>>>> The problem with catching the exception in the
> > > > >>>> downgrade
> > > > >>>>>>>>>> process
> > > > >>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>> would require new code in the Kafka version
> being
> > > > >>>>>> downgraded
> > > > >>>>>>>>>> to.
> > > > >>>>>>>>>>>>>>>> Since
> > > > >>>>>>>>>>>>>>>>>>>>> users could conceivably downgrade to almost
> *any*
> > > > >>>> older
> > > > >>>>>>>> version
> > > > >>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>> Kafka
> > > > >>>>>>>>>>>>>>>>>>>>> Streams, I'm not sure how we could add that
> code?
> > > > >>>>>>>>>>>>>>>>>>>>> The only way I can think of doing it would be
> to
> > > > >>>> provide
> > > > >>>>>> a
> > > > >>>>>>>>>>>>>>>> dedicated
> > > > >>>>>>>>>>>>>>>>>>>>> downgrade tool, that goes through every local
> > > > >> store
> > > > >>>> and
> > > > >>>>>>>> removes
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> offsets column families. But that seems like an
> > > > >>>>>> unnecessary
> > > > >>>>>>>>>>>>>>>> amount of
> > > > >>>>>>>>>>>>>>>>>>>> extra
> > > > >>>>>>>>>>>>>>>>>>>>> code to maintain just to handle a somewhat
> niche
> > > > >>>>>> situation,
> > > > >>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> alternative (automatically wipe and restore
> > > > >> stores)
> > > > >>>>>> should be
> > > > >>>>>>>>>>>>>>>>>>>> acceptable.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've
> > > > >>>> requested.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> 3a.
> > > > >>>>>>>>>>>>>>>>>>>>> I agree that IsolationLevel makes more sense at
> > > > >>>>>> query-time,
> > > > >>>>>>>> and
> > > > >>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>> actually
> > > > >>>>>>>>>>>>>>>>>>>>> initially attempted to place the IsolationLevel
> > > > >> at
> > > > >>>>>>>> query-time,
> > > > >>>>>>>>>>>>>>>> but I
> > > > >>>>>>>>>>>>>>>>>> ran
> > > > >>>>>>>>>>>>>>>>>>>>> into some problems:
> > > > >>>>>>>>>>>>>>>>>>>>> - The key issue is that, under ALOS we're not
> > > > >> staging
> > > > >>>>>> writes
> > > > >>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>> transactions, so can't perform writes at the
> > > > >>>>>> READ_COMMITTED
> > > > >>>>>>>>>>>>>>>> isolation
> > > > >>>>>>>>>>>>>>>>>>>>> level. However, this may be addressed if we
> > > > >> decide to
> > > > >>>>>>>> *always*
> > > > >>>>>>>>>>>>>>>> use
> > > > >>>>>>>>>>>>>>>>>>>>> transactions as discussed under 3b.
> > > > >>>>>>>>>>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> > > > >>>> implementations. I
> > > > >>>>>>>>>> remember
> > > > >>>>>>>>>>>>>>>>>> having
> > > > >>>>>>>>>>>>>>>>>>>>> some difficulty understanding the IQv1
> internals,
> > > > >>>> which
> > > > >>>>>> made
> > > > >>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>> difficult
> > > > >>>>>>>>>>>>>>>>>>>>> to determine what needed to be changed.
> However,
> > > > >> I
> > > > >>>>>> *think*
> > > > >>>>>>>> this
> > > > >>>>>>>>>>>>>>>> can be
> > > > >>>>>>>>>>>>>>>>>>>>> addressed for both implementations by wrapping
> > > > >> the
> > > > >>>>>>>> RocksDBStore
> > > > >>>>>>>>>>>>>>>> in an
> > > > >>>>>>>>>>>>>>>>>>>>> IsolationLevel-dependent wrapper, that
> overrides
> > > > >> read
> > > > >>>>>> methods
> > > > >>>>>>>>>>>>>>>> (get,
> > > > >>>>>>>>>>>>>>>>>>>> etc.)
> > > > >>>>>>>>>>>>>>>>>>>>> to either read directly from the database or
> > > > >> from the
> > > > >>>>>> ongoing
> > > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > > >>>>>>>>>>>>>>>>>>>>> But IQv1 might still be difficult.
> > > > >>>>>>>>>>>>>>>>>>>>> - If IsolationLevel becomes a query constraint,
> > > > >> then
> > > > >>>> all
> > > > >>>>>>>> other
> > > > >>>>>>>>>>>>>>>>>>>> StateStores
> > > > >>>>>>>>>>>>>>>>>>>>> will need to respect it, including the
> in-memory
> > > > >>>> stores.
> > > > >>>>>> This
> > > > >>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>> require
> > > > >>>>>>>>>>>>>>>>>>>>> us to adapt in-memory stores to stage their
> > > > >> writes so
> > > > >>>>>> they
> > > > >>>>>>>> can
> > > > >>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>> isolated
> > > > >>>>>>>>>>>>>>>>>>>>> from READ_COMMITTTED queries. It would also
> > > > >> become an
> > > > >>>>>>>> important
> > > > >>>>>>>>>>>>>>>>>>>>> consideration for third-party stores on
> upgrade,
> > > > >> as
> > > > >>>>>> without
> > > > >>>>>>>>>>>>>>>> changes,
> > > > >>>>>>>>>>>>>>>>>>>> they
> > > > >>>>>>>>>>>>>>>>>>>>> would not support READ_COMMITTED queries
> > > > >> correctly.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> Ultimately, I may need some help making the
> > > > >> necessary
> > > > >>>>>> change
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>>>>>> IQv1
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>> support this, but I don't think it's
> > > > >> fundamentally
> > > > >>>>>>>> impossible,
> > > > >>>>>>>>>>>>>>>> if we
> > > > >>>>>>>>>>>>>>>>>>>> want
> > > > >>>>>>>>>>>>>>>>>>>>> to pursue this route.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> 3b.
> > > > >>>>>>>>>>>>>>>>>>>>> The main reason I chose to keep ALOS
> > > > >> un-transactional
> > > > >>>>>> was to
> > > > >>>>>>>>>>>>>>>> minimize
> > > > >>>>>>>>>>>>>>>>>>>>> behavioural change for most users (I believe
> most
> > > > >>>> Streams
> > > > >>>>>>>> users
> > > > >>>>>>>>>>>>>>>> use
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> default configuration, which is ALOS). That
> said,
> > > > >>>> it's
> > > > >>>>>> clear
> > > > >>>>>>>>>>>>>>>> that if
> > > > >>>>>>>>>>>>>>>>>>>> ALOS
> > > > >>>>>>>>>>>>>>>>>>>>> also used transactional stores, the only change
> > > > >> in
> > > > >>>>>> behaviour
> > > > >>>>>>>>>>>>>>>> would be
> > > > >>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>> it would become *more correct*, which could be
> > > > >>>>>> considered a
> > > > >>>>>>>>>> "bug
> > > > >>>>>>>>>>>>>>>> fix"
> > > > >>>>>>>>>>>>>>>>>> by
> > > > >>>>>>>>>>>>>>>>>>>>> users, rather than a change they need to
> handle.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> I believe that performance using transactions
> > > > >> (aka.
> > > > >>>>>> RocksDB
> > > > >>>>>>>>>>>>>>>>>>>> WriteBatches)
> > > > >>>>>>>>>>>>>>>>>>>>> should actually be *better* than the un-batched
> > > > >>>>>> write-path
> > > > >>>>>>>> that
> > > > >>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>> currently used[1]. The only "performance"
> > > > >>>> consideration
> > > > >>>>>> will
> > > > >>>>>>>> be
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> increased memory usage that transactions
> require.
> > > > >>>> Given
> > > > >>>>>> the
> > > > >>>>>>>>>>>>>>>>>> mitigations
> > > > >>>>>>>>>>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>> this memory that we have in place, I would
> expect
> > > > >>>> that
> > > > >>>>>> this
> > > > >>>>>>>> is
> > > > >>>>>>>>>>>>>>>> not a
> > > > >>>>>>>>>>>>>>>>>>>>> problem for most users.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> If we're happy to do so, we can make ALOS also
> > > > >> use
> > > > >>>>>>>>>> transactions.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> Link 1:
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>
> > > > >>>>
> > > > >>
> > >
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> > > > >>>>>>>>>>>> cadonna@apache.org
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Thanks for the updates and sorry for the delay
> > > > >> on my
> > > > >>>>>> side!
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >>>>>>>>>>>>>>>>>>>>>> Making the default implementation for flush()
> a
> > > > >>>> no-op
> > > > >>>>>> sounds
> > > > >>>>>>>>>>>>>>>> good to
> > > > >>>>>>>>>>>>>>>>>>>> me.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>>>>>>>>>> I think what was bugging me here is that a
> > > > >>>> third-party
> > > > >>>>>> state
> > > > >>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>> needs
> > > > >>>>>>>>>>>>>>>>>>>>>> to implement the state store interface. That
> > > > >> means
> > > > >>>> they
> > > > >>>>>> need
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>> implement a wrapper around the actual state
> > > > >> store
> > > > >>>> as we
> > > > >>>>>> do
> > > > >>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>> RocksDB
> > > > >>>>>>>>>>>>>>>>>>>>>> with RocksDBStore. So, a third-party state
> > > > >> store can
> > > > >>>>>> always
> > > > >>>>>>>>>>>>>>>> estimate
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> uncommitted bytes, if it wants, because the
> > > > >> wrapper
> > > > >>>> can
> > > > >>>>>>>> record
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> added
> > > > >>>>>>>>>>>>>>>>>>>>>> bytes.
> > > > >>>>>>>>>>>>>>>>>>>>>> One case I can think of where returning -1
> makes
> > > > >>>> sense
> > > > >>>>>> is
> > > > >>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>> Streams
> > > > >>>>>>>>>>>>>>>>>>>>>> does not need to estimate the size of the
> write
> > > > >>>> batch
> > > > >>>>>> and
> > > > >>>>>>>>>>>>>>>> trigger
> > > > >>>>>>>>>>>>>>>>>>>>>> extraordinary commits, because the third-party
> > > > >> state
> > > > >>>>>> store
> > > > >>>>>>>>>>>>>>>> takes care
> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>> memory. But in that case the method could also
> > > > >> just
> > > > >>>>>> return
> > > > >>>>>>>> 0.
> > > > >>>>>>>>>>>>>>>> Even
> > > > >>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>> case would be better solved with a method that
> > > > >>>> returns
> > > > >>>>>>>> whether
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>>>> store manages itself the memory used for
> > > > >> uncommitted
> > > > >>>>>> bytes
> > > > >>>>>>>> or
> > > > >>>>>>>>>>>>>>>> not.
> > > > >>>>>>>>>>>>>>>>>>>>>> Said that, I am fine with keeping the -1
> return
> > > > >>>> value,
> > > > >>>>>> I was
> > > > >>>>>>>>>>>>>>>> just
> > > > >>>>>>>>>>>>>>>>>>>>>> wondering when and if it will be used.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Regarding returning 0 for transactional state
> > > > >> stores
> > > > >>>>>> when
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>>> batch
> > > > >>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>> empty, I was just wondering because you
> > > > >> explicitly
> > > > >>>>>> stated
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> "or {@code 0} if this StateStore does not
> > > > >> support
> > > > >>>>>>>>>>>> transactions."
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> So it seemed to me returning 0 could only
> > > > >> happen for
> > > > >>>>>>>>>>>>>>>>>> non-transactional
> > > > >>>>>>>>>>>>>>>>>>>>>> state stores.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> a) What do you think if we move the isolation
> > > > >> level
> > > > >>>> to
> > > > >>>>>> IQ
> > > > >>>>>>>> (v1
> > > > >>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>> v2)?
> > > > >>>>>>>>>>>>>>>>>>>>>> In the end this is the only component that
> > > > >> really
> > > > >>>> needs
> > > > >>>>>> to
> > > > >>>>>>>>>>>>>>>> specify
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> isolation level. It is similar to the Kafka
> > > > >> consumer
> > > > >>>>>> that
> > > > >>>>>>>> can
> > > > >>>>>>>>>>>>>>>> choose
> > > > >>>>>>>>>>>>>>>>>>>>>> with what isolation level to read the input
> > > > >> topic.
> > > > >>>>>>>>>>>>>>>>>>>>>> For IQv1 the isolation level should go into
> > > > >>>>>>>>>>>>>>>> StoreQueryParameters. For
> > > > >>>>>>>>>>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> b) Point a) raises the question what should
> > > > >> happen
> > > > >>>>>> during
> > > > >>>>>>>>>>>>>>>>>> at-least-once
> > > > >>>>>>>>>>>>>>>>>>>>>> processing when the state store does not use
> > > > >>>>>> transactions?
> > > > >>>>>>>>>> John
> > > > >>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> past proposed to also use transactions on
> state
> > > > >>>> stores
> > > > >>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>>>> at-least-once. I like that idea, because it
> > > > >> avoids
> > > > >>>>>>>> aggregating
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>> same
> > > > >>>>>>>>>>>>>>>>>>>>>> records over and over again in the case of a
> > > > >>>> failure. We
> > > > >>>>>>>> had a
> > > > >>>>>>>>>>>>>>>> case
> > > > >>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>> the past where a Streams applications in
> > > > >>>> at-least-once
> > > > >>>>>> mode
> > > > >>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>> failing
> > > > >>>>>>>>>>>>>>>>>>>>>> continuously for some reasons I do not
> remember
> > > > >>>> before
> > > > >>>>>>>>>>>>>>>> committing the
> > > > >>>>>>>>>>>>>>>>>>>>>> offsets. After each failover, the app
> aggregated
> > > > >>>> again
> > > > >>>>>> and
> > > > >>>>>>>>>>>>>>>> again the
> > > > >>>>>>>>>>>>>>>>>>>>>> same records. Of course the aggregate
> increased
> > > > >> to
> > > > >>>> very
> > > > >>>>>>>> wrong
> > > > >>>>>>>>>>>>>>>> values
> > > > >>>>>>>>>>>>>>>>>>>>>> just because of the failover. With
> transactions
> > > > >> on
> > > > >>>> the
> > > > >>>>>> state
> > > > >>>>>>>>>>>>>>>> stores
> > > > >>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>> could have avoided this. The app would have
> > > > >> output
> > > > >>>> the
> > > > >>>>>> same
> > > > >>>>>>>>>>>>>>>> aggregate
> > > > >>>>>>>>>>>>>>>>>>>>>> multiple times (i.e., after each failover) but
> > > > >> at
> > > > >>>> least
> > > > >>>>>> the
> > > > >>>>>>>>>>>>>>>> value of
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> aggregate would not depend on the number of
> > > > >>>> failovers.
> > > > >>>>>>>>>>>>>>>> Outputting the
> > > > >>>>>>>>>>>>>>>>>>>>>> same aggregate multiple times would be
> incorrect
> > > > >>>> under
> > > > >>>>>>>>>>>>>>>> exactly-once
> > > > >>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>>>> it is OK for at-least-once.
> > > > >>>>>>>>>>>>>>>>>>>>>> If it makes sense to add a config to turn on
> > > > >> and off
> > > > >>>>>>>>>>>>>>>> transactions on
> > > > >>>>>>>>>>>>>>>>>>>>>> state stores under at-least-once or just use
> > > > >>>>>> transactions in
> > > > >>>>>>>>>>>>>>>> any case
> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>> a question we should also discuss in this KIP.
> > > > >> It
> > > > >>>>>> depends a
> > > > >>>>>>>>>> bit
> > > > >>>>>>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I
> would
> > > > >>>> add a
> > > > >>>>>>>> config.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> 4.
> > > > >>>>>>>>>>>>>>>>>>>>>> Your points are all valid. I tend to say to
> > > > >> keep the
> > > > >>>>>> metrics
> > > > >>>>>>>>>>>>>>>> around
> > > > >>>>>>>>>>>>>>>>>>>>>> flush() until we remove flush() completely
> from
> > > > >> the
> > > > >>>>>>>> interface.
> > > > >>>>>>>>>>>>>>>> Calls
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>> flush() might still exist since existing
> > > > >> processors
> > > > >>>>>> might
> > > > >>>>>>>>>> still
> > > > >>>>>>>>>>>>>>>> call
> > > > >>>>>>>>>>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For
> > > > >>>> sure, we
> > > > >>>>>> need
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> document
> > > > >>>>>>>>>>>>>>>>>>>>>> how the metrics change due to the transactions
> > > > >> in
> > > > >>>> the
> > > > >>>>>>>> upgrade
> > > > >>>>>>>>>>>>>>>> notes.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> 5.
> > > > >>>>>>>>>>>>>>>>>>>>>> I see. Then you should describe how the
> > > > >> .position
> > > > >>>> files
> > > > >>>>>> are
> > > > >>>>>>>>>>>>>>>> handled
> > > > >>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>> a dedicated section of the KIP or incorporate
> > > > >> the
> > > > >>>>>>>> description
> > > > >>>>>>>>>>>>>>>> in the
> > > > >>>>>>>>>>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only
> > > > >>>>>> mentioning it
> > > > >>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> "Compatibility, 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:
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>
> > > > >>>>
> > > > >>
> > >
> >
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> It is a bit brittle, but it works.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for taking the time to review the KIP.
> > > > >> I'm
> > > > >>>> back
> > > > >>>>>> from
> > > > >>>>>>>>>>>>>>>> leave
> > > > >>>>>>>>>>>>>>>>>> now
> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>> intend to move this forwards as quickly as I
> > > > >> can.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Addressing your points:
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >>>>>>>>>>>>>>>>>>>>>>> Because flush() is part of the StateStore
> API,
> > > > >> it's
> > > > >>>>>> exposed
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>> custom
> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors, which might be making calls to
> > > > >> flush().
> > > > >>>>>> This
> > > > >>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>> actually
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> case in a few integration tests.
> > > > >>>>>>>>>>>>>>>>>>>>>>> To maintain as much compatibility as
> possible,
> > > > >> I'd
> > > > >>>>>> prefer
> > > > >>>>>>>> not
> > > > >>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>> make
> > > > >>>>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>> an UnsupportedOperationException, as it will
> > > > >> cause
> > > > >>>>>>>> previously
> > > > >>>>>>>>>>>>>>>>>> working
> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors to start throwing exceptions at
> > > > >> runtime.
> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that it doesn't make sense for it to
> > > > >> proxy
> > > > >>>>>>>> commit(),
> > > > >>>>>>>>>>>>>>>> though,
> > > > >>>>>>>>>>>>>>>>>>>> as
> > > > >>>>>>>>>>>>>>>>>>>>>>> that would cause it to violate the
> "StateStores
> > > > >>>> commit
> > > > >>>>>> only
> > > > >>>>>>>>>>>>>>>> when the
> > > > >>>>>>>>>>>>>>>>>>>> Task
> > > > >>>>>>>>>>>>>>>>>>>>>>> commits" rule.
> > > > >>>>>>>>>>>>>>>>>>>>>>> Instead, I think we should make this a no-op.
> > > > >> That
> > > > >>>> way,
> > > > >>>>>>>>>>>>>>>> existing
> > > > >>>>>>>>>>>>>>>>>> user
> > > > >>>>>>>>>>>>>>>>>>>>>>> Processors will continue to work as-before,
> > > > >> without
> > > > >>>>>>>> violation
> > > > >>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>>>>> consistency that would be caused by premature
> > > > >>>>>> flush/commit
> > > > >>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>> StateStore
> > > > >>>>>>>>>>>>>>>>>>>>>>> data to disk.
> > > > >>>>>>>>>>>>>>>>>>>>>>> What do you think?
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
> > > > >>>>>> implementation
> > > > >>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, but is unable to estimate the
> > > > >>>>>> uncommitted
> > > > >>>>>>>>>>>> memory
> > > > >>>>>>>>>>>>>>>>>> usage,
> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> method will return -1.
> > > > >>>>>>>>>>>>>>>>>>>>>>> The intention here is to permit third-party
> > > > >>>>>> implementations
> > > > >>>>>>>>>>>>>>>> that may
> > > > >>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>> able to estimate memory usage.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been
> > > > >> written to
> > > > >>>> the
> > > > >>>>>>>> store
> > > > >>>>>>>>>>>>>>>> yet. I
> > > > >>>>>>>>>>>>>>>>>>>>>> thought
> > > > >>>>>>>>>>>>>>>>>>>>>>> that was implied by "This method will return
> an
> > > > >>>>>>>> approximation
> > > > >>>>>>>>>>>>>>>> of the
> > > > >>>>>>>>>>>>>>>>>>>>>> memory
> > > > >>>>>>>>>>>>>>>>>>>>>>> would be freed by the next call to {@link
> > > > >>>>>> #commit(Map)}"
> > > > >>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>> "@return
> > > > >>>>>>>>>>>>>>>>>>>> The
> > > > >>>>>>>>>>>>>>>>>>>>>>> approximate size of all records awaiting
> {@link
> > > > >>>>>>>>>>>> #commit(Map)}",
> > > > >>>>>>>>>>>>>>>>>>>> however,
> > > > >>>>>>>>>>>>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you
> > > > >> think
> > > > >>>> this
> > > > >>>>>> is
> > > > >>>>>>>>>>>>>>>> unclear?
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>>>>>>>>> I realise this is probably the most
> contentious
> > > > >>>> point
> > > > >>>>>> in my
> > > > >>>>>>>>>>>>>>>> design,
> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>> I'm
> > > > >>>>>>>>>>>>>>>>>>>>>>> open to changing it if I'm unable to convince
> > > > >> you
> > > > >>>> of
> > > > >>>>>> the
> > > > >>>>>>>>>>>>>>>> benefits.
> > > > >>>>>>>>>>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > > > >>>>>>>>>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are
> directly
> > > > >>>> provided
> > > > >>>>>>>>>>>>>>>> StateStores
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>> query, and it may be important for users to
> > > > >>>>>>>> programmatically
> > > > >>>>>>>>>>>>>>>> know
> > > > >>>>>>>>>>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>>>>>>>>>>> mode the StateStore is operating under. If we
> > > > >>>> simply
> > > > >>>>>>>> provide
> > > > >>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the
> > > > >>>> internal
> > > > >>>>>>>> streams
> > > > >>>>>>>>>>>>>>>>>>>> engine), or
> > > > >>>>>>>>>>>>>>>>>>>>>>> similar, then users will need to understand
> the
> > > > >>>>>> operation
> > > > >>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>> consequences
> > > > >>>>>>>>>>>>>>>>>>>>>>> of each available processing mode and how it
> > > > >>>> pertains
> > > > >>>>>> to
> > > > >>>>>>>>>> their
> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Interactive Query users aren't the only
> people
> > > > >> that
> > > > >>>>>> care
> > > > >>>>>>>>>> about
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> processing.mode/IsolationLevel of a
> StateStore:
> > > > >>>>>>>> implementers
> > > > >>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>> custom
> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStores also need to understand the
> > > > >> behaviour
> > > > >>>>>> expected
> > > > >>>>>>>> of
> > > > >>>>>>>>>>>>>>>> their
> > > > >>>>>>>>>>>>>>>>>>>>>>> implementation. KIP-892 introduces some
> > > > >> assumptions
> > > > >>>>>> into
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>>> Streams
> > > > >>>>>>>>>>>>>>>>>>>>>> Engine
> > > > >>>>>>>>>>>>>>>>>>>>>>> about how StateStores operate under each
> > > > >> processing
> > > > >>>>>> mode,
> > > > >>>>>>>> and
> > > > >>>>>>>>>>>>>>>> it's
> > > > >>>>>>>>>>>>>>>>>>>>>>> important that custom implementations adhere
> to
> > > > >>>> those
> > > > >>>>>>>>>>>>>>>> assumptions in
> > > > >>>>>>>>>>>>>>>>>>>>>> order
> > > > >>>>>>>>>>>>>>>>>>>>>>> to maintain the consistency guarantees.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> IsolationLevels provide a high-level contract
> > > > >> on
> > > > >>>> the
> > > > >>>>>>>>>> behaviour
> > > > >>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> StateStore: a user knows that under
> > > > >> READ_COMMITTED,
> > > > >>>>>> they
> > > > >>>>>>>> will
> > > > >>>>>>>>>>>>>>>> see
> > > > >>>>>>>>>>>>>>>>>>>> writes
> > > > >>>>>>>>>>>>>>>>>>>>>>> only after the Task has committed, and under
> > > > >>>>>>>> READ_UNCOMMITTED
> > > > >>>>>>>>>>>>>>>> they
> > > > >>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>> see
> > > > >>>>>>>>>>>>>>>>>>>>>>> writes immediately. No understanding of the
> > > > >>>> details of
> > > > >>>>>> each
> > > > >>>>>>>>>>>>>>>>>>>>>> processing.mode
> > > > >>>>>>>>>>>>>>>>>>>>>>> is required, either for IQ users or
> StateStore
> > > > >>>>>>>> implementers.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> An argument can be made that these
> contractual
> > > > >>>>>> guarantees
> > > > >>>>>>>> can
> > > > >>>>>>>>>>>>>>>> simply
> > > > >>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>> documented for the processing.mode (i.e. that
> > > > >>>>>> exactly-once
> > > > >>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED
> and
> > > > >>>>>>>> at-least-once
> > > > >>>>>>>>>>>>>>>> behaves
> > > > >>>>>>>>>>>>>>>>>>>> like
> > > > >>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several
> small
> > > > >>>> issues
> > > > >>>>>> with
> > > > >>>>>>>>>>>>>>>> this I'd
> > > > >>>>>>>>>>>>>>>>>>>>>> prefer
> > > > >>>>>>>>>>>>>>>>>>>>>>> to avoid:
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>          - Where would we document these
> > > > >> contracts,
> > > > >>>> in
> > > > >>>>>> a way
> > > > >>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>> difficult
> > > > >>>>>>>>>>>>>>>>>>>>>>>          for users/implementers to
> miss/ignore?
> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It's not clear to users that the
> > > > >>>> processing
> > > > >>>>>> mode
> > > > >>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>> communicating
> > > > >>>>>>>>>>>>>>>>>>>>>>>          an expectation of read isolation,
> > > > >> unless
> > > > >>>> they
> > > > >>>>>> read
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> documentation. Users
> > > > >>>>>>>>>>>>>>>>>>>>>>>          rarely consult documentation unless
> > > > >> they
> > > > >>>> feel
> > > > >>>>>> they
> > > > >>>>>>>>>> need
> > > > >>>>>>>>>>>>>>>> to, so
> > > > >>>>>>>>>>>>>>>>>>>> it's
> > > > >>>>>>>>>>>>>>>>>>>>>> likely
> > > > >>>>>>>>>>>>>>>>>>>>>>>          this detail would get missed by many
> > > > >> users.
> > > > >>>>>>>>>>>>>>>>>>>>>>>          - It tightly couples processing
> modes
> > > > >> to
> > > > >>>> read
> > > > >>>>>>>>>> isolation.
> > > > >>>>>>>>>>>>>>>> Adding
> > > > >>>>>>>>>>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>>>>>>>>>          processing modes, or changing the
> read
> > > > >>>>>> isolation of
> > > > >>>>>>>>>>>>>>>> existing
> > > > >>>>>>>>>>>>>>>>>>>>>> processing
> > > > >>>>>>>>>>>>>>>>>>>>>>>          modes would be difficult/impossible.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Ultimately, the cost of introducing
> > > > >>>> IsolationLevels is
> > > > >>>>>>>> just a
> > > > >>>>>>>>>>>>>>>> single
> > > > >>>>>>>>>>>>>>>>>>>>>>> method, since we re-use the existing
> > > > >> IsolationLevel
> > > > >>>>>> enum
> > > > >>>>>>>> from
> > > > >>>>>>>>>>>>>>>> Kafka.
> > > > >>>>>>>>>>>>>>>>>>>> This
> > > > >>>>>>>>>>>>>>>>>>>>>>> gives us a clear place to document the
> > > > >> contractual
> > > > >>>>>>>> guarantees
> > > > >>>>>>>>>>>>>>>>>> expected
> > > > >>>>>>>>>>>>>>>>>>>>>>> of/provided by StateStores, that is
> accessible
> > > > >>>> both by
> > > > >>>>>> the
> > > > >>>>>>>>>>>>>>>>>> StateStore
> > > > >>>>>>>>>>>>>>>>>>>>>>> itself, and by IQ users.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> (Writing this I've just realised that the
> > > > >>>> StateStore
> > > > >>>>>> and IQ
> > > > >>>>>>>>>>>>>>>> APIs
> > > > >>>>>>>>>>>>>>>>>>>> actually
> > > > >>>>>>>>>>>>>>>>>>>>>>> don't provide access to StateStoreContext
> that
> > > > >> IQ
> > > > >>>> users
> > > > >>>>>>>> would
> > > > >>>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>>>>> direct
> > > > >>>>>>>>>>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> > > > >>>>>>>>>> isolationLevel()
> > > > >>>>>>>>>>>>>>>>>> itself
> > > > >>>>>>>>>>>>>>>>>>>>>> too?)
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 4.
> > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the
> metrics
> > > > >>>> in-place
> > > > >>>>>>>>>>>>>>>> either, as
> > > > >>>>>>>>>>>>>>>>>>>> it's a
> > > > >>>>>>>>>>>>>>>>>>>>>>> backwards incompatible change. My concern is
> > > > >> that,
> > > > >>>> if
> > > > >>>>>> we
> > > > >>>>>>>>>> leave
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> existing
> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics in place, they will be
> > > > >> confusing to
> > > > >>>>>> users.
> > > > >>>>>>>>>>>>>>>> Right
> > > > >>>>>>>>>>>>>>>>>> now,
> > > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics record explicit flushes to
> > > > >> disk,
> > > > >>>> but
> > > > >>>>>> under
> > > > >>>>>>>>>>>>>>>> KIP-892,
> > > > >>>>>>>>>>>>>>>>>>>> even
> > > > >>>>>>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() will not explicitly flush data to
> > > > >> disk -
> > > > >>>>>> RocksDB
> > > > >>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>> decide
> > > > >>>>>>>>>>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>>>>>>> when to flush memtables to disk itself.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd
> > > > >> have
> > > > >>>> two
> > > > >>>>>>>>>>>> options,
> > > > >>>>>>>>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>>>>>>>>>> both
> > > > >>>>>>>>>>>>>>>>>>>>>>> seem pretty bad to me:
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>          1. Have them record calls to
> commit(),
> > > > >>>> which
> > > > >>>>>> would
> > > > >>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>> misleading, as
> > > > >>>>>>>>>>>>>>>>>>>>>>>          data is no longer explicitly
> "flushed"
> > > > >> to
> > > > >>>> disk
> > > > >>>>>> by
> > > > >>>>>>>> this
> > > > >>>>>>>>>>>>>>>> call.
> > > > >>>>>>>>>>>>>>>>>>>>>>>          2. Have them record nothing at all,
> > > > >> which
> > > > >>>> is
> > > > >>>>>>>>>> equivalent
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> removing
> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>          metrics, except that users will see
> > the
> > > > >>>> metric
> > > > >>>>>>>> still
> > > > >>>>>>>>>>>>>>>> exists and
> > > > >>>>>>>>>>>>>>>>>>>> so
> > > > >>>>>>>>>>>>>>>>>>>>>> assume
> > > > >>>>>>>>>>>>>>>>>>>>>>>          that the metric is correct, and that
> > > > >>>> there's a
> > > > >>>>>>>> problem
> > > > >>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>> their
> > > > >>>>>>>>>>>>>>>>>>>>>> system
> > > > >>>>>>>>>>>>>>>>>>>>>>>          when there isn't.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree that removing them is also a bad
> > > > >> solution,
> > > > >>>> and
> > > > >>>>>> I'd
> > > > >>>>>>>>>>>>>>>> like some
> > > > >>>>>>>>>>>>>>>>>>>>>>> guidance on the best path forward here.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 5.
> > > > >>>>>>>>>>>>>>>>>>>>>>> Position files are updated on every write to
> a
> > > > >>>>>> StateStore.
> > > > >>>>>>>>>>>>>>>> Since our
> > > > >>>>>>>>>>>>>>>>>>>>>> writes
> > > > >>>>>>>>>>>>>>>>>>>>>>> are now buffered until commit(), we can't
> > > > >> update
> > > > >>>> the
> > > > >>>>>>>> Position
> > > > >>>>>>>>>>>>>>>> file
> > > > >>>>>>>>>>>>>>>>>>>> until
> > > > >>>>>>>>>>>>>>>>>>>>>>> commit() has been called, otherwise it would
> be
> > > > >>>>>>>> inconsistent
> > > > >>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> data
> > > > >>>>>>>>>>>>>>>>>>>>>>> in the event of a rollback. Consequently, we
> > > > >> need
> > > > >>>> to
> > > > >>>>>> manage
> > > > >>>>>>>>>>>>>>>> these
> > > > >>>>>>>>>>>>>>>>>>>> offsets
> > > > >>>>>>>>>>>>>>>>>>>>>>> the same way we manage the checkpoint
> offsets,
> > > > >> and
> > > > >>>>>> ensure
> > > > >>>>>>>>>>>>>>>> they're
> > > > >>>>>>>>>>>>>>>>>> only
> > > > >>>>>>>>>>>>>>>>>>>>>>> written on commit().
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 6.
> > > > >>>>>>>>>>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet
> what
> > > > >>>> tests to
> > > > >>>>>>>>>> write.
> > > > >>>>>>>>>>>>>>>> How
> > > > >>>>>>>>>>>>>>>>>>>>>> explicit
> > > > >>>>>>>>>>>>>>>>>>>>>>> do we need to be here in the KIP?
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed
> > > > >> to be
> > > > >>>>>>>> seamless,
> > > > >>>>>>>>>>>>>>>> and we
> > > > >>>>>>>>>>>>>>>>>>>>>> should
> > > > >>>>>>>>>>>>>>>>>>>>>>> definitely add some tests around that.
> > > > >> Downgrade,
> > > > >>>> it
> > > > >>>>>>>>>>>>>>>> transpires,
> > > > >>>>>>>>>>>>>>>>>> isn't
> > > > >>>>>>>>>>>>>>>>>>>>>>> currently possible, as the extra column
> family
> > > > >> for
> > > > >>>>>> offset
> > > > >>>>>>>>>>>>>>>> storage is
> > > > >>>>>>>>>>>>>>>>>>>>>>> incompatible with the pre-KIP-892
> > > > >> implementation:
> > > > >>>> when
> > > > >>>>>> you
> > > > >>>>>>>>>>>>>>>> open a
> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > > > >>>>>>>>>>>>>>>>>>>>>>> database, you must open all available column
> > > > >>>> families
> > > > >>>>>> or
> > > > >>>>>>>>>>>>>>>> receive an
> > > > >>>>>>>>>>>>>>>>>>>>>> error.
> > > > >>>>>>>>>>>>>>>>>>>>>>> What currently happens on downgrade is that
> it
> > > > >>>>>> attempts to
> > > > >>>>>>>>>>>>>>>> open the
> > > > >>>>>>>>>>>>>>>>>>>>>> store,
> > > > >>>>>>>>>>>>>>>>>>>>>>> throws an error about the offsets column
> > > > >> family not
> > > > >>>>>> being
> > > > >>>>>>>>>>>>>>>> opened,
> > > > >>>>>>>>>>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task.
> Given
> > > > >> that
> > > > >>>>>>>>>> downgrades
> > > > >>>>>>>>>>>>>>>>>> should
> > > > >>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>> uncommon, I think this is acceptable
> > > > >> behaviour, as
> > > > >>>> the
> > > > >>>>>>>>>>>>>>>> end-state is
> > > > >>>>>>>>>>>>>>>>>>>>>>> consistent, even if it results in an
> > > > >> undesirable
> > > > >>>> state
> > > > >>>>>>>>>>>> restore.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Should I document the upgrade/downgrade
> > > > >> behaviour
> > > > >>>>>>>> explicitly
> > > > >>>>>>>>>>>>>>>> in the
> > > > >>>>>>>>>>>>>>>>>>>> KIP?
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Nick!
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates!
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Since calls to flush() will not exist
> anymore
> > > > >>>> after
> > > > >>>>>> this
> > > > >>>>>>>> KIP
> > > > >>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>> released, I would rather throw an
> unsupported
> > > > >>>>>> operation
> > > > >>>>>>>>>>>>>>>> exception
> > > > >>>>>>>>>>>>>>>>>> by
> > > > >>>>>>>>>>>>>>>>>>>>>>>> default.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> When would a state store return -1 from
> > > > >>>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
> > > > >> while
> > > > >>>>>> being
> > > > >>>>>>>>>>>>>>>>>>>> transactional?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Wouldn't
> > > > >>>> StateStore#approximateNumUncommittedBytes()
> > > > >>>>>> also
> > > > >>>>>>>>>>>>>>>> return 0
> > > > >>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the state store is transactional but nothing
> > > > >> has
> > > > >>>> been
> > > > >>>>>>>>>> written
> > > > >>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> state store yet?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this
> > > > >> KIP
> > > > >>>> really
> > > > >>>>>>>> need
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> introduce
> > > > >>>>>>>>>>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> > > > >>>> StateStoreContext
> > > > >>>>>> has
> > > > >>>>>>>>>>>>>>>> already
> > > > >>>>>>>>>>>>>>>>>>>>>>>> appConfigs() which basically exposes the
> same
> > > > >>>>>> information,
> > > > >>>>>>>>>>>>>>>> i.e., if
> > > > >>>>>>>>>>>>>>>>>>>> EOS
> > > > >>>>>>>>>>>>>>>>>>>>>>>> is enabled or not.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore
> > > > >>>> interface
> > > > >>>>>> as
> > > > >>>>>>>>>>>>>>>> loosely
> > > > >>>>>>>>>>>>>>>>>>>> coupled
> > > > >>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > > > >>>>>> implementers
> > > > >>>>>>>>>>>> more
> > > > >>>>>>>>>>>>>>>>>>>> freedom,
> > > > >>>>>>>>>>>>>>>>>>>>>>>> and reduce the amount of internal knowledge
> > > > >>>> required."
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> While I understand the intent, I doubt that
> it
> > > > >>>>>> decreases
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> coupling of
> > > > >>>>>>>>>>>>>>>>>>>>>>>> a StateStore interface and the Streams
> engine.
> > > > >>>>>>>>>> READ_COMMITTED
> > > > >>>>>>>>>>>>>>>> only
> > > > >>>>>>>>>>>>>>>>>>>>>>>> applies to IQ but not to reads by
> processors.
> > > > >>>> Thus,
> > > > >>>>>>>>>>>>>>>> implementers
> > > > >>>>>>>>>>>>>>>>>>>> need to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand how Streams accesses the state
> > > > >> stores.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> I would like to hear what others think about
> > > > >> this.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> 4.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Great exposing new metrics for transactional
> > > > >> state
> > > > >>>>>> stores!
> > > > >>>>>>>>>>>>>>>>>> However, I
> > > > >>>>>>>>>>>>>>>>>>>>>>>> would prefer to add new metrics and
> deprecate
> > > > >> (in
> > > > >>>> the
> > > > >>>>>>>> docs)
> > > > >>>>>>>>>>>>>>>> the 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 also need to adapt/add
> > > > >> integration
> > > > >>>>>> tests
> > > > >>>>>>>>>>>>>>>> besides
> > > > >>>>>>>>>>>>>>>>>> unit
> > > > >>>>>>>>>>>>>>>>>>>>>>>> tests. Additionally, we probably need
> > > > >> integration
> > > > >>>> or
> > > > >>>>>>>> system
> > > > >>>>>>>>>>>>>>>> tests
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> verify that upgrades and downgrades between
> > > > >>>>>> transactional
> > > > >>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>> non-transactional state stores work as
> > > > >> expected.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion
> in
> > > > >> the
> > > > >>>> KIP,
> > > > >>>>>>>> under
> > > > >>>>>>>>>>>>>>>>>>>> "Rejected
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea
> > > > >> worth
> > > > >>>>>> pursuing,
> > > > >>>>>>>>>>>>>>>> but I
> > > > >>>>>>>>>>>>>>>>>>>> believe
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> that it's out of the scope of this KIP,
> > > > >> because
> > > > >>>> it
> > > > >>>>>>>> solves a
> > > > >>>>>>>>>>>>>>>>>>>> different
> > > > >>>>>>>>>>>>>>>>>>>>>> set
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of
> > > > >> this
> > > > >>>> one
> > > > >>>>>> has
> > > > >>>>>>>>>>>>>>>> already
> > > > >>>>>>>>>>>>>>>>>> grown
> > > > >>>>>>>>>>>>>>>>>>>>>>>> quite
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> large!
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford
> <
> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP (
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>
> > > > >>>>
> > > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > >>>>>>>>>>>>>>>>>>>>>>>> )
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing
> > > > >> back
> > > > >>>>>> "Atomic
> > > > >>>>>>>>>>>>>>>>>>>> Checkpointing"
> > > > >>>>>>>>>>>>>>>>>>>>>>>> (for
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think
> > > > >> the one
> > > > >>>>>> thing
> > > > >>>>>>>>>>>>>>>> missing is
> > > > >>>>>>>>>>>>>>>>>>>> some
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> changes to metrics (notably the store
> > > > >> "flush"
> > > > >>>>>> metrics
> > > > >>>>>>>> will
> > > > >>>>>>>>>>>>>>>> need
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> renamed to "commit").
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> The reason I brought back Atomic
> > > > >> Checkpointing
> > > > >>>> was
> > > > >>>>>> to
> > > > >>>>>>>>>>>>>>>> decouple
> > > > >>>>>>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>>>>>> flush
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> from store commit. This is important,
> > > > >> because
> > > > >>>> with
> > > > >>>>>>>>>>>>>>>> Transactional
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush"
> on
> > > > >>>> *every*
> > > > >>>>>> Task
> > > > >>>>>>>>>>>>>>>> commit,
> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> just when the StateStore is closing,
> > > > >> otherwise
> > > > >>>> our
> > > > >>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>> buffer
> > > > >>>>>>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> never be written and persisted, instead
> > > > >> growing
> > > > >>>>>>>> unbounded!
> > > > >>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>>>>>> experimented
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a
> > > > >> store
> > > > >>>>>> flush
> > > > >>>>>>>>>>>>>>>> whenever
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed
> its
> > > > >>>>>> configured
> > > > >>>>>>>>>>>>>>>> size, but
> > > > >>>>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> brittle: it prevented the transaction
> buffer
> > > > >>>> from
> > > > >>>>>> being
> > > > >>>>>>>>>>>>>>>>>> configured
> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> unbounded, and it still would have
> required
> > > > >>>> explicit
> > > > >>>>>>>>>>>>>>>> flushes of
> > > > >>>>>>>>>>>>>>>>>>>>>> RocksDB,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> yielding sub-optimal performance and
> memory
> > > > >>>>>> utilization.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the
> > > > >> "right"
> > > > >>>> way
> > > > >>>>>> to
> > > > >>>>>>>>>>>>>>>> resolve
> > > > >>>>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog
> > > > >> offsets
> > > > >>>> that
> > > > >>>>>>>>>>>>>>>> correspond
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> most
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> recently written records are always
> > > > >> atomically
> > > > >>>>>> written
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> (by writing them to the same transaction
> > > > >>>> buffer),
> > > > >>>>>> we can
> > > > >>>>>>>>>>>>>>>> avoid
> > > > >>>>>>>>>>>>>>>>>>>>>> forcibly
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk,
> > > > >> letting
> > > > >>>>>> RocksDB
> > > > >>>>>>>>>>>>>>>> flush
> > > > >>>>>>>>>>>>>>>>>> them
> > > > >>>>>>>>>>>>>>>>>>>>>> only
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> when necessary, without losing any of our
> > > > >>>>>> consistency
> > > > >>>>>>>>>>>>>>>> guarantees.
> > > > >>>>>>>>>>>>>>>>>>>> See
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> updated KIP for more info.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I have fully implemented these changes,
> > > > >>>> although I'm
> > > > >>>>>>>> still
> > > > >>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>> entirely
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> happy with the implementation for
> segmented
> > > > >>>>>> StateStores,
> > > > >>>>>>>>>> so
> > > > >>>>>>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>> plan
> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests
> > > > >> pass. If
> > > > >>>>>> you'd
> > > > >>>>>>>> like
> > > > >>>>>>>>>>>>>>>> to try
> > > > >>>>>>>>>>>>>>>>>>>> out
> > > > >>>>>>>>>>>>>>>>>>>>>> or
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> review this highly experimental and
> > > > >> incomplete
> > > > >>>>>> branch,
> > > > >>>>>>>>>> it's
> > > > >>>>>>>>>>>>>>>>>>>> available
> > > > >>>>>>>>>>>>>>>>>>>>>>>> here:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > > > >>>>>>>> .
> > > > >>>>>>>>>>>>>>>> Note:
> > > > >>>>>>>>>>>>>>>>>>>> it's
> > > > >>>>>>>>>>>>>>>>>>>>>>>> built
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable
> > > > >> base
> > > > >>>> to
> > > > >>>>>> build
> > > > >>>>>>>>>>>>>>>> and test
> > > > >>>>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>> on,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples
> > > > >> comparisons
> > > > >>>> in a
> > > > >>>>>>>> live
> > > > >>>>>>>>>>>>>>>>>>>>>> environment. I
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's
> > > > >> nearer
> > > > >>>>>>>>>> completion
> > > > >>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>> has
> > > > >>>>>>>>>>>>>>>>>>>>>> been
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> proven on our main application.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I would really appreciate help in
> reviewing
> > > > >> and
> > > > >>>>>> testing:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and
> Window)
> > > > >>>> stores
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Global stores
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> As I do not currently use either of these,
> > > > >> so my
> > > > >>>>>> primary
> > > > >>>>>>>>>>>>>>>> test
> > > > >>>>>>>>>>>>>>>>>>>>>>>> environment
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> doesn't test these areas.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next
> > > > >> week
> > > > >>>> for
> > > > >>>>>> a few
> > > > >>>>>>>>>>>>>>>> weeks,
> > > > >>>>>>>>>>>>>>>>>> so
> > > > >>>>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> not have time to move this forward until
> > > > >> late
> > > > >>>>>> August.
> > > > >>>>>>>> That
> > > > >>>>>>>>>>>>>>>> said,
> > > > >>>>>>>>>>>>>>>>>>>> your
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I
> just
> > > > >>>> won't be
> > > > >>>>>>>> able
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> respond
> > > > >>>>>>>>>>>>>>>>>>>>>> as
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> quickly as usual.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford
> <
> > > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact
> > > > >> on IQ
> > > > >>>> is
> > > > >>>>>> not
> > > > >>>>>>>>>>>>>>>> something
> > > > >>>>>>>>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>>>> had
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> considered.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the state
> > > > >> store
> > > > >>>>>> from the
> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> > > > >> the
> > > > >>>>>>>> checkpoint
> > > > >>>>>>>>>>>>>>>> (thus,
> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> > > > >> data
> > > > >>>>>> and/or
> > > > >>>>>>>>>>>>>>>> number of
> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you
> > > > >> suggesting
> > > > >>>>>> that we
> > > > >>>>>>>>>>>>>>>> add an
> > > > >>>>>>>>>>>>>>>>>>>>>>>> additional
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> config for the max number of commit
> > > > >> intervals
> > > > >>>>>> between
> > > > >>>>>>>>>>>>>>>>>> checkpoints?
> > > > >>>>>>>>>>>>>>>>>>>>>> That
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when
> the
> > > > >>>>>> transaction
> > > > >>>>>>>>>>>>>>>> buffers
> > > > >>>>>>>>>>>>>>>>>> are
> > > > >>>>>>>>>>>>>>>>>>>>>>>> nearly
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of
> > > > >> commit
> > > > >>>>>>>> intervals
> > > > >>>>>>>>>>>>>>>> have
> > > > >>>>>>>>>>>>>>>>>>>>>> elapsed,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> whichever comes first?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although
> > > > >> this
> > > > >>>>>>>> re-ignites
> > > > >>>>>>>>>>>>>>>> an
> > > > >>>>>>>>>>>>>>>>>>>> earlier
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> debate about whether a config should be
> > > > >>>> measured in
> > > > >>>>>>>>>>>>>>>> "number of
> > > > >>>>>>>>>>>>>>>>>>>> commit
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute
> > > > >> time.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the
> > > > >> reason
> > > > >>>> I
> > > > >>>>>> was
> > > > >>>>>>>>>>>>>>>> pursuing
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Atomic
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable
> > > > >> flush
> > > > >>>> from
> > > > >>>>>>>>>>>>>>>> checkpointing,
> > > > >>>>>>>>>>>>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every
> > > > >> commit
> > > > >>>>>> without
> > > > >>>>>>>> any
> > > > >>>>>>>>>>>>>>>>>>>> performance
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is
> definitely
> > > > >> the
> > > > >>>>>> "best"
> > > > >>>>>>>>>>>>>>>> solution,
> > > > >>>>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm not
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back
> > > > >> into
> > > > >>>> this
> > > > >>>>>> KIP.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
> > > > >>>>>> transactional
> > > > >>>>>>>>>>>> logic
> > > > >>>>>>>>>>>>>>>>>>>> directly
> > > > >>>>>>>>>>>>>>>>>>>>>>>> into
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with
> > > > >> the
> > > > >>>>>>>>>>>>>>>>>>>>>> StateStore#newTransaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> method, and reduces the number of new
> > > > >> classes
> > > > >>>>>>>> introduced,
> > > > >>>>>>>>>>>>>>>>>>>>>> significantly
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and
> > > > >> the
> > > > >>>>>>>> complexity
> > > > >>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>> drastically
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic
> > > > >>>> Checkpoints
> > > > >>>>>>>> into
> > > > >>>>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>> KIP.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno
> Cadonna
> > > > >> <
> > > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very
> interesting!
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> As far as I understand, you want to
> > > > >> atomically
> > > > >>>>>> update
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the
> > > > >>>> memtable
> > > > >>>>>> of a
> > > > >>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write the checkpoint not after the
> commit
> > > > >> time
> > > > >>>>>> elapsed
> > > > >>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>> after
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that
> > > > >> would
> > > > >>>> lead
> > > > >>>>>> to
> > > > >>>>>>>>>>>>>>>> exceeding
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > >>>> before the
> > > > >>>>>>>> next
> > > > >>>>>>>>>>>>>>>> commit
> > > > >>>>>>>>>>>>>>>>>>>>>>>> interval
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> ends.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would
> > > > >> commit
> > > > >>>>>> every
> > > > >>>>>>>>>>>>>>>> commit
> > > > >>>>>>>>>>>>>>>>>>>> interval
> > > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store will only be atomically
> > > > >>>> updated
> > > > >>>>>>>> roughly
> > > > >>>>>>>>>>>>>>>> every
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> of
> > > > >>>> data.
> > > > >>>>>> Also
> > > > >>>>>>>> IQ
> > > > >>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>> then
> > > > >>>>>>>>>>>>>>>>>>>>>>>> only
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see new data roughly every
> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> After a failure the state store needs to
> > > > >>>> restore
> > > > >>>>>> up to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Is this correct?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the state
> > > > >> store
> > > > >>>>>> from
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> > > > >> the
> > > > >>>>>>>> checkpoint
> > > > >>>>>>>>>>>>>>>> (thus,
> > > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> > > > >> data
> > > > >>>>>> and/or
> > > > >>>>>>>>>>>>>>>> number of
> > > > >>>>>>>>>>>>>>>>>>>>>> commit
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have
> > > > >> the
> > > > >>>> same
> > > > >>>>>> delay
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>>>>>>>>> records
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ
> because
> > > > >> both
> > > > >>>>>> would
> > > > >>>>>>>>>>>>>>>> appear
> > > > >>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However,
> > > > >>>> after a
> > > > >>>>>>>> failure
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it might restore data that is already in
> > > > >> the
> > > > >>>> state
> > > > >>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>> because
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable
> > > > >> offset
> > > > >>>>>> (i.e.
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>>> last
> > > > >>>>>>>>>>>>>>>>>>>>>> committed
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset) of the changelog topics.
> Restoring
> > > > >>>> data
> > > > >>>>>> that
> > > > >>>>>>>> is
> > > > >>>>>>>>>>>>>>>> already
> > > > >>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should
> > > > >> not
> > > > >>>>>> violated.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> This solution needs at least one new
> > > > >> config to
> > > > >>>>>> specify
> > > > >>>>>>>>>>>>>>>> when a
> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> should be written.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> A small correction to your previous
> e-mail
> > > > >>>> that
> > > > >>>>>> does
> > > > >>>>>>>> not
> > > > >>>>>>>>>>>>>>>> change
> > > > >>>>>>>>>>>>>>>>>>>>>>>> anything
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit
> > > > >>>> interval
> > > > >>>>>> is 30
> > > > >>>>>>>>>>>>>>>> seconds,
> > > > >>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>>> five
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> seconds.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've begun performance testing my
> branch
> > > > >> on
> > > > >>>> our
> > > > >>>>>>>> staging
> > > > >>>>>>>>>>>>>>>>>>>>>> environment,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
> > > > >>>> non-trivial
> > > > >>>>>>>>>>>>>>>> application.
> > > > >>>>>>>>>>>>>>>>>> I'm
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate
> > > > >> that
> > > > >>>> we
> > > > >>>>>> saw
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>>> last
> > > > >>>>>>>>>>>>>>>>>>>> time
> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP,
> > > > >> but
> > > > >>>> this
> > > > >>>>>>>> time,
> > > > >>>>>>>>>> I
> > > > >>>>>>>>>>>>>>>>>> think I
> > > > >>>>>>>>>>>>>>>>>>>>>> know
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> why.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit,
> > > > >> which is
> > > > >>>>>> called
> > > > >>>>>>>> at
> > > > >>>>>>>>>>>>>>>> the end
> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Task
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit process, has the following
> > > > >> behaviour:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under ALOS: checkpoint the
> > > > >> state
> > > > >>>>>> stores.
> > > > >>>>>>>>>> This
> > > > >>>>>>>>>>>>>>>>>> includes
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            flushing memtables in
> RocksDB.
> > > > >>>> This is
> > > > >>>>>>>>>>>> acceptable
> > > > >>>>>>>>>>>>>>>>>>>> because the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> default
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit.interval.ms is 5
> > > > >> seconds,
> > > > >>>> so
> > > > >>>>>>>> forcibly
> > > > >>>>>>>>>>>>>>>> flushing
> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> every 5
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            seconds is acceptable for
> most
> > > > >>>>>>>> applications.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under EOS: checkpointing
> is
> > > > >> not
> > > > >>>> done,
> > > > >>>>>>>>>> *unless*
> > > > >>>>>>>>>>>>>>>> it's
> > > > >>>>>>>>>>>>>>>>>>>> being
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> forced, due
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to e.g. the Task closing or
> > > > >> being
> > > > >>>>>> revoked.
> > > > >>>>>>>>>> This
> > > > >>>>>>>>>>>>>>>> means
> > > > >>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>> under
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> normal
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing conditions, the
> > > > >> state
> > > > >>>> stores
> > > > >>>>>>>> will
> > > > >>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>> checkpointed,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            not have memtables flushed
> at
> > > > >> all ,
> > > > >>>>>> unless
> > > > >>>>>>>>>>>> RocksDB
> > > > >>>>>>>>>>>>>>>>>>>> decides to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush them on
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            its own. Checkpointing
> stores
> > > > >> and
> > > > >>>>>>>>>> force-flushing
> > > > >>>>>>>>>>>>>>>> their
> > > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            done when a Task is being
> > > > >> closed.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint
> > > > >>>> stores on
> > > > >>>>>> at
> > > > >>>>>>>>>>>> least
> > > > >>>>>>>>>>>>>>>>>> *some*
> > > > >>>>>>>>>>>>>>>>>>>>>>>> normal
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the
> > > > >> RocksDB
> > > > >>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>> buffers
> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets
> > > > >> are
> > > > >>>>>> synced to
> > > > >>>>>>>>>>>>>>>> disk to
> > > > >>>>>>>>>>>>>>>>>>>>>> prevent
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
> > > > >>>> Consequently,
> > > > >>>>>> my
> > > > >>>>>>>>>>>>>>>> current
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task
> > > > >> commit,
> > > > >>>>>> which
> > > > >>>>>>>> is
> > > > >>>>>>>>>>>>>>>> far too
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> frequent.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000
> > > > >> records,
> > > > >>>>>> which
> > > > >>>>>>>> is
> > > > >>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>> change
> > > > >>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
> > > > >> performance
> > > > >>>>>> problems
> > > > >>>>>>>> for
> > > > >>>>>>>>>>>>>>>> some
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> applications.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions,
> and
> > > > >> I'm
> > > > >>>>>>>> currently
> > > > >>>>>>>>>>>>>>>> leaning
> > > > >>>>>>>>>>>>>>>>>>>>>>>> towards
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> using the
> > > > >>>> statestore.transaction.buffer.max.bytes
> > > > >>>>>>>>>>>>>>>>>> configuration
> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to
> > > > >>>> exceed it.
> > > > >>>>>>>> This
> > > > >>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> complement the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
> > > > >> functionality
> > > > >>>> that
> > > > >>>>>> this
> > > > >>>>>>>>>>>>>>>>>>>> configuration
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Currently, we use
> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > >>>>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> force an
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            early Task commit if
> > processing
> > > > >>>> more
> > > > >>>>>>>> records
> > > > >>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>> cause
> > > > >>>>>>>>>>>>>>>>>>>> our
> > > > >>>>>>>>>>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            transactions to exceed the
> > > > >> memory
> > > > >>>>>> assigned
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>>>>>> them.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - New functionality: when a
> > > > >> Task
> > > > >>>> *does*
> > > > >>>>>>>>>> commit,
> > > > >>>>>>>>>>>>>>>> we will
> > > > >>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the stores (and hence flush
> > the
> > > > >>>>>> transaction
> > > > >>>>>>>>>>>>>>>> buffers)
> > > > >>>>>>>>>>>>>>>>>>>> unless
> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> expect to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            cross the
> > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > >>>>>>>>>>>>>>>>>>>> threshold
> > > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the next
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick
> > > > >> Telford <
> > > > >>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I
> > > > >> meant in
> > > > >>>>>> terms of
> > > > >>>>>>>>>>>>>>>>>>>> understanding
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance profile under various
> > > > >>>>>> circumstances. The
> > > > >>>>>>>>>>>>>>>> more
> > > > >>>>>>>>>>>>>>>>>>>> complex
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would
> > > > >> be for
> > > > >>>>>> users
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> understand
> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance they see. For example,
> > > > >> spilling
> > > > >>>>>> records
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>>>>>> disk
> > > > >>>>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold
> > > > >>>> would, I
> > > > >>>>>>>>>> expect,
> > > > >>>>>>>>>>>>>>>>>> reduce
> > > > >>>>>>>>>>>>>>>>>>>>>> write
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
> > > > >>>> throughput
> > > > >>>>>> could
> > > > >>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>> unexpected,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> > > > >>>> diagnose/understand for
> > > > >>>>>>>>>> users.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early
> > > > >> commit"
> > > > >>>>>> concept is
> > > > >>>>>>>>>>>>>>>>>> relatively
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to
> document,
> > > > >> and
> > > > >>>>>>>>>>>> conceptually
> > > > >>>>>>>>>>>>>>>>>> fairly
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> obvious to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric
> to
> > > > >>>> make it
> > > > >>>>>>>>>> easier
> > > > >>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> understand
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it happens though.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an
> > > > >>>> indirect
> > > > >>>>>> way
> > > > >>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>> telling
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
> > > > >>>>>> READ_COMMITTED
> > > > >>>>>>>>>>>>>>>>>>>> essentially
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> requires
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that
> > > > >> two
> > > > >>>>>> threads
> > > > >>>>>>>>>>>>>>>> calling
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see
> writes
> > > > >>>> from
> > > > >>>>>> the
> > > > >>>>>>>>>> other
> > > > >>>>>>>>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> until
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> > > > >>>>>> READ_UNCOMMITTED, all
> > > > >>>>>>>>>>>>>>>> bets are
> > > > >>>>>>>>>>>>>>>>>>>> off,
> > > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe
> > > > >> written
> > > > >>>>>> records
> > > > >>>>>>>> at
> > > > >>>>>>>>>>>>>>>> any
> > > > >>>>>>>>>>>>>>>>>>>> time,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> which is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That
> > > > >> said,
> > > > >>>>>>>> StateStores
> > > > >>>>>>>>>>>>>>>> are
> > > > >>>>>>>>>>>>>>>>>> free
> > > > >>>>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implement
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these guarantees however they can,
> > > > >> which is
> > > > >>>> a
> > > > >>>>>> bit
> > > > >>>>>>>> more
> > > > >>>>>>>>>>>>>>>>>> relaxed
> > > > >>>>>>>>>>>>>>>>>>>>>> than
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions".
> > > > >> For
> > > > >>>>>> example,
> > > > >>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > > > >>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED ==
> > > > >>>> WBWI-based
> > > > >>>>>>>>>>>>>>>>>> "transactions",
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to
> the
> > > > >>>>>> database.
> > > > >>>>>>>> But
> > > > >>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>> other
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> storage
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to
> > > > >> *always*
> > > > >>>> use
> > > > >>>>>>>>>>>>>>>> transactions,
> > > > >>>>>>>>>>>>>>>>>>>> even
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage
> > > > >> engines
> > > > >>>>>> that
> > > > >>>>>>>>>> don't
> > > > >>>>>>>>>>>>>>>>>> provide
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation
> > > > >> guarantees
> > > > >>>> can
> > > > >>>>>> be
> > > > >>>>>>>> met
> > > > >>>>>>>>>>>>>>>> using a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> different
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> technique.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the
> > > > >> StateStore
> > > > >>>>>> interface
> > > > >>>>>>>> as
> > > > >>>>>>>>>>>>>>>>>> loosely
> > > > >>>>>>>>>>>>>>>>>>>>>>>> coupled
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible,
> to
> > > > >> give
> > > > >>>>>>>>>>>>>>>> implementers
> > > > >>>>>>>>>>>>>>>>>> more
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> freedom, and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal
> knowledge
> > > > >>>>>> required.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> > > > >>>> "IsolationLevel"
> > > > >>>>>> might
> > > > >>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> right
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it
> > > > >> much
> > > > >>>> more
> > > > >>>>>>>>>>>>>>>> explicit if
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> required, e.g.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7-8.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I can make these changes either later
> > > > >> today
> > > > >>>> or
> > > > >>>>>>>>>>>> tomorrow.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Small update:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and
> > > > >> fixed a
> > > > >>>>>> bunch of
> > > > >>>>>>>>>>>>>>>> issues
> > > > >>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests
> > > > >> pass,
> > > > >>>>>> which is
> > > > >>>>>>>>>>>>>>>>>> promising,
> > > > >>>>>>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to undergo some performance
> > > > >> testing. I
> > > > >>>>>> haven't
> > > > >>>>>>>>>>>>>>>> (yet)
> > > > >>>>>>>>>>>>>>>>>>>> worked
> > > > >>>>>>>>>>>>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff,
> > > > >> but I
> > > > >>>>>> would
> > > > >>>>>>>>>>>>>>>> expect
> > > > >>>>>>>>>>>>>>>>>> that,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
> > > > >>>> difference. The
> > > > >>>>>>>>>> branch
> > > > >>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>> available
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> at
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> > > > >>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>> anyone
> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno
> > > > >> Cadonna
> > > > >>>> <
> > > > >>>>>>>>>>>>>>>>>>>> cadonna@apache.org>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was
> > > > >> actually
> > > > >>>> also
> > > > >>>>>> my
> > > > >>>>>>>>>>>>>>>> point. I
> > > > >>>>>>>>>>>>>>>>>>>>>>>> understood
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion
> > > > >> path
> > > > >>>> as
> > > > >>>>>> a way
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>> avoid
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> early
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted
> the
> > > > >>>> intent.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually
> > > > >> it is
> > > > >>>>>> public
> > > > >>>>>>>>>>>>>>>> API. My
> > > > >>>>>>>>>>>>>>>>>>>>>> question
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal
> > > > >>>>>> processing.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the
> > > > >> size
> > > > >>>> of the
> > > > >>>>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>>>> buffer
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit
> > > > >> affects
> > > > >>>> the
> > > > >>>>>> whole
> > > > >>>>>>>>>>>>>>>>>> processing
> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a
> > > > >> state
> > > > >>>> store
> > > > >>>>>> is
> > > > >>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>> confined to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over
> and
> > > > >>>>>> changes the
> > > > >>>>>>>>>>>>>>>> behavior
> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you
> > > > >> that
> > > > >>>> it
> > > > >>>>>> is a
> > > > >>>>>>>>>>>>>>>> decent
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compromise. I
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides
> > > > >> and
> > > > >>>> list
> > > > >>>>>> the
> > > > >>>>>>>>>>>>>>>> options to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> overcome
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all
> > > > >>>> options
> > > > >>>>>> seem
> > > > >>>>>>>>>>>> quite
> > > > >>>>>>>>>>>>>>>>>> heavy
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compared
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand
> > > > >> what you
> > > > >>>>>> mean
> > > > >>>>>>>> with
> > > > >>>>>>>>>>>>>>>> "less
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> predictable
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
> > > > >>>> alternatives
> > > > >>>>>>>> really
> > > > >>>>>>>>>>>>>>>>>>>>>> interesting.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> But I
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good
> > > > >> and
> > > > >>>> we
> > > > >>>>>> should
> > > > >>>>>>>>>>>>>>>> continue
> > > > >>>>>>>>>>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>>>>>>>>>> it!
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my
> > > > >> e-mail on
> > > > >>>>>> June
> > > > >>>>>>>>>> 20th:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning
> > > > >> behind
> > > > >>>>>> putting
> > > > >>>>>>>>>>>>>>>> isolation
> > > > >>>>>>>>>>>>>>>>>>>>>> level
> > > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks!
> Should
> > > > >>>> that
> > > > >>>>>> also
> > > > >>>>>>>> be
> > > > >>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>> way
> > > > >>>>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>> give
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to
> > > > >> decide
> > > > >>>>>> whether
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>>>>>> turn on
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned
> > > > >> about
> > > > >>>>>> how do
> > > > >>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>> know
> > > > >>>>>>>>>>>>>>>>>>>> if a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written
> > > > >> under
> > > > >>>> EOS,
> > > > >>>>>> if
> > > > >>>>>>>> you
> > > > >>>>>>>>>>>>>>>> do not
> > > > >>>>>>>>>>>>>>>>>>>>>> have a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
> > > > >>>> transactional or
> > > > >>>>>> not.
> > > > >>>>>>>>>> If
> > > > >>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file
> can
> > > > >> be
> > > > >>>>>> written
> > > > >>>>>>>>>>>>>>>> during
> > > > >>>>>>>>>>>>>>>>>>>> normal
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state
> > > > >> store
> > > > >>>> is not
> > > > >>>>>>>>>>>>>>>>>> transactional,
> > > > >>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written
> > > > >> under
> > > > >>>> EOS.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My point was about not only
> > > > >> considering the
> > > > >>>>>> bytes
> > > > >>>>>>>> in
> > > > >>>>>>>>>>>>>>>> memory
> > > > >>>>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>>> config
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but
> > > > >> also
> > > > >>>>>> bytes
> > > > >>>>>>>> that
> > > > >>>>>>>>>>>>>>>> might
> > > > >>>>>>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> spilled
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering
> > > > >>>> whether you
> > > > >>>>>>>>>> should
> > > > >>>>>>>>>>>>>>>>>> remove
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory
> > > > >>>> bytes to
> > > > >>>>>> be
> > > > >>>>>>>>>> used
> > > > >>>>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store
> > > > >> records." My
> > > > >>>>>>>> thinking
> > > > >>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>> even
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes
> to
> > > > >>>> disk,
> > > > >>>>>>>>>> limiting
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> overall
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> bytes
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it
> > > > >> again
> > > > >>>> and
> > > > >>>>>>>>>>>>>>>> considering
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>> recent
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too
> much
> > > > >>>> sense
> > > > >>>>>>>> anymore.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I like the name
> > > > >>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes that
> > > > >>>>>>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 8.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> > > > >>>>>> implementation
> > > > >>>>>>>>>>>>>>>> details) of
> > > > >>>>>>>>>>>>>>>>>>>> how
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of
> > > > >> changelog
> > > > >>>>>>>>>>>>>>>> transactions,
> > > > >>>>>>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would
> be
> > > > >>>> great.
> > > > >>>>>>>> Would
> > > > >>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>> great
> > > > >>>>>>>>>>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about
> the
> > > > >>>>>> behavior in
> > > > >>>>>>>>>>>>>>>> case of
> > > > >>>>>>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> failure.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional
> > > > >> state
> > > > >>>>>> store
> > > > >>>>>>>>>>>>>>>> recover
> > > > >>>>>>>>>>>>>>>>>>>> after a
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
> > > > >>>> transaction
> > > > >>>>>>>> buffer,
> > > > >>>>>>>>>>>>>>>> etc.
> > > > >>>>>>>>>>>>>>>>>>>> (that
> > > > >>>>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> what
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford
> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue
> that
> > > > >>>>>>>>>>>>>>>> WriteBatchWithIndex
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely
> to
> > > > >>>> exceed)
> > > > >>>>>>>>>>>>>>>> configured
> > > > >>>>>>>>>>>>>>>>>>>> memory
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needs to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
> > > > >>>> Ultimately,
> > > > >>>>>> any
> > > > >>>>>>>>>>>>>>>> approach
> > > > >>>>>>>>>>>>>>>>>>>> based
> > > > >>>>>>>>>>>>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cracking
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it
> in
> > > > >>>> ways
> > > > >>>>>> it's
> > > > >>>>>>>> not
> > > > >>>>>>>>>>>>>>>> really
> > > > >>>>>>>>>>>>>>>>>>>>>>>> designed
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for is
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen
> > > > >> performance
> > > > >>>> or
> > > > >>>>>>>>>>>>>>>> consistency
> > > > >>>>>>>>>>>>>>>>>>>> issues.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing
> > > > >> these
> > > > >>>>>> early
> > > > >>>>>>>>>>>>>>>> commits?
> > > > >>>>>>>>>>>>>>>>>>>> While
> > > > >>>>>>>>>>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to
> > > > >>>> ensure
> > > > >>>>>>>>>>>>>>>> consistency
> > > > >>>>>>>>>>>>>>>>>>>> whilst
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maintaining
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem
> > > > >> *very*
> > > > >>>>>>>>>>>>>>>> complicated, and
> > > > >>>>>>>>>>>>>>>>>>>> might
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for
> > > > >> users
> > > > >>>> as a
> > > > >>>>>>>>>>>>>>>> consequence.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope
> of
> > > > >> this
> > > > >>>>>> KIP is
> > > > >>>>>>>>>>>>>>>> growing a
> > > > >>>>>>>>>>>>>>>>>>>> bit
> > > > >>>>>>>>>>>>>>>>>>>>>>>> out
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss
> > > > >> ideas
> > > > >>>> for
> > > > >>>>>>>> future
> > > > >>>>>>>>>>>>>>>>>>>>>>>> improvements, I
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > > >>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope
> > > > >> down
> > > > >>>> to a
> > > > >>>>>>>> design
> > > > >>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>> achieves
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> most
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
> > > > >>>>>> restorations
> > > > >>>>>>>>>>>>>>>> during
> > > > >>>>>>>>>>>>>>>>>> dirty
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design
> > > > >> that
> > > > >>>>>> this KIP
> > > > >>>>>>>>>>>>>>>> produces
> > > > >>>>>>>>>>>>>>>>>>>> can
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ultimately
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially
> > > > >> if
> > > > >>>> the
> > > > >>>>>> bulk
> > > > >>>>>>>> of
> > > > >>>>>>>>>>>>>>>> it is
> > > > >>>>>>>>>>>>>>>>>>>>>> internal
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next
> week
> > > > >>>> trying
> > > > >>>>>> to
> > > > >>>>>>>>>>>>>>>> re-work
> > > > >>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove
> > > > >> the
> > > > >>>>>>>>>>>>>>>> newTransaction()
> > > > >>>>>>>>>>>>>>>>>>>>>> method,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> such
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail
> of
> > > > >>>>>>>> RocksDBStore.
> > > > >>>>>>>>>>>>>>>> That
> > > > >>>>>>>>>>>>>>>>>>>> way, if
> > > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the
> > > > >> future,
> > > > >>>>>> like
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>>> SST
> > > > >>>>>>>>>>>>>>>>>> file
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> management
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so
> > > > >> with
> > > > >>>>>> little/no
> > > > >>>>>>>>>>>> API
> > > > >>>>>>>>>>>>>>>>>>>> changes.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>
> > > > >>>>>>
> > > > >>>>
> > > > >>
> > > > >
> > >
> >
>

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

Posted by Sophie Blee-Goldman <so...@responsive.dev>.
Hey Nick, sounds like things are moving along here. I think you're already
aware, but just as a reminder to us all, the KIP freeze deadline for 3.7 is
approaching and is currently set for Nov 18. I think we can all agree that
it would be great to have this KIP accepted by then, and it sounds like
much of the implementation is already completed, in which case this might
be able to make it into 3.7.

It sounds like this discussion is wrapping up so I just want to summarize
my understanding of the current plan regarding configs since it doesn't
seem like the KIP has been updated to include this yet.

Basically we're all in agreement to go with option #1 that you presented
earlier, right? Assuming so, I just want to clear up some details around
the default behavior. What happens if:
1. User sets EOS and sets READ_UNCOMMITTED: txn state stores will be
disabled/feature-flagged
2. User sets EOS and does not set anything for the isolation level:
although the default is READ_UNCOMMITTED, enabling EOS will change the
default to READ_COMMITTED and txn state stores will be used

As for the future, it sounds like when READ_UNCOMMITTED mode is
implemented, we will basically just remove this "feature flag" and txn
state stores will always be used for all EOS cases. EOS users will be able
to configure the isolation level independently, although it will still
default to READ_COMMITTED when EOS is enabled and it wasn't explicitly
configured.

Is this understanding correct? I think this was the right choice
regardless, as it will give people a way to disable the txn stores in an
emergency -- as a project we went a long time with little pressure to
feature flag things, and our users paid the cost for that. Even if we
managed to ship something without bugs, it was often only after an intense
period of successive blocker bugs that delayed the entire released for
weeks. Other times, major bugs slipped through and some versions became
unusable for certain use cases. So having some way to disable the txn state
stores when EOS is used feels like a good strategy, since you just never
know what might get through all the testing we do.

If this ends up slipping to 4.0 and you manage to implement the
READ_UNCOMMITTED mode within the same release, I think it's worth
considering to add in an additional feature flag, even if it's just a
backdoor internal config (eg as we did in KIP-441 with the internal task
assignor config). But for now let's assume the first release this KIP
appears in will have the behavior as described above, with READ_UNCOMMITTED
mode acting as a feature flag

Nick -- please let us know when you've updated the KIP to clarify the
config behavior, and are ready for a vote!


On Sun, Oct 29, 2023 at 12:02 PM Colt McNealy <co...@littlehorse.io> wrote:

> Guozhang—I agree, I am in favor of moving forward with the KIP now that the
> Transactional State Stores will be behind a feature flag.
>
> Nick—I just did a bit more light testing of your branch `KIP-892-3.5.0`
> with your most recent changes. I couldn't detect a performance difference
> versus trunk (in the past there was a slight degradation of performance on
> the restoration path, but that has been fixed). I don't believe that your
> branch has the state updater thread enabled, so I didn't test that path too
> heavily.
>
> As expected, however, our internal correctness tests failed due to the IQ
> read-your-own-writes issue we discussed previously. The community as a
> whole would vastly benefit from this KIP getting over the finish line in
> 3.7.0, and so long as it is behind a feature flag so that we at LittleHorse
> can still guarantee RYOW for our users, I think it's purely a win 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 <guozhang.wang.us@gmail.com
> >
> wrote:
>
> > I'd agree with you guys that as long as we are in agreement about the
> > configuration semantics, that would be a big win to move forward for
> > this KIP. As for the TaskCorruptedException handling like wiping state
> > stores, we can discuss that in the PR rather than in the KIP.
> >
> > Just to clarify, I'm onboard with the latest proposal, and probably we
> > can move on for voting on this KIP now?
> >
> > Guozhang
> >
> > On Thu, Oct 19, 2023 at 5:33 AM Bruno Cadonna <ca...@apache.org>
> wrote:
> > >
> > > Hi Nick,
> > >
> > > What you and Lucas wrote about the different configurations of ALOS/EOS
> > > and READ_COMMITTED/READ_UNCOMMITTED make sense to me. My earlier
> > > concerns about changelogs diverging from the content of the local state
> > > stores turned out to not apply. So I think, we can move on with those
> > > configurations.
> > >
> > > Regarding the TaskCorruptedException and wiping out the state stores
> > > under EOS, couldn't we abort the transaction on the state store and
> > > close the task dirty? If the Kafka transaction was indeed committed,
> the
> > > store would restore the missing part from the changelog topic. If the
> > > Kafka transaction was not committed, changelog topic and state store
> are
> > > in-sync.
> > >
> > > In any case, IMO those are implementation details that we do not need
> to
> > > discuss and solve in the KIP discussion. We can solve them on the PR.
> > > The important thing is that the processing guarantees hold.
> > >
> > > Best,
> > > Bruno
> > >
> > > On 10/18/23 3:56 PM, Nick Telford wrote:
> > > > Hi Lucas,
> > > >
> > > > TaskCorruptedException is how Streams signals that the Task state
> > needs to
> > > > be wiped, so we can't retain that exception without also wiping state
> > on
> > > > timeouts.
> > > >
> > > > Regards,
> > > > Nick
> > > >
> > > > On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <lbrutschy@confluent.io
> > .invalid>
> > > > wrote:
> > > >
> > > >> Hi Nick,
> > > >>
> > > >> I think indeed the better behavior would be to retry
> commitTransaction
> > > >> until we risk running out of time to meet `max.poll.interval.ms`.
> > > >>
> > > >> However, if it's handled as a `TaskCorruptedException` at the
> moment,
> > > >> I would do the same in this KIP, and leave exception handling
> > > >> improvements to future work. 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 <
> nick.telford@gmail.com>
> > > >> wrote:
> > > >>>
> > > >>> Hi Lucas,
> > > >>>
> > > >>> Yeah, this is pretty much the direction I'm thinking of going in
> > now. You
> > > >>> make an interesting point about committing on-error under
> > > >>> ALOS/READ_COMMITTED, although I haven't had a chance to think
> > through the
> > > >>> implications yet.
> > > >>>
> > > >>> Something that I ran into earlier this week is an issue with the
> new
> > > >>> handling of TimeoutException. Without TX stores, TimeoutException
> > under
> > > >> EOS
> > > >>> throws a TaskCorruptedException, which wipes the stores. However,
> > with TX
> > > >>> stores, TimeoutException is now just bubbled up and dealt with as
> it
> > is
> > > >>> under ALOS. The problem arises when the Producer#commitTransaction
> > call
> > > >>> times out: Streams attempts to ignore the error and continue
> > producing,
> > > >>> which causes the next call to Producer#send to throw
> > > >>> "IllegalStateException: Cannot attempt operation `send` because the
> > > >>> previous call to `commitTransaction` timed out and must be
> retried".
> > > >>>
> > > >>> I'm not sure what we should do here: retrying the commitTransaction
> > seems
> > > >>> logical, but what if it times out again? Where do we draw the line
> > and
> > > >>> shutdown the instance?
> > > >>>
> > > >>> Regards,
> > > >>> Nick
> > > >>>
> > > >>> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <
> lbrutschy@confluent.io
> > > >> .invalid>
> > > >>> wrote:
> > > >>>
> > > >>>> Hi all,
> > > >>>>
> > > >>>> I think I liked your suggestion of allowing EOS with
> > READ_UNCOMMITTED,
> > > >>>> but keep wiping the state on error, and I'd vote for this solution
> > > >>>> when introducing `default.state.isolation.level`. This way, we'd
> > have
> > > >>>> the most low-risk roll-out of this feature (no behavior change
> > without
> > > >>>> reconfiguration), with the possibility of switching to the most
> > sane /
> > > >>>> battle-tested default settings in 4.0. Essentially, we'd have a
> > > >>>> feature flag but call it `default.state.isolation.level` and don't
> > > >>>> have to deprecate it later.
> > > >>>>
> > > >>>> So the possible configurations would then be this:
> > > >>>>
> > > >>>> 1. ALOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB,
> > IQ
> > > >>>> reads from DB.
> > > >>>> 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> > > >>>> WriteBatch/DB. Flush on error (see note below).
> > > >>>> 3. EOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB,
> IQ
> > > >>>> reads from DB. Wipe state on error.
> > > >>>> 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> > > >>>> WriteBatch/DB.
> > > >>>>
> > > >>>> I believe the feature is important enough that we will see good
> > > >>>> adoption even without changing the default. In 4.0, when we have
> > seen
> > > >>>> this being adopted and is battle-tested, we make READ_COMMITTED
> the
> > > >>>> default for EOS, or even READ_COMITTED always the default,
> depending
> > > >>>> on our experiences. And we could add a clever implementation of
> > > >>>> READ_UNCOMITTED with WriteBatches later.
> > > >>>>
> > > >>>> The only smell here is that `default.state.isolation.level`
> wouldn't
> > > >>>> be purely an IQ setting, but it would also (slightly) change the
> > > >>>> behavior of the processing, but that seems unavoidable as long as
> we
> > > >>>> haven't solve READ_UNCOMITTED IQ with WriteBatches.
> > > >>>>
> > > >>>> Minor: As for Bruno's point 4, I think if we are concerned about
> > this
> > > >>>> 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).
> > > >>>>
> > > >>>> Cheers,
> > > >>>> Lucas
> > > >>>>
> > > >>>> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <
> > nick.telford@gmail.com>
> > > >>>> wrote:
> > > >>>>>
> > > >>>>> Hi Guozhang,
> > > >>>>>
> > > >>>>> The KIP as it stands introduces a new configuration,
> > > >>>>> default.state.isolation.level, which is independent of
> > > >> processing.mode.
> > > >>>>> It's intended that this new configuration be used to configure a
> > > >> global
> > > >>>> IQ
> > > >>>>> isolation level in the short term, with a future KIP introducing
> > the
> > > >>>>> capability to change the isolation level on a per-query basis,
> > > >> falling
> > > >>>> back
> > > >>>>> to the "default" defined by this config. That's why I called it
> > > >>>> "default",
> > > >>>>> for future-proofing.
> > > >>>>>
> > > >>>>> However, it currently includes the caveat that READ_UNCOMMITTED
> is
> > > >> not
> > > >>>>> available under EOS. I think this is the coupling you are
> alluding
> > > >> to?
> > > >>>>>
> > > >>>>> This isn't intended to be a restriction of the API, but is
> > currently
> > > >> a
> > > >>>>> technical limitation. However, after discussing with some users
> > about
> > > >>>>> use-cases that would require READ_UNCOMMITTED under EOS, I'm
> > > >> inclined to
> > > >>>>> remove that clause and put in the necessary work to make that
> > > >> combination
> > > >>>>> possible now.
> > > >>>>>
> > > >>>>> I currently see two possible approaches:
> > > >>>>>
> > > >>>>>     1. Disable TX StateStores internally when the IsolationLevel
> is
> > > >>>>>     READ_UNCOMMITTED and the processing.mode is EOS. This is more
> > > >>>> difficult
> > > >>>>>     than it sounds, as there are many assumptions being made
> > > >> throughout
> > > >>>> the
> > > >>>>>     internals about the guarantees StateStores provide. It would
> > > >>>> definitely add
> > > >>>>>     a lot of extra "if (read_uncommitted && eos)" branches,
> > > >> complicating
> > > >>>>>     maintenance and testing.
> > > >>>>>     2. Invest the time *now* to make READ_UNCOMMITTED of EOS
> > > >> StateStores
> > > >>>>>     possible. I have some ideas on how this could be achieved,
> but
> > > >> they
> > > >>>> would
> > > >>>>>     need testing and could introduce some additional issues. The
> > > >> benefit
> > > >>>> of
> > > >>>>>     this approach is that it would make query-time
> IsolationLevels
> > > >> much
> > > >>>> simpler
> > > >>>>>     to implement in the future.
> > > >>>>>
> > > >>>>> Unfortunately, both will require considerable work that will
> > further
> > > >>>> delay
> > > >>>>> this KIP, which was the reason I placed the restriction in the
> KIP
> > > >> in the
> > > >>>>> first place.
> > > >>>>>
> > > >>>>> Regards,
> > > >>>>> Nick
> > > >>>>>
> > > >>>>> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
> > > >> guozhang.wang.us@gmail.com>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>>> Hello Nick,
> > > >>>>>>
> > > >>>>>> First of all, thanks a lot for the great effort you've put in
> > > >> driving
> > > >>>>>> this KIP! I really like it coming through finally, as many
> people
> > > >> in
> > > >>>>>> the community have raised this. At the same time I honestly
> feel a
> > > >> bit
> > > >>>>>> ashamed for not putting enough of my time supporting it and
> > > >> pushing it
> > > >>>>>> through the finish line (you raised this KIP almost a year ago).
> > > >>>>>>
> > > >>>>>> I briefly passed through the DISCUSS thread so far, not sure
> I've
> > > >> 100
> > > >>>>>> percent digested all the bullet points. But with the goal of
> > > >> trying to
> > > >>>>>> help take it through the finish line in mind, I'd want to throw
> > > >>>>>> thoughts on top of my head only on the point #4 above which I
> felt
> > > >> may
> > > >>>>>> be the main hurdle for the current KIP to drive to a consensus
> > now.
> > > >>>>>>
> > > >>>>>> The general question I asked myself is, whether we want to
> couple
> > > >> "IQ
> > > >>>>>> reading mode" with "processing mode". While technically I tend
> to
> > > >>>>>> agree with you that, it's feels like a bug if some single user
> > > >> chose
> > > >>>>>> "EOS" for processing mode while choosing "read uncommitted" for
> IQ
> > > >>>>>> reading mode, at the same time, I'm thinking if it's possible
> that
> > > >>>>>> there could be two different persons (or even two teams) that
> > > >> would be
> > > >>>>>> using the stream API to build the app, and the IQ API to query
> the
> > > >>>>>> running state of the app. I know this is less of a technical
> thing
> > > >> but
> > > >>>>>> rather a more design stuff, but if it could be ever the case,
> I'm
> > > >>>>>> wondering if the personale using the IQ API knows about the
> risks
> > > >> of
> > > >>>>>> using read uncommitted but still chose so for the favor of
> > > >>>>>> performance, no matter if the underlying stream processing mode
> > > >>>>>> configured by another personale is EOS or not. In that regard,
> I'm
> > > >>>>>> leaning towards a "leaving the door open, and close it later if
> we
> > > >>>>>> found it's a bad idea" aspect with a configuration 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 can still enforce
> > > >>>>>> coupling them.
> > > >>>>>>
> > > >>>>>> Again, just my 2c here. Thanks again for the great patience and
> > > >>>>>> diligence on this KIP.
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Guozhang
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
> > > >> nick.telford@gmail.com>
> > > >>>>>> wrote:
> > > >>>>>>>
> > > >>>>>>> Hi Bruno,
> > > >>>>>>>
> > > >>>>>>> 4.
> > > >>>>>>> I'll hold off on making that change until we have a consensus
> as
> > > >> to
> > > >>>> what
> > > >>>>>>> configuration to use to control all of this, as it'll be
> > > >> affected by
> > > >>>> the
> > > >>>>>>> decision on EOS isolation levels.
> > > >>>>>>>
> > > >>>>>>> 5.
> > > >>>>>>> Done. I've chosen "committedOffsets".
> > > >>>>>>>
> > > >>>>>>> Regards,
> > > >>>>>>> Nick
> > > >>>>>>>
> > > >>>>>>> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <
> cadonna@apache.org>
> > > >>>> wrote:
> > > >>>>>>>
> > > >>>>>>>> Hi Nick,
> > > >>>>>>>>
> > > >>>>>>>> 1.
> > > >>>>>>>> Yeah, you are probably right that it does not make too much
> > > >> sense.
> > > >>>>>>>> Thanks for the clarification!
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> 4.
> > > >>>>>>>> Yes, sorry for the back and forth, but I think for the sake of
> > > >> the
> > > >>>> KIP
> > > >>>>>>>> it is better to let the ALOS behavior as it is for now due to
> > > >> the
> > > >>>>>>>> possible issues you would run into. Maybe we can find a
> > > >> solution
> > > >>>> in the
> > > >>>>>>>> future. Now the question returns to whether we really need
> > > >>>>>>>> default.state.isolation.level. Maybe the config could be the
> > > >>>> feature
> > > >>>>>>>> flag Sophie requested.
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> 5.
> > > >>>>>>>> There 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 committedOffsetFor().
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> Best,
> > > >>>>>>>> Bruno
> > > >>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>> On 10/13/23 3:21 PM, Nick Telford wrote:
> > > >>>>>>>>> Hi Bruno,
> > > >>>>>>>>>
> > > >>>>>>>>> Thanks for getting back to me.
> > > >>>>>>>>>
> > > >>>>>>>>> 1.
> > > >>>>>>>>> I think this should be possible. Are you thinking of the
> > > >>>> situation
> > > >>>>>> where
> > > >>>>>>>> a
> > > >>>>>>>>> user may downgrade to a previous version of Kafka Streams? In
> > > >>>> that
> > > >>>>>> case,
> > > >>>>>>>>> sadly, the RocksDBStore would get wiped by the older version
> > > >> of
> > > >>>> Kafka
> > > >>>>>>>>> Streams anyway, because that version wouldn't understand the
> > > >>>> extra
> > > >>>>>> column
> > > >>>>>>>>> family (that holds offsets), so the missing Position file
> > > >> would
> > > >>>>>>>>> automatically get rebuilt when the store is rebuilt from the
> > > >>>>>> changelog.
> > > >>>>>>>>> Are there other situations than downgrade where a
> > > >> transactional
> > > >>>> store
> > > >>>>>>>> could
> > > >>>>>>>>> be replaced by a non-transactional one? I can't think of any.
> > > >>>>>>>>>
> > > >>>>>>>>> 2.
> > > >>>>>>>>> Ahh yes, the Test Plan - my Kryptonite! This section
> > > >> definitely
> > > >>>>>> needs to
> > > >>>>>>>> be
> > > >>>>>>>>> fleshed out. I'll work on that. How much detail do you need?
> > > >>>>>>>>>
> > > >>>>>>>>> 3.
> > > >>>>>>>>> See my previous email discussing this.
> > > >>>>>>>>>
> > > >>>>>>>>> 4.
> > > >>>>>>>>> Hmm, this is an interesting point. Are you suggesting that
> > > >> under
> > > >>>> ALOS
> > > >>>>>>>>> READ_COMMITTED should not be supported?
> > > >>>>>>>>>
> > > >>>>>>>>> Regards,
> > > >>>>>>>>> Nick
> > > >>>>>>>>>
> > > >>>>>>>>> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
> > > >> cadonna@apache.org>
> > > >>>>>> wrote:
> > > >>>>>>>>>
> > > >>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>
> > > >>>>>>>>>> I think the KIP is converging!
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> 1.
> > > >>>>>>>>>> I am wondering whether it makes sense to write the position
> > > >> file
> > > >>>>>> during
> > > >>>>>>>>>> close as we do for the checkpoint file, so that in case the
> > > >>>> state
> > > >>>>>> store
> > > >>>>>>>>>> is replaced with a non-transactional state store the
> > > >>>>>> non-transactional
> > > >>>>>>>>>> state store finds the position file. I think, this is not
> > > >>>> strictly
> > > >>>>>>>>>> needed, but would be a nice behavior instead of just
> > > >> deleting
> > > >>>> the
> > > >>>>>>>>>> position file.
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> 2.
> > > >>>>>>>>>> The test plan does not mention integration tests. Do you not
> > > >>>> need to
> > > >>>>>>>>>> extend existing ones and add new ones. Also for upgrading
> > > >> and
> > > >>>>>>>>>> downgrading you might need integration and/or system tests.
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> 3.
> > > >>>>>>>>>> I think Sophie made a point. Although, IQ reading from
> > > >>>> uncommitted
> > > >>>>>> data
> > > >>>>>>>>>> under EOS might be considered a bug by some people. Thus,
> > > >> your
> > > >>>> KIP
> > > >>>>>> would
> > > >>>>>>>>>> fix a bug rather than changing the intended behavior.
> > > >> However, I
> > > >>>>>> also
> > > >>>>>>>>>> see that a feature flag would help users that rely on this
> > > >> buggy
> > > >>>>>>>>>> behavior (at least until AK 4.0).
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> 4.
> > > >>>>>>>>>> This is related to the previous point. I assume that the
> > > >>>> difference
> > > >>>>>>>>>> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is
> > > >> that in
> > > >>>> the
> > > >>>>>>>>>> former you enable transactions on the state store and in the
> > > >>>> latter
> > > >>>>>> you
> > > >>>>>>>>>> disable them. If my assumption is correct, I think that is
> > > >> an
> > > >>>> issue.
> > > >>>>>>>>>> Let's assume under ALOS Streams fails over a couple of times
> > > >>>> more or
> > > >>>>>>>>>> less at the same step in processing after value 3 is added
> > > >> to an
> > > >>>>>>>>>> aggregation but the offset of the corresponding input record
> > > >>>> was not
> > > >>>>>>>>>> committed. Without transactions disabled, the aggregation
> > > >> value
> > > >>>>>> would
> > > >>>>>>>>>> increase by 3 for each failover. With transactions enabled,
> > > >>>> value 3
> > > >>>>>>>>>> would only be added to the aggregation once when the offset
> > > >> of
> > > >>>> the
> > > >>>>>> input
> > > >>>>>>>>>> record is committed and the transaction finally completes.
> > > >> So
> > > >>>> the
> > > >>>>>>>>>> content of the state store would change depending on the
> > > >>>>>> configuration
> > > >>>>>>>>>> for IQ. IMO, the content of the state store should be
> > > >>>> independent
> > > >>>>>> from
> > > >>>>>>>>>> IQ. Given this issue, I propose to not use transactions with
> > > >>>> ALOS at
> > > >>>>>>>>>> all. I was a big proponent of using transactions with ALOS,
> > > >> but
> > > >>>> I
> > > >>>>>>>>>> realized that transactions with ALOS is not as easy as
> > > >> enabling
> > > >>>>>>>>>> transactions on state stores. Another aspect that is
> > > >>>> problematic is
> > > >>>>>> that
> > > >>>>>>>>>> the changelog topic which actually replicates the state
> > > >> store
> > > >>>> is not
> > > >>>>>>>>>> transactional under ALOS. Thus, it might happen that the
> > > >> state
> > > >>>>>> store and
> > > >>>>>>>>>> the changelog differ in their content. All of this is maybe
> > > >>>> solvable
> > > >>>>>>>>>> somehow, but for the sake of this KIP, I would leave it for
> > > >> the
> > > >>>>>> future.
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> Best,
> > > >>>>>>>>>> Bruno
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > > >>>>>>>>>>> Hey Nick! First of all thanks for taking up this awesome
> > > >>>> feature,
> > > >>>>>> I'm
> > > >>>>>>>>>> sure
> > > >>>>>>>>>>> every single
> > > >>>>>>>>>>> Kafka Streams user and dev would agree that it is sorely
> > > >>>> needed.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> I've just been catching up on the KIP and surrounding
> > > >>>> discussion,
> > > >>>>>> so
> > > >>>>>>>>>> please
> > > >>>>>>>>>>> forgive me
> > > >>>>>>>>>>> for any misunderstandings or misinterpretations of the
> > > >> current
> > > >>>>>> plan and
> > > >>>>>>>>>>> don't hesitate to
> > > >>>>>>>>>>> correct me.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Before I jump in, I just want to say that having seen this
> > > >>>> drag on
> > > >>>>>> for
> > > >>>>>>>> so
> > > >>>>>>>>>>> long, my singular
> > > >>>>>>>>>>> goal in responding is to help this KIP past a perceived
> > > >>>> impasse so
> > > >>>>>> we
> > > >>>>>>>> can
> > > >>>>>>>>>>> finally move on
> > > >>>>>>>>>>> to voting and implementing it. Long discussions are to be
> > > >>>> expected
> > > >>>>>> for
> > > >>>>>>>>>>> major features like
> > > >>>>>>>>>>> this but it's completely on us as the Streams devs to make
> > > >> sure
> > > >>>>>> there
> > > >>>>>>>> is
> > > >>>>>>>>>> an
> > > >>>>>>>>>>> end in sight
> > > >>>>>>>>>>> for any ongoing discussion.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> With that said, it's my understanding that the KIP as
> > > >> currently
> > > >>>>>>>> proposed
> > > >>>>>>>>>> is
> > > >>>>>>>>>>> just not tenable
> > > >>>>>>>>>>> for Kafka Streams, and would prevent some EOS users from
> > > >>>> upgrading
> > > >>>>>> to
> > > >>>>>>>> the
> > > >>>>>>>>>>> version it
> > > >>>>>>>>>>> first appears in. Given that we can't predict or guarantee
> > > >>>> whether
> > > >>>>>> any
> > > >>>>>>>> of
> > > >>>>>>>>>>> the followup KIPs
> > > >>>>>>>>>>> would be completed in the same release cycle as this one,
> > > >> we
> > > >>>> need
> > > >>>>>> to
> > > >>>>>>>> make
> > > >>>>>>>>>>> sure that the
> > > >>>>>>>>>>> feature is either compatible with all current users or else
> > > >>>>>>>>>> feature-flagged
> > > >>>>>>>>>>> so that they may
> > > >>>>>>>>>>> opt in/out.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Therefore, IIUC we need to have either (or both) of these
> > > >> as
> > > >>>>>>>>>>> fully-implemented config options:
> > > >>>>>>>>>>> 1. default.state.isolation.level
> > > >>>>>>>>>>> 2. enable.transactional.state.stores
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> This way EOS users for whom read_committed semantics are
> > > >> not
> > > >>>>>> viable can
> > > >>>>>>>>>>> still upgrade,
> > > >>>>>>>>>>> and either use the isolation.level config to leverage the
> > > >> new
> > > >>>> txn
> > > >>>>>> state
> > > >>>>>>>>>>> stores without sacrificing
> > > >>>>>>>>>>> their application semantics, or else simply keep the
> > > >>>> transactional
> > > >>>>>>>> state
> > > >>>>>>>>>>> stores disabled until we
> > > >>>>>>>>>>> are able to fully implement the isolation level
> > > >> configuration
> > > >>>> at
> > > >>>>>> either
> > > >>>>>>>>>> an
> > > >>>>>>>>>>> application or query level.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Frankly you are the expert here and know much more about
> > > >> the
> > > >>>>>> tradeoffs
> > > >>>>>>>> in
> > > >>>>>>>>>>> both semantics and
> > > >>>>>>>>>>> effort level of implementing one of these configs vs the
> > > >>>> other. In
> > > >>>>>> my
> > > >>>>>>>>>>> opinion, either option would
> > > >>>>>>>>>>> be fine and I would leave the decision of which one to
> > > >> include
> > > >>>> in
> > > >>>>>> this
> > > >>>>>>>>>> KIP
> > > >>>>>>>>>>> completely up to you.
> > > >>>>>>>>>>> I just don't see a way for the KIP to proceed without some
> > > >>>>>> variation of
> > > >>>>>>>>>> the
> > > >>>>>>>>>>> above that would allow
> > > >>>>>>>>>>> EOS users to opt-out of read_committed.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> (If it's all the same to you, I would recommend 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 that gets released.
> > > >>>>>>>>>>> So personally, my choice would be to add the feature flag
> > > >> and
> > > >>>>>> leave it
> > > >>>>>>>>>> off
> > > >>>>>>>>>>> by default. If all 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
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > > >>>>>> nick.telford@gmail.com>
> > > >>>>>>>>>> wrote:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>> Hi Bruno,
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Agreed, I can live with that for now.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> In an effort to keep the scope of this KIP from
> > > >> expanding, I'm
> > > >>>>>> leaning
> > > >>>>>>>>>>>> towards just providing a configurable
> > > >>>>>> default.state.isolation.level
> > > >>>>>>>> and
> > > >>>>>>>>>>>> removing IsolationLevel from the StateStoreContext. This
> > > >>>> would be
> > > >>>>>>>>>>>> compatible with adding support for query-time
> > > >> IsolationLevels
> > > >>>> in
> > > >>>>>> the
> > > >>>>>>>>>>>> future, whilst providing a way for users to select an
> > > >>>> isolation
> > > >>>>>> level
> > > >>>>>>>>>> now.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> The big problem with this, however, is that if a user
> > > >> selects
> > > >>>>>>>>>>>> processing.mode
> > > >>>>>>>>>>>> = "exactly-once(-v2|-beta)", and
> > > >>>> default.state.isolation.level =
> > > >>>>>>>>>>>> "READ_UNCOMMITTED", we need to guarantee that the data
> > > >> isn't
> > > >>>>>> written
> > > >>>>>>>> to
> > > >>>>>>>>>>>> disk until commit() is called, but we also need to permit
> > > >> IQ
> > > >>>>>> threads
> > > >>>>>>>> to
> > > >>>>>>>>>>>> read from the ongoing transaction.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> A simple solution would be to (temporarily) forbid this
> > > >>>>>> combination of
> > > >>>>>>>>>>>> configuration, and have default.state.isolation.level
> > > >>>>>> automatically
> > > >>>>>>>>>> switch
> > > >>>>>>>>>>>> to READ_COMMITTED when processing.mode is anything other
> > > >> than
> > > >>>>>>>>>>>> at-least-once. Do you think this would be acceptable?
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> In a later KIP, we can add support for query-time
> > > >> isolation
> > > >>>>>> levels and
> > > >>>>>>>>>>>> solve this particular problem there, which would relax
> > > >> this
> > > >>>>>>>> restriction.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> > > >>>> cadonna@apache.org>
> > > >>>>>>>>>> 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,
> > > >>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > > >>>>>>>>>>>>>> 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 <
> > > >>>>>> nick.telford@gmail.com
> > > >>>>>>>>>
> > > >>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Hi everyone,
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> I agree that having IsolationLevel be determined at
> > > >>>> query-time
> > > >>>>>> is
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>> ideal design, but there are a few sticking points:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>> There needs to be some way to communicate the
> > > >>>> IsolationLevel
> > > >>>>>> down
> > > >>>>>>>> to
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>> RocksDBStore itself, so that the query can respect it.
> > > >>>> Since
> > > >>>>>> stores
> > > >>>>>>>>>>>> are
> > > >>>>>>>>>>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
> > > >>>>>> MeteredStore,
> > > >>>>>>>>>>>>> etc.),
> > > >>>>>>>>>>>>>>> we need some way to deliver that information to the
> > > >> bottom
> > > >>>>>> layer.
> > > >>>>>>>> For
> > > >>>>>>>>>>>>> IQv2,
> > > >>>>>>>>>>>>>>> we can use the existing State#query() method, but IQv1
> > > >> has
> > > >>>> no
> > > >>>>>> way
> > > >>>>>>>> to
> > > >>>>>>>>>>>> do
> > > >>>>>>>>>>>>>>> this.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> A simple approach, which would potentially open up
> > > >> other
> > > >>>>>> options,
> > > >>>>>>>>>>>> would
> > > >>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> > > >>>>>>>>>>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > > >>>>>>>> ReadOnlyKeyValueStore
> > > >>>>>>>>>>>>> (and
> > > >>>>>>>>>>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore,
> > > >>>> etc.).
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>> As mentioned above, RocksDB WriteBatches are not
> > > >>>> thread-safe,
> > > >>>>>> which
> > > >>>>>>>>>>>>> causes
> > > >>>>>>>>>>>>>>> a problem if we want to provide READ_UNCOMMITTED
> > > >>>> Iterators. I
> > > >>>>>> also
> > > >>>>>>>>>>>> had a
> > > >>>>>>>>>>>>>>> look at RocksDB Transactions[1], but they solve a very
> > > >>>>>> different
> > > >>>>>>>>>>>>> problem,
> > > >>>>>>>>>>>>>>> and have the same thread-safety issue.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> One possible approach that I mentioned is chaining
> > > >>>>>> WriteBatches:
> > > >>>>>>>>>> every
> > > >>>>>>>>>>>>>>> time a new Interactive Query is received (i.e.
> > > >>>> readOnlyView,
> > > >>>>>> see
> > > >>>>>>>>>>>> above,
> > > >>>>>>>>>>>>>>> is called) we "freeze" the existing WriteBatch, and
> > > >> start a
> > > >>>>>> new one
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>> writes. The Interactive Query queries the "chain" of
> > > >>>> previous
> > > >>>>>>>>>>>>> WriteBatches
> > > >>>>>>>>>>>>>>> + the underlying database; while the StreamThread
> > > >> starts
> > > >>>>>> writing to
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>> *new* WriteBatch. On-commit, the StreamThread would
> > > >> write
> > > >>>> *all*
> > > >>>>>>>>>>>>>>> WriteBatches in the chain to the database (that have
> > > >> not
> > > >>>> yet
> > > >>>>>> been
> > > >>>>>>>>>>>>> written).
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> WriteBatches would be closed/freed only when they have
> > > >> been
> > > >>>>>> both
> > > >>>>>>>>>>>>>>> committed, and all open Interactive Queries on them
> > > >> have
> > > >>>> been
> > > >>>>>>>> closed.
> > > >>>>>>>>>>>>> This
> > > >>>>>>>>>>>>>>> would require some reference counting.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Obviously a drawback of this approach is the potential
> > > >> for
> > > >>>>>>>> increased
> > > >>>>>>>>>>>>>>> memory usage: if an Interactive Query is long-lived,
> > > >> for
> > > >>>>>> example by
> > > >>>>>>>>>>>>> doing a
> > > >>>>>>>>>>>>>>> full scan over a large database, or even just pausing
> > > >> in
> > > >>>> the
> > > >>>>>> middle
> > > >>>>>>>>>> of
> > > >>>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>> iteration, then the existing chain of WriteBatches
> > > >> could be
> > > >>>>>> kept
> > > >>>>>>>>>>>> around
> > > >>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>> a long time, potentially forever.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> A.
> > > >>>>>>>>>>>>>>> Going off on a tangent, it looks like in addition to
> > > >>>> supporting
> > > >>>>>>>>>>>>>>> READ_COMMITTED queries, we could go further and support
> > > >>>>>>>>>>>> REPEATABLE_READ
> > > >>>>>>>>>>>>>>> queries (i.e. where subsequent reads to the same key
> > > >> in the
> > > >>>>>> same
> > > >>>>>>>>>>>>>>> Interactive Query are guaranteed to yield the same
> > > >> value)
> > > >>>> by
> > > >>>>>> making
> > > >>>>>>>>>>>> use
> > > >>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so
> > > >> the
> > > >>>>>>>>>> performance
> > > >>>>>>>>>>>>>>> impact is likely to be negligible, but they do require
> > > >>>> that the
> > > >>>>>>>>>>>>> Interactive
> > > >>>>>>>>>>>>>>> Query session can be explicitly closed.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> This could be achieved if we made the above
> > > >> readOnlyView
> > > >>>>>> interface
> > > >>>>>>>>>>>> look
> > > >>>>>>>>>>>>>>> more like:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> > > >>>>>>>>>>>> ReadOnlyKeyValueStore<K,
> > > >>>>>>>>>>>>>>> V>, AutoCloseable {}
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > > >>>>>>>>>>>>>>>         ...
> > > >>>>>>>>>>>>>>>         ReadOnlyKeyValueView<K, V>
> > > >>>> readOnlyView(IsolationLevel
> > > >>>>>>>>>>>>> isolationLevel);
> > > >>>>>>>>>>>>>>> }
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> But this would be a breaking change, as existing IQv1
> > > >>>> queries
> > > >>>>>> are
> > > >>>>>>>>>>>>>>> guaranteed to never call store.close(), and therefore
> > > >> these
> > > >>>>>> would
> > > >>>>>>>>>> leak
> > > >>>>>>>>>>>>>>> memory under REPEATABLE_READ.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> B.
> > > >>>>>>>>>>>>>>> One thing that's notable: MyRocks states that they
> > > >> support
> > > >>>>>>>>>>>>> READ_COMMITTED
> > > >>>>>>>>>>>>>>> and REPEATABLE_READ, but they make no mention of
> > > >>>>>>>>>>>> READ_UNCOMMITTED[3][4].
> > > >>>>>>>>>>>>>>> This could be because doing so is technically
> > > >>>>>> difficult/impossible
> > > >>>>>>>>>>>> using
> > > >>>>>>>>>>>>>>> the primitives available in RocksDB.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Lucas, to address your points:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> U1.
> > > >>>>>>>>>>>>>>> It's only "SHOULD" to permit alternative (i.e.
> > > >> non-RocksDB)
> > > >>>>>>>>>>>>>>> implementations of StateStore that do not support
> > > >> atomic
> > > >>>>>> writes.
> > > >>>>>>>>>>>>> Obviously
> > > >>>>>>>>>>>>>>> in those cases, the guarantees Kafka Streams
> > > >>>> provides/expects
> > > >>>>>> would
> > > >>>>>>>>>> be
> > > >>>>>>>>>>>>>>> relaxed. Do you think we should require all
> > > >>>> implementations to
> > > >>>>>>>>>> support
> > > >>>>>>>>>>>>>>> atomic writes?
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> U2.
> > > >>>>>>>>>>>>>>> Stores can support multiple IsolationLevels. As we've
> > > >>>> discussed
> > > >>>>>>>>>> above,
> > > >>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>> ideal scenario would be to specify the IsolationLevel
> > > >> at
> > > >>>>>>>> query-time.
> > > >>>>>>>>>>>>>>> Failing that, I think the second-best approach is to
> > > >>>> define the
> > > >>>>>>>>>>>>>>> IsolationLevel for *all* queries based on the
> > > >>>> processing.mode,
> > > >>>>>>>> which
> > > >>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>> what the default StateStoreContext#isolationLevel()
> > > >>>> achieves.
> > > >>>>>> Would
> > > >>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>> prefer an alternative?
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> While the existing implementation is equivalent to
> > > >>>>>>>> READ_UNCOMMITTED,
> > > >>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>> can yield unexpected results/errors under EOS, if a
> > > >>>>>> transaction is
> > > >>>>>>>>>>>>> rolled
> > > >>>>>>>>>>>>>>> back. While this would be a change in behaviour for
> > > >> users,
> > > >>>> it
> > > >>>>>> would
> > > >>>>>>>>>>>> look
> > > >>>>>>>>>>>>>>> more like a bug fix than a breaking change. That said,
> > > >> we
> > > >>>>>> *could*
> > > >>>>>>>>>> make
> > > >>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>> configurable, and default to the existing behaviour
> > > >>>>>>>>>> (READ_UNCOMMITTED)
> > > >>>>>>>>>>>>>>> instead of inferring it from the processing.mode?
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> N1, N2.
> > > >>>>>>>>>>>>>>> These were only primitives to avoid boxing costs, but
> > > >> since
> > > >>>>>> this is
> > > >>>>>>>>>>>> not
> > > >>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>> performance sensitive area, it should be fine to
> > > >> change if
> > > >>>>>> that's
> > > >>>>>>>>>>>>> desirable.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> N3.
> > > >>>>>>>>>>>>>>> It's because the store "manages its own offsets", which
> > > >>>>>> includes
> > > >>>>>>>> both
> > > >>>>>>>>>>>>>>> committing the offset, *and providing it* via
> > > >>>>>> getCommittedOffset().
> > > >>>>>>>>>>>>>>> Personally, I think "managesOffsets" conveys this best,
> > > >>>> but I
> > > >>>>>> don't
> > > >>>>>>>>>>>> mind
> > > >>>>>>>>>>>>>>> changing it if the nomenclature is unclear.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Sorry for the massive emails/essays!
> > > >>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> 1:
> > > >> https://github.com/facebook/rocksdb/wiki/Transactions
> > > >>>>>>>>>>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > > >>>>>>>>>>>>>>> 3:
> > > >>>>>>>>
> > > >> https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > >>>>>>>>>>>>>>> 4:
> > > >>>> https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > > >>>>>>>>>>>>>>> <lb...@confluent.io.invalid> wrote:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> since I last read it in April, the KIP has become much
> > > >>>>>> cleaner and
> > > >>>>>>>>>>>>>>>> easier to read. Great work!
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> It feels to me the last big open point is whether we
> > > >> can
> > > >>>>>> implement
> > > >>>>>>>>>>>>>>>> isolation level as a query parameter. I understand
> > > >> that
> > > >>>> there
> > > >>>>>> are
> > > >>>>>>>>>>>>>>>> implementation concerns, but as Colt says, it would
> > > >> be a
> > > >>>> great
> > > >>>>>>>>>>>>>>>> addition, and would also simplify the migration path
> > > >> for
> > > >>>> this
> > > >>>>>>>>>> change.
> > > >>>>>>>>>>>>>>>> Is the implementation problem you mentioned caused by
> > > >> the
> > > >>>>>>>> WriteBatch
> > > >>>>>>>>>>>>>>>> not having a notion of a snapshot, as the underlying
> > > >> DB
> > > >>>>>> iterator
> > > >>>>>>>>>>>> does?
> > > >>>>>>>>>>>>>>>> In that case, I am not sure a chain of WriteBatches
> > > >> as you
> > > >>>>>> propose
> > > >>>>>>>>>>>>>>>> would fully solve the problem, but maybe I didn't dig
> > > >>>> enough
> > > >>>>>> into
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> details to fully understand it.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> If it's not possible to implement it now, would it be
> > > >> an
> > > >>>>>> option to
> > > >>>>>>>>>>>>>>>> make sure in this KIP that we do not fully close the
> > > >> door
> > > >>>> on
> > > >>>>>>>>>>>> per-query
> > > >>>>>>>>>>>>>>>> isolation levels in the interface, as it may be
> > > >> possible
> > > >>>> to
> > > >>>>>>>>>> implement
> > > >>>>>>>>>>>>>>>> the missing primitives in RocksDB or Speedb in the
> > > >> future.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Understanding:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to
> > > >> be
> > > >>>>>> persisted
> > > >>>>>>>>>>>>>>>> atomically with the records?
> > > >>>>>>>>>>>>>>>> * U2) Don't understand the default implementation of
> > > >>>>>>>>>>>> `isolationLevel`.
> > > >>>>>>>>>>>>>>>> The isolation level should be a property of the
> > > >> underlying
> > > >>>>>> store,
> > > >>>>>>>>>> and
> > > >>>>>>>>>>>>>>>> not be defined by the default config? Existing stores
> > > >>>> probably
> > > >>>>>>>> don't
> > > >>>>>>>>>>>>>>>> guarantee READ_COMMITTED, so the default should be to
> > > >>>> return
> > > >>>>>>>>>>>>>>>> READ_UNCOMMITTED.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Nits:
> > > >>>>>>>>>>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong`
> > > >>>> return
> > > >>>>>> type,
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>>> avoid the `null`?
> > > >>>>>>>>>>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
> > > >>>>>> `OptionalLong`
> > > >>>>>>>>>>>>>>>> return type, to avoid the `-1`?
> > > >>>>>>>>>>>>>>>> * N3) I don't understand why `managesOffsets` uses the
> > > >>>>>> 'manage'
> > > >>>>>>>>>> verb,
> > > >>>>>>>>>>>>>>>> whereas all other methods use the "commits" verb. I'd
> > > >>>> suggest
> > > >>>>>>>>>>>>>>>> `commitsOffsets`.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Either way, it feels this KIP is very close to the
> > > >> finish
> > > >>>>>> line,
> > > >>>>>>>> I'm
> > > >>>>>>>>>>>>>>>> looking forward to seeing this in production!
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Cheers,
> > > >>>>>>>>>>>>>>>> Lucas
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> > > >>>>>> colt@littlehorse.io
> > > >>>>>>>>>
> > > >>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Making IsolationLevel a query-time constraint,
> > > >> rather
> > > >>>> than
> > > >>>>>>>> linking
> > > >>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>> the processing.guarantee.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> As I understand it, would this allow even a user of
> > > >> EOS
> > > >>>> to
> > > >>>>>>>> control
> > > >>>>>>>>>>>>>>>> whether
> > > >>>>>>>>>>>>>>>>> reading committed or uncommitted records? If so, I am
> > > >>>> highly
> > > >>>>>> in
> > > >>>>>>>>>>>> favor
> > > >>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>> this.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> I know that I was one of the early people to point
> > > >> out
> > > >>>> the
> > > >>>>>>>> current
> > > >>>>>>>>>>>>>>>>> shortcoming that IQ reads uncommitted records, but
> > > >> just
> > > >>>> this
> > > >>>>>>>>>>>> morning I
> > > >>>>>>>>>>>>>>>>> realized a pattern we use which means that (for
> > > >> certain
> > > >>>>>> queries)
> > > >>>>>>>>>> our
> > > >>>>>>>>>>>>>>>> system
> > > >>>>>>>>>>>>>>>>> needs to be able to read uncommitted records, which
> > > >> is
> > > >>>> the
> > > >>>>>>>> current
> > > >>>>>>>>>>>>>>>> behavior
> > > >>>>>>>>>>>>>>>>> of Kafka Streams in EOS.***
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> If IsolationLevel being a query-time decision allows
> > > >> for
> > > >>>>>> this,
> > > >>>>>>>> then
> > > >>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>> would be amazing. I would also vote that the default
> > > >>>> behavior
> > > >>>>>>>>>> should
> > > >>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>> reading uncommitted records, because it is totally
> > > >>>> possible
> > > >>>>>> for a
> > > >>>>>>>>>>>>> valid
> > > >>>>>>>>>>>>>>>>> application to depend on that behavior, and breaking
> > > >> it
> > > >>>> in a
> > > >>>>>>>> minor
> > > >>>>>>>>>>>>>>>> release
> > > >>>>>>>>>>>>>>>>> might be a bit strong.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> *** (Note, for the curious reader....) Our
> > > >> use-case/query
> > > >>>>>> pattern
> > > >>>>>>>>>>>> is a
> > > >>>>>>>>>>>>>>>> bit
> > > >>>>>>>>>>>>>>>>> complex, but reading "uncommitted" records is
> > > >> actually
> > > >>>> safe
> > > >>>>>> in
> > > >>>>>>>> our
> > > >>>>>>>>>>>>> case
> > > >>>>>>>>>>>>>>>>> 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 soon as the processor processes
> > > >> the
> > > >>>>>> result.
> > > >>>>>>>> If
> > > >>>>>>>>>>>>> they
> > > >>>>>>>>>>>>>>>>> can't read uncommitted records, they may receive a
> > > >> "201 -
> > > >>>>>>>> Created"
> > > >>>>>>>>>>>>>>>>> response, immediately followed by a "404 - 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 <
> > > >>>>>>>>>>>> nick.telford@gmail.com>
> > > >>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Addendum:
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> I think we would also face the same problem with the
> > > >>>>>> approach
> > > >>>>>>>> John
> > > >>>>>>>>>>>>>>>> outlined
> > > >>>>>>>>>>>>>>>>>> earlier (using the record cache as a transaction
> > > >> buffer
> > > >>>> and
> > > >>>>>>>>>>>> flushing
> > > >>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>> straight to SST files). This is because the record
> > > >> cache
> > > >>>>>> (the
> > > >>>>>>>>>>>>>>>> ThreadCache
> > > >>>>>>>>>>>>>>>>>> class) is not thread-safe, so every commit would
> > > >>>> invalidate
> > > >>>>>> open
> > > >>>>>>>>>> IQ
> > > >>>>>>>>>>>>>>>>>> Iterators in the same way that RocksDB WriteBatches
> > > >> do.
> > > >>>>>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > > >>>>>>>>>>>> nick.telford@gmail.com>
> > > >>>>>>>>>>>>>>>>>> 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.
> > > >>>>>>>>>>>>>>>>>>> 2. Making IsolationLevel a query-time constraint,
> > > >>>> rather
> > > >>>>>> than
> > > >>>>>>>>>>>>>>>> linking it
> > > >>>>>>>>>>>>>>>>>>> to the processing.guarantee.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> There's a wrinkle that makes this a challenge:
> > > >>>> Interactive
> > > >>>>>>>>>> Queries
> > > >>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>> open an Iterator, when using transactions and
> > > >>>>>> READ_UNCOMMITTED.
> > > >>>>>>>>>>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries
> > > >>>> need
> > > >>>>>> to be
> > > >>>>>>>>>>>> able
> > > >>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>> read records from the currently uncommitted
> > > >> transaction
> > > >>>>>> buffer
> > > >>>>>>>>>>>>>>>>>>> (WriteBatch). This includes for Iterators, which
> > > >> should
> > > >>>>>> iterate
> > > >>>>>>>>>>>>>>>> both the
> > > >>>>>>>>>>>>>>>>>>> transaction buffer and underlying database (using
> > > >>>>>>>>>>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> The issue is that when the StreamThread commits, it
> > > >>>> writes
> > > >>>>>> the
> > > >>>>>>>>>>>>>>>> current
> > > >>>>>>>>>>>>>>>>>>> WriteBatch to RocksDB *and then clears the
> > > >> WriteBatch*.
> > > >>>>>>>> Clearing
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>> WriteBatch while an Interactive Query holds an open
> > > >>>>>> Iterator on
> > > >>>>>>>>>> it
> > > >>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>> invalidate the Iterator. Worse, it turns out that
> > > >>>> Iterators
> > > >>>>>>>> over
> > > >>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>> WriteBatch become invalidated not just when the
> > > >>>> WriteBatch
> > > >>>>>> is
> > > >>>>>>>>>>>>>>>> cleared,
> > > >>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>> also when the Iterators' current key receives a new
> > > >>>> write.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Now that I'm writing this, I remember that this is
> > > >> the
> > > >>>>>> major
> > > >>>>>>>>>>>> reason
> > > >>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>> switched the original design from having a
> > > >> query-time
> > > >>>>>>>>>>>>>>>> IsolationLevel to
> > > >>>>>>>>>>>>>>>>>>> having the IsolationLevel linked to the
> > > >>>> transactionality
> > > >>>>>> of the
> > > >>>>>>>>>>>>>>>> stores
> > > >>>>>>>>>>>>>>>>>>> themselves.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> It *might* be possible to resolve this, by having a
> > > >>>>>> "chain" of
> > > >>>>>>>>>>>>>>>>>>> WriteBatches, with the StreamThread switching to a
> > > >> new
> > > >>>>>>>> WriteBatch
> > > >>>>>>>>>>>>>>>>>> whenever
> > > >>>>>>>>>>>>>>>>>>> a new Interactive Query attempts to read from the
> > > >>>>>> database, but
> > > >>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>> could
> > > >>>>>>>>>>>>>>>>>>> cause some performance problems/memory pressure
> > > >> when
> > > >>>>>> subjected
> > > >>>>>>>> to
> > > >>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>> high
> > > >>>>>>>>>>>>>>>>>>> Interactive Query load. It would also reduce the
> > > >>>>>> efficiency of
> > > >>>>>>>>>>>>>>>>>> WriteBatches
> > > >>>>>>>>>>>>>>>>>>> on-commit, as we'd have to write N WriteBatches,
> > > >> where
> > > >>>> N
> > > >>>>>> is the
> > > >>>>>>>>>>>>>>>> number of
> > > >>>>>>>>>>>>>>>>>>> Interactive Queries since the last commit.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> I realise this is getting into the weeds of the
> > > >>>>>> implementation,
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>> you'd
> > > >>>>>>>>>>>>>>>>>>> rather we focus on the API for now, but I think
> > > >> it's
> > > >>>>>> important
> > > >>>>>>>> to
> > > >>>>>>>>>>>>>>>>>> consider
> > > >>>>>>>>>>>>>>>>>>> how to implement the desired API, in case we come
> > > >> up
> > > >>>> with
> > > >>>>>> an
> > > >>>>>>>> API
> > > >>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>> cannot be implemented efficiently, or even at all!
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Thoughts?
> > > >>>>>>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> > > >>>>>>>> cadonna@apache.org
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> 6.
> > > >>>>>>>>>>>>>>>>>>>> Of course, you are right! My bad!
> > > >>>>>>>>>>>>>>>>>>>> Wiping out the state in the downgrading case is
> > > >> fine.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> 3a.
> > > >>>>>>>>>>>>>>>>>>>> Focus on the public facing changes for the KIP. We
> > > >>>> will
> > > >>>>>> manage
> > > >>>>>>>>>> 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.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > > >>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Thanks for getting back to me!
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>>>>>>>> The fact that implementations can always track
> > > >>>> estimated
> > > >>>>>>>> memory
> > > >>>>>>>>>>>>>>>> usage
> > > >>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as
> > > >> an
> > > >>>>>> option,
> > > >>>>>>>> and
> > > >>>>>>>>>>>>>>>> I'll
> > > >>>>>>>>>>>>>>>>>>>> clarify
> > > >>>>>>>>>>>>>>>>>>>>> the JavaDoc that 0 is not just for
> > > >> non-transactional
> > > >>>>>> stores,
> > > >>>>>>>>>>>>>>>> which is
> > > >>>>>>>>>>>>>>>>>>>>> currently misleading.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> 6.
> > > >>>>>>>>>>>>>>>>>>>>> The problem with catching the exception in the
> > > >>>> downgrade
> > > >>>>>>>>>> process
> > > >>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>> would require new code in the Kafka version being
> > > >>>>>> downgraded
> > > >>>>>>>>>> to.
> > > >>>>>>>>>>>>>>>> Since
> > > >>>>>>>>>>>>>>>>>>>>> users could conceivably downgrade to almost *any*
> > > >>>> older
> > > >>>>>>>> version
> > > >>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>> Kafka
> > > >>>>>>>>>>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
> > > >>>>>>>>>>>>>>>>>>>>> The only way I can think of doing it would be to
> > > >>>> provide
> > > >>>>>> a
> > > >>>>>>>>>>>>>>>> dedicated
> > > >>>>>>>>>>>>>>>>>>>>> downgrade tool, that goes through every local
> > > >> store
> > > >>>> and
> > > >>>>>>>> removes
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> offsets column families. But that seems like an
> > > >>>>>> unnecessary
> > > >>>>>>>>>>>>>>>> amount of
> > > >>>>>>>>>>>>>>>>>>>> extra
> > > >>>>>>>>>>>>>>>>>>>>> code to maintain just to handle a somewhat niche
> > > >>>>>> situation,
> > > >>>>>>>>>> when
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> alternative (automatically wipe and restore
> > > >> stores)
> > > >>>>>> should be
> > > >>>>>>>>>>>>>>>>>>>> acceptable.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've
> > > >>>> requested.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> 3a.
> > > >>>>>>>>>>>>>>>>>>>>> I agree that IsolationLevel makes more sense at
> > > >>>>>> query-time,
> > > >>>>>>>> and
> > > >>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>> actually
> > > >>>>>>>>>>>>>>>>>>>>> initially attempted to place the IsolationLevel
> > > >> at
> > > >>>>>>>> query-time,
> > > >>>>>>>>>>>>>>>> but I
> > > >>>>>>>>>>>>>>>>>> ran
> > > >>>>>>>>>>>>>>>>>>>>> into some problems:
> > > >>>>>>>>>>>>>>>>>>>>> - The key issue is that, under ALOS we're not
> > > >> staging
> > > >>>>>> writes
> > > >>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>> transactions, so can't perform writes at the
> > > >>>>>> READ_COMMITTED
> > > >>>>>>>>>>>>>>>> isolation
> > > >>>>>>>>>>>>>>>>>>>>> level. However, this may be addressed if we
> > > >> decide to
> > > >>>>>>>> *always*
> > > >>>>>>>>>>>>>>>> use
> > > >>>>>>>>>>>>>>>>>>>>> transactions as discussed under 3b.
> > > >>>>>>>>>>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> > > >>>> implementations. I
> > > >>>>>>>>>> remember
> > > >>>>>>>>>>>>>>>>>> having
> > > >>>>>>>>>>>>>>>>>>>>> some difficulty understanding the IQv1 internals,
> > > >>>> which
> > > >>>>>> made
> > > >>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>> difficult
> > > >>>>>>>>>>>>>>>>>>>>> to determine what needed to be changed. However,
> > > >> I
> > > >>>>>> *think*
> > > >>>>>>>> this
> > > >>>>>>>>>>>>>>>> can be
> > > >>>>>>>>>>>>>>>>>>>>> addressed for both implementations by wrapping
> > > >> the
> > > >>>>>>>> RocksDBStore
> > > >>>>>>>>>>>>>>>> in an
> > > >>>>>>>>>>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides
> > > >> read
> > > >>>>>> methods
> > > >>>>>>>>>>>>>>>> (get,
> > > >>>>>>>>>>>>>>>>>>>> etc.)
> > > >>>>>>>>>>>>>>>>>>>>> to either read directly from the database or
> > > >> from the
> > > >>>>>> ongoing
> > > >>>>>>>>>>>>>>>>>>>> transaction.
> > > >>>>>>>>>>>>>>>>>>>>> But IQv1 might still be difficult.
> > > >>>>>>>>>>>>>>>>>>>>> - If IsolationLevel becomes a query constraint,
> > > >> then
> > > >>>> all
> > > >>>>>>>> other
> > > >>>>>>>>>>>>>>>>>>>> StateStores
> > > >>>>>>>>>>>>>>>>>>>>> will need to respect it, including the in-memory
> > > >>>> stores.
> > > >>>>>> This
> > > >>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>> require
> > > >>>>>>>>>>>>>>>>>>>>> us to adapt in-memory stores to stage their
> > > >> writes so
> > > >>>>>> they
> > > >>>>>>>> can
> > > >>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>> isolated
> > > >>>>>>>>>>>>>>>>>>>>> from READ_COMMITTTED queries. It would also
> > > >> become an
> > > >>>>>>>> important
> > > >>>>>>>>>>>>>>>>>>>>> consideration for third-party stores on upgrade,
> > > >> as
> > > >>>>>> without
> > > >>>>>>>>>>>>>>>> changes,
> > > >>>>>>>>>>>>>>>>>>>> they
> > > >>>>>>>>>>>>>>>>>>>>> would not support READ_COMMITTED queries
> > > >> correctly.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Ultimately, I may need some help making the
> > > >> necessary
> > > >>>>>> change
> > > >>>>>>>> to
> > > >>>>>>>>>>>>>>>> IQv1
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>> support this, but I don't think it's
> > > >> fundamentally
> > > >>>>>>>> impossible,
> > > >>>>>>>>>>>>>>>> if we
> > > >>>>>>>>>>>>>>>>>>>> want
> > > >>>>>>>>>>>>>>>>>>>>> to pursue this route.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> 3b.
> > > >>>>>>>>>>>>>>>>>>>>> The main reason I chose to keep ALOS
> > > >> un-transactional
> > > >>>>>> was to
> > > >>>>>>>>>>>>>>>> minimize
> > > >>>>>>>>>>>>>>>>>>>>> behavioural change for most users (I believe most
> > > >>>> Streams
> > > >>>>>>>> users
> > > >>>>>>>>>>>>>>>> use
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> default configuration, which is ALOS). That said,
> > > >>>> it's
> > > >>>>>> clear
> > > >>>>>>>>>>>>>>>> that if
> > > >>>>>>>>>>>>>>>>>>>> ALOS
> > > >>>>>>>>>>>>>>>>>>>>> also used transactional stores, the only change
> > > >> in
> > > >>>>>> behaviour
> > > >>>>>>>>>>>>>>>> would be
> > > >>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>> it would become *more correct*, which could be
> > > >>>>>> considered a
> > > >>>>>>>>>> "bug
> > > >>>>>>>>>>>>>>>> fix"
> > > >>>>>>>>>>>>>>>>>> by
> > > >>>>>>>>>>>>>>>>>>>>> users, rather than a change they need to handle.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> I believe that performance using transactions
> > > >> (aka.
> > > >>>>>> RocksDB
> > > >>>>>>>>>>>>>>>>>>>> WriteBatches)
> > > >>>>>>>>>>>>>>>>>>>>> should actually be *better* than the un-batched
> > > >>>>>> write-path
> > > >>>>>>>> that
> > > >>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>> currently used[1]. The only "performance"
> > > >>>> consideration
> > > >>>>>> will
> > > >>>>>>>> be
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> increased memory usage that transactions require.
> > > >>>> Given
> > > >>>>>> the
> > > >>>>>>>>>>>>>>>>>> mitigations
> > > >>>>>>>>>>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>> this memory that we have in place, I would expect
> > > >>>> that
> > > >>>>>> this
> > > >>>>>>>> is
> > > >>>>>>>>>>>>>>>> not a
> > > >>>>>>>>>>>>>>>>>>>>> problem for most users.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> If we're happy to do so, we can make ALOS also
> > > >> use
> > > >>>>>>>>>> transactions.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Link 1:
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>
> > > >>>>
> > > >>
> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> > > >>>>>>>>>>>> cadonna@apache.org
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Thanks for the updates and sorry for the delay
> > > >> on my
> > > >>>>>> side!
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>>>>>>>>> Making the default implementation for flush() a
> > > >>>> no-op
> > > >>>>>> sounds
> > > >>>>>>>>>>>>>>>> good to
> > > >>>>>>>>>>>>>>>>>>>> me.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>>>>>>>>> I think what was bugging me here is that a
> > > >>>> third-party
> > > >>>>>> state
> > > >>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>> needs
> > > >>>>>>>>>>>>>>>>>>>>>> to implement the state store interface. That
> > > >> means
> > > >>>> they
> > > >>>>>> need
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>> implement a wrapper around the actual state
> > > >> store
> > > >>>> as we
> > > >>>>>> do
> > > >>>>>>>> for
> > > >>>>>>>>>>>>>>>>>> RocksDB
> > > >>>>>>>>>>>>>>>>>>>>>> with RocksDBStore. So, a third-party state
> > > >> store can
> > > >>>>>> always
> > > >>>>>>>>>>>>>>>> estimate
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> uncommitted bytes, if it wants, because the
> > > >> wrapper
> > > >>>> can
> > > >>>>>>>> record
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> added
> > > >>>>>>>>>>>>>>>>>>>>>> bytes.
> > > >>>>>>>>>>>>>>>>>>>>>> One case I can think of where returning -1 makes
> > > >>>> sense
> > > >>>>>> is
> > > >>>>>>>> when
> > > >>>>>>>>>>>>>>>>>> Streams
> > > >>>>>>>>>>>>>>>>>>>>>> does not need to estimate the size of the write
> > > >>>> batch
> > > >>>>>> and
> > > >>>>>>>>>>>>>>>> trigger
> > > >>>>>>>>>>>>>>>>>>>>>> extraordinary commits, because the third-party
> > > >> state
> > > >>>>>> store
> > > >>>>>>>>>>>>>>>> takes care
> > > >>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>> memory. But in that case the method could also
> > > >> just
> > > >>>>>> return
> > > >>>>>>>> 0.
> > > >>>>>>>>>>>>>>>> Even
> > > >>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>> case would be better solved with a method that
> > > >>>> returns
> > > >>>>>>>> whether
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>>>> store manages itself the memory used for
> > > >> uncommitted
> > > >>>>>> bytes
> > > >>>>>>>> or
> > > >>>>>>>>>>>>>>>> not.
> > > >>>>>>>>>>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return
> > > >>>> value,
> > > >>>>>> I was
> > > >>>>>>>>>>>>>>>> just
> > > >>>>>>>>>>>>>>>>>>>>>> wondering when and if it will be used.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Regarding returning 0 for transactional state
> > > >> stores
> > > >>>>>> when
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>>> batch
> > > >>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>> empty, I was just wondering because you
> > > >> explicitly
> > > >>>>>> stated
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> "or {@code 0} if this StateStore does not
> > > >> support
> > > >>>>>>>>>>>> transactions."
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> So it seemed to me returning 0 could only
> > > >> happen for
> > > >>>>>>>>>>>>>>>>>> non-transactional
> > > >>>>>>>>>>>>>>>>>>>>>> state stores.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> a) What do you think if we move the isolation
> > > >> level
> > > >>>> to
> > > >>>>>> IQ
> > > >>>>>>>> (v1
> > > >>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>> v2)?
> > > >>>>>>>>>>>>>>>>>>>>>> In the end this is the only component that
> > > >> really
> > > >>>> needs
> > > >>>>>> to
> > > >>>>>>>>>>>>>>>> specify
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> isolation level. It is similar to the Kafka
> > > >> consumer
> > > >>>>>> that
> > > >>>>>>>> can
> > > >>>>>>>>>>>>>>>> choose
> > > >>>>>>>>>>>>>>>>>>>>>> with what isolation level to read the input
> > > >> topic.
> > > >>>>>>>>>>>>>>>>>>>>>> For IQv1 the isolation level should go into
> > > >>>>>>>>>>>>>>>> StoreQueryParameters. For
> > > >>>>>>>>>>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> b) Point a) raises the question what should
> > > >> happen
> > > >>>>>> during
> > > >>>>>>>>>>>>>>>>>> at-least-once
> > > >>>>>>>>>>>>>>>>>>>>>> processing when the state store does not use
> > > >>>>>> transactions?
> > > >>>>>>>>>> John
> > > >>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> past proposed to also use transactions on state
> > > >>>> stores
> > > >>>>>> for
> > > >>>>>>>>>>>>>>>>>>>>>> at-least-once. I like that idea, because it
> > > >> avoids
> > > >>>>>>>> aggregating
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> same
> > > >>>>>>>>>>>>>>>>>>>>>> records over and over again in the case of a
> > > >>>> failure. We
> > > >>>>>>>> had a
> > > >>>>>>>>>>>>>>>> case
> > > >>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>> the past where a Streams applications in
> > > >>>> at-least-once
> > > >>>>>> mode
> > > >>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>> failing
> > > >>>>>>>>>>>>>>>>>>>>>> continuously for some reasons I do not remember
> > > >>>> before
> > > >>>>>>>>>>>>>>>> committing the
> > > >>>>>>>>>>>>>>>>>>>>>> offsets. After each failover, the app aggregated
> > > >>>> again
> > > >>>>>> and
> > > >>>>>>>>>>>>>>>> again the
> > > >>>>>>>>>>>>>>>>>>>>>> same records. Of course the aggregate increased
> > > >> to
> > > >>>> very
> > > >>>>>>>> wrong
> > > >>>>>>>>>>>>>>>> values
> > > >>>>>>>>>>>>>>>>>>>>>> just because of the failover. With transactions
> > > >> on
> > > >>>> the
> > > >>>>>> state
> > > >>>>>>>>>>>>>>>> stores
> > > >>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>> could have avoided this. The app would have
> > > >> output
> > > >>>> the
> > > >>>>>> same
> > > >>>>>>>>>>>>>>>> aggregate
> > > >>>>>>>>>>>>>>>>>>>>>> multiple times (i.e., after each failover) but
> > > >> at
> > > >>>> least
> > > >>>>>> the
> > > >>>>>>>>>>>>>>>> value of
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> aggregate would not depend on the number of
> > > >>>> failovers.
> > > >>>>>>>>>>>>>>>> Outputting the
> > > >>>>>>>>>>>>>>>>>>>>>> same aggregate multiple times would be incorrect
> > > >>>> under
> > > >>>>>>>>>>>>>>>> exactly-once
> > > >>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>>>> it is OK for at-least-once.
> > > >>>>>>>>>>>>>>>>>>>>>> If it makes sense to add a config to turn on
> > > >> and off
> > > >>>>>>>>>>>>>>>> transactions on
> > > >>>>>>>>>>>>>>>>>>>>>> state stores under at-least-once or just use
> > > >>>>>> transactions in
> > > >>>>>>>>>>>>>>>> any case
> > > >>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>> a question we should also discuss in this KIP.
> > > >> It
> > > >>>>>> depends a
> > > >>>>>>>>>> bit
> > > >>>>>>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would
> > > >>>> add a
> > > >>>>>>>> config.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> 4.
> > > >>>>>>>>>>>>>>>>>>>>>> Your points are all valid. I tend to say to
> > > >> keep the
> > > >>>>>> metrics
> > > >>>>>>>>>>>>>>>> around
> > > >>>>>>>>>>>>>>>>>>>>>> flush() until we remove flush() completely from
> > > >> the
> > > >>>>>>>> interface.
> > > >>>>>>>>>>>>>>>> Calls
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>> flush() might still exist since existing
> > > >> processors
> > > >>>>>> might
> > > >>>>>>>>>> still
> > > >>>>>>>>>>>>>>>> call
> > > >>>>>>>>>>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For
> > > >>>> sure, we
> > > >>>>>> need
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> document
> > > >>>>>>>>>>>>>>>>>>>>>> how the metrics change due to the transactions
> > > >> in
> > > >>>> the
> > > >>>>>>>> upgrade
> > > >>>>>>>>>>>>>>>> notes.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> 5.
> > > >>>>>>>>>>>>>>>>>>>>>> I see. Then you should describe how the
> > > >> .position
> > > >>>> files
> > > >>>>>> are
> > > >>>>>>>>>>>>>>>> handled
> > > >>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>> a dedicated section of the KIP or incorporate
> > > >> the
> > > >>>>>>>> description
> > > >>>>>>>>>>>>>>>> in the
> > > >>>>>>>>>>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only
> > > >>>>>> mentioning it
> > > >>>>>>>>>> in
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> "Compatibility, 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:
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>
> > > >>>>
> > > >>
> >
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> It is a bit brittle, but it works.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Thanks for taking the time to review the KIP.
> > > >> I'm
> > > >>>> back
> > > >>>>>> from
> > > >>>>>>>>>>>>>>>> leave
> > > >>>>>>>>>>>>>>>>>> now
> > > >>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>> intend to move this forwards as quickly as I
> > > >> can.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Addressing your points:
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>>>>>>>>>> Because flush() is part of the StateStore API,
> > > >> it's
> > > >>>>>> exposed
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>> custom
> > > >>>>>>>>>>>>>>>>>>>>>>> Processors, which might be making calls to
> > > >> flush().
> > > >>>>>> This
> > > >>>>>>>> was
> > > >>>>>>>>>>>>>>>>>> actually
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> case in a few integration tests.
> > > >>>>>>>>>>>>>>>>>>>>>>> To maintain as much compatibility as possible,
> > > >> I'd
> > > >>>>>> prefer
> > > >>>>>>>> not
> > > >>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>> make
> > > >>>>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>> an UnsupportedOperationException, as it will
> > > >> cause
> > > >>>>>>>> previously
> > > >>>>>>>>>>>>>>>>>> working
> > > >>>>>>>>>>>>>>>>>>>>>>> Processors to start throwing exceptions at
> > > >> runtime.
> > > >>>>>>>>>>>>>>>>>>>>>>> I agree that it doesn't make sense for it to
> > > >> proxy
> > > >>>>>>>> commit(),
> > > >>>>>>>>>>>>>>>> though,
> > > >>>>>>>>>>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>>>>>>>>>> that would cause it to violate the "StateStores
> > > >>>> commit
> > > >>>>>> only
> > > >>>>>>>>>>>>>>>> when the
> > > >>>>>>>>>>>>>>>>>>>> Task
> > > >>>>>>>>>>>>>>>>>>>>>>> commits" rule.
> > > >>>>>>>>>>>>>>>>>>>>>>> Instead, I think we should make this a no-op.
> > > >> That
> > > >>>> way,
> > > >>>>>>>>>>>>>>>> existing
> > > >>>>>>>>>>>>>>>>>> user
> > > >>>>>>>>>>>>>>>>>>>>>>> Processors will continue to work as-before,
> > > >> without
> > > >>>>>>>> violation
> > > >>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>>>>> consistency that would be caused by premature
> > > >>>>>> flush/commit
> > > >>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>> StateStore
> > > >>>>>>>>>>>>>>>>>>>>>>> data to disk.
> > > >>>>>>>>>>>>>>>>>>>>>>> What do you think?
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
> > > >>>>>> implementation
> > > >>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>> transactional, but is unable to estimate the
> > > >>>>>> uncommitted
> > > >>>>>>>>>>>> memory
> > > >>>>>>>>>>>>>>>>>> usage,
> > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> method will return -1.
> > > >>>>>>>>>>>>>>>>>>>>>>> The intention here is to permit third-party
> > > >>>>>> implementations
> > > >>>>>>>>>>>>>>>> that may
> > > >>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>> able to estimate memory usage.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been
> > > >> written to
> > > >>>> the
> > > >>>>>>>> store
> > > >>>>>>>>>>>>>>>> yet. I
> > > >>>>>>>>>>>>>>>>>>>>>> thought
> > > >>>>>>>>>>>>>>>>>>>>>>> that was implied by "This method will return an
> > > >>>>>>>> approximation
> > > >>>>>>>>>>>>>>>> of the
> > > >>>>>>>>>>>>>>>>>>>>>> memory
> > > >>>>>>>>>>>>>>>>>>>>>>> would be freed by the next call to {@link
> > > >>>>>> #commit(Map)}"
> > > >>>>>>>> and
> > > >>>>>>>>>>>>>>>>>> "@return
> > > >>>>>>>>>>>>>>>>>>>> The
> > > >>>>>>>>>>>>>>>>>>>>>>> approximate size of all records awaiting {@link
> > > >>>>>>>>>>>> #commit(Map)}",
> > > >>>>>>>>>>>>>>>>>>>> however,
> > > >>>>>>>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you
> > > >> think
> > > >>>> this
> > > >>>>>> is
> > > >>>>>>>>>>>>>>>> unclear?
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>>>>>> I realise this is probably the most contentious
> > > >>>> point
> > > >>>>>> in my
> > > >>>>>>>>>>>>>>>> design,
> > > >>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>> I'm
> > > >>>>>>>>>>>>>>>>>>>>>>> open to changing it if I'm unable to convince
> > > >> you
> > > >>>> of
> > > >>>>>> the
> > > >>>>>>>>>>>>>>>> benefits.
> > > >>>>>>>>>>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > > >>>>>>>>>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly
> > > >>>> provided
> > > >>>>>>>>>>>>>>>> StateStores
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>> query, and it may be important for users to
> > > >>>>>>>> programmatically
> > > >>>>>>>>>>>>>>>> know
> > > >>>>>>>>>>>>>>>>>>>> which
> > > >>>>>>>>>>>>>>>>>>>>>>> mode the StateStore is operating under. If we
> > > >>>> simply
> > > >>>>>>>> provide
> > > >>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the
> > > >>>> internal
> > > >>>>>>>> streams
> > > >>>>>>>>>>>>>>>>>>>> engine), or
> > > >>>>>>>>>>>>>>>>>>>>>>> similar, then users will need to understand the
> > > >>>>>> operation
> > > >>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>> consequences
> > > >>>>>>>>>>>>>>>>>>>>>>> of each available processing mode and how it
> > > >>>> pertains
> > > >>>>>> to
> > > >>>>>>>>>> their
> > > >>>>>>>>>>>>>>>>>>>>>> StateStore.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Interactive Query users aren't the only people
> > > >> that
> > > >>>>>> care
> > > >>>>>>>>>> about
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
> > > >>>>>>>> implementers
> > > >>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>> custom
> > > >>>>>>>>>>>>>>>>>>>>>>> StateStores also need to understand the
> > > >> behaviour
> > > >>>>>> expected
> > > >>>>>>>> of
> > > >>>>>>>>>>>>>>>> their
> > > >>>>>>>>>>>>>>>>>>>>>>> implementation. KIP-892 introduces some
> > > >> assumptions
> > > >>>>>> into
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>>> Streams
> > > >>>>>>>>>>>>>>>>>>>>>> Engine
> > > >>>>>>>>>>>>>>>>>>>>>>> about how StateStores operate under each
> > > >> processing
> > > >>>>>> mode,
> > > >>>>>>>> and
> > > >>>>>>>>>>>>>>>> it's
> > > >>>>>>>>>>>>>>>>>>>>>>> important that custom implementations adhere to
> > > >>>> those
> > > >>>>>>>>>>>>>>>> assumptions in
> > > >>>>>>>>>>>>>>>>>>>>>> order
> > > >>>>>>>>>>>>>>>>>>>>>>> to maintain the consistency guarantees.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> IsolationLevels provide a high-level contract
> > > >> on
> > > >>>> the
> > > >>>>>>>>>> behaviour
> > > >>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> StateStore: a user knows that under
> > > >> READ_COMMITTED,
> > > >>>>>> they
> > > >>>>>>>> will
> > > >>>>>>>>>>>>>>>> see
> > > >>>>>>>>>>>>>>>>>>>> writes
> > > >>>>>>>>>>>>>>>>>>>>>>> only after the Task has committed, and under
> > > >>>>>>>> READ_UNCOMMITTED
> > > >>>>>>>>>>>>>>>> they
> > > >>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>> see
> > > >>>>>>>>>>>>>>>>>>>>>>> writes immediately. No understanding of the
> > > >>>> details of
> > > >>>>>> each
> > > >>>>>>>>>>>>>>>>>>>>>> processing.mode
> > > >>>>>>>>>>>>>>>>>>>>>>> is required, either for IQ users or StateStore
> > > >>>>>>>> implementers.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> An argument can be made that these contractual
> > > >>>>>> guarantees
> > > >>>>>>>> can
> > > >>>>>>>>>>>>>>>> simply
> > > >>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>> documented for the processing.mode (i.e. that
> > > >>>>>> exactly-once
> > > >>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
> > > >>>>>>>> at-least-once
> > > >>>>>>>>>>>>>>>> behaves
> > > >>>>>>>>>>>>>>>>>>>> like
> > > >>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small
> > > >>>> issues
> > > >>>>>> with
> > > >>>>>>>>>>>>>>>> this I'd
> > > >>>>>>>>>>>>>>>>>>>>>> prefer
> > > >>>>>>>>>>>>>>>>>>>>>>> to avoid:
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>          - Where would we document these
> > > >> contracts,
> > > >>>> in
> > > >>>>>> a way
> > > >>>>>>>>>> that
> > > >>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>> difficult
> > > >>>>>>>>>>>>>>>>>>>>>>>          for users/implementers to miss/ignore?
> > > >>>>>>>>>>>>>>>>>>>>>>>          - It's not clear to users that the
> > > >>>> processing
> > > >>>>>> mode
> > > >>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>> communicating
> > > >>>>>>>>>>>>>>>>>>>>>>>          an expectation of read isolation,
> > > >> unless
> > > >>>> they
> > > >>>>>> read
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> documentation. Users
> > > >>>>>>>>>>>>>>>>>>>>>>>          rarely consult documentation unless
> > > >> they
> > > >>>> feel
> > > >>>>>> they
> > > >>>>>>>>>> need
> > > >>>>>>>>>>>>>>>> to, so
> > > >>>>>>>>>>>>>>>>>>>> it's
> > > >>>>>>>>>>>>>>>>>>>>>> likely
> > > >>>>>>>>>>>>>>>>>>>>>>>          this detail would get missed by many
> > > >> users.
> > > >>>>>>>>>>>>>>>>>>>>>>>          - It tightly couples processing modes
> > > >> to
> > > >>>> read
> > > >>>>>>>>>> isolation.
> > > >>>>>>>>>>>>>>>> Adding
> > > >>>>>>>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>>>>>>>>          processing modes, or changing the read
> > > >>>>>> isolation of
> > > >>>>>>>>>>>>>>>> existing
> > > >>>>>>>>>>>>>>>>>>>>>> processing
> > > >>>>>>>>>>>>>>>>>>>>>>>          modes would be difficult/impossible.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Ultimately, the cost of introducing
> > > >>>> IsolationLevels is
> > > >>>>>>>> just a
> > > >>>>>>>>>>>>>>>> single
> > > >>>>>>>>>>>>>>>>>>>>>>> method, since we re-use the existing
> > > >> IsolationLevel
> > > >>>>>> enum
> > > >>>>>>>> from
> > > >>>>>>>>>>>>>>>> Kafka.
> > > >>>>>>>>>>>>>>>>>>>> This
> > > >>>>>>>>>>>>>>>>>>>>>>> gives us a clear place to document the
> > > >> contractual
> > > >>>>>>>> guarantees
> > > >>>>>>>>>>>>>>>>>> expected
> > > >>>>>>>>>>>>>>>>>>>>>>> of/provided by StateStores, that is accessible
> > > >>>> both by
> > > >>>>>> the
> > > >>>>>>>>>>>>>>>>>> StateStore
> > > >>>>>>>>>>>>>>>>>>>>>>> itself, and by IQ users.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> (Writing this I've just realised that the
> > > >>>> StateStore
> > > >>>>>> and IQ
> > > >>>>>>>>>>>>>>>> APIs
> > > >>>>>>>>>>>>>>>>>>>> actually
> > > >>>>>>>>>>>>>>>>>>>>>>> don't provide access to StateStoreContext that
> > > >> IQ
> > > >>>> users
> > > >>>>>>>> would
> > > >>>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>>>>> direct
> > > >>>>>>>>>>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> > > >>>>>>>>>> isolationLevel()
> > > >>>>>>>>>>>>>>>>>> itself
> > > >>>>>>>>>>>>>>>>>>>>>> too?)
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 4.
> > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics
> > > >>>> in-place
> > > >>>>>>>>>>>>>>>> either, as
> > > >>>>>>>>>>>>>>>>>>>> it's a
> > > >>>>>>>>>>>>>>>>>>>>>>> backwards incompatible change. My concern is
> > > >> that,
> > > >>>> if
> > > >>>>>> we
> > > >>>>>>>>>> leave
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> existing
> > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics in place, they will be
> > > >> confusing to
> > > >>>>>> users.
> > > >>>>>>>>>>>>>>>> Right
> > > >>>>>>>>>>>>>>>>>> now,
> > > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics record explicit flushes to
> > > >> disk,
> > > >>>> but
> > > >>>>>> under
> > > >>>>>>>>>>>>>>>> KIP-892,
> > > >>>>>>>>>>>>>>>>>>>> even
> > > >>>>>>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>> commit() will not explicitly flush data to
> > > >> disk -
> > > >>>>>> RocksDB
> > > >>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>> decide
> > > >>>>>>>>>>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>>>>>>> when to flush memtables to disk itself.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd
> > > >> have
> > > >>>> two
> > > >>>>>>>>>>>> options,
> > > >>>>>>>>>>>>>>>>>> which
> > > >>>>>>>>>>>>>>>>>>>>>> both
> > > >>>>>>>>>>>>>>>>>>>>>>> seem pretty bad to me:
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>          1. Have them record calls to commit(),
> > > >>>> which
> > > >>>>>> would
> > > >>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>> misleading, as
> > > >>>>>>>>>>>>>>>>>>>>>>>          data is no longer explicitly "flushed"
> > > >> to
> > > >>>> disk
> > > >>>>>> by
> > > >>>>>>>> this
> > > >>>>>>>>>>>>>>>> call.
> > > >>>>>>>>>>>>>>>>>>>>>>>          2. Have them record nothing at all,
> > > >> which
> > > >>>> is
> > > >>>>>>>>>> equivalent
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> removing
> > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>          metrics, except that users will see
> the
> > > >>>> metric
> > > >>>>>>>> still
> > > >>>>>>>>>>>>>>>> exists and
> > > >>>>>>>>>>>>>>>>>>>> so
> > > >>>>>>>>>>>>>>>>>>>>>> assume
> > > >>>>>>>>>>>>>>>>>>>>>>>          that the metric is correct, and that
> > > >>>> there's a
> > > >>>>>>>> problem
> > > >>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>> their
> > > >>>>>>>>>>>>>>>>>>>>>> system
> > > >>>>>>>>>>>>>>>>>>>>>>>          when there isn't.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> I agree that removing them is also a bad
> > > >> solution,
> > > >>>> and
> > > >>>>>> I'd
> > > >>>>>>>>>>>>>>>> like some
> > > >>>>>>>>>>>>>>>>>>>>>>> guidance on the best path forward here.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 5.
> > > >>>>>>>>>>>>>>>>>>>>>>> Position files are updated on every write to a
> > > >>>>>> StateStore.
> > > >>>>>>>>>>>>>>>> Since our
> > > >>>>>>>>>>>>>>>>>>>>>> writes
> > > >>>>>>>>>>>>>>>>>>>>>>> are now buffered until commit(), we can't
> > > >> update
> > > >>>> the
> > > >>>>>>>> Position
> > > >>>>>>>>>>>>>>>> file
> > > >>>>>>>>>>>>>>>>>>>> until
> > > >>>>>>>>>>>>>>>>>>>>>>> commit() has been called, otherwise it would be
> > > >>>>>>>> inconsistent
> > > >>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> data
> > > >>>>>>>>>>>>>>>>>>>>>>> in the event of a rollback. Consequently, we
> > > >> need
> > > >>>> to
> > > >>>>>> manage
> > > >>>>>>>>>>>>>>>> these
> > > >>>>>>>>>>>>>>>>>>>> offsets
> > > >>>>>>>>>>>>>>>>>>>>>>> the same way we manage the checkpoint offsets,
> > > >> and
> > > >>>>>> ensure
> > > >>>>>>>>>>>>>>>> they're
> > > >>>>>>>>>>>>>>>>>> only
> > > >>>>>>>>>>>>>>>>>>>>>>> written on commit().
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 6.
> > > >>>>>>>>>>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what
> > > >>>> tests to
> > > >>>>>>>>>> write.
> > > >>>>>>>>>>>>>>>> How
> > > >>>>>>>>>>>>>>>>>>>>>> explicit
> > > >>>>>>>>>>>>>>>>>>>>>>> do we need to be here in the KIP?
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed
> > > >> to be
> > > >>>>>>>> seamless,
> > > >>>>>>>>>>>>>>>> and we
> > > >>>>>>>>>>>>>>>>>>>>>> should
> > > >>>>>>>>>>>>>>>>>>>>>>> definitely add some tests around that.
> > > >> Downgrade,
> > > >>>> it
> > > >>>>>>>>>>>>>>>> transpires,
> > > >>>>>>>>>>>>>>>>>> isn't
> > > >>>>>>>>>>>>>>>>>>>>>>> currently possible, as the extra column family
> > > >> for
> > > >>>>>> offset
> > > >>>>>>>>>>>>>>>> storage is
> > > >>>>>>>>>>>>>>>>>>>>>>> incompatible with the pre-KIP-892
> > > >> implementation:
> > > >>>> when
> > > >>>>>> you
> > > >>>>>>>>>>>>>>>> open a
> > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > > >>>>>>>>>>>>>>>>>>>>>>> database, you must open all available column
> > > >>>> families
> > > >>>>>> or
> > > >>>>>>>>>>>>>>>> receive an
> > > >>>>>>>>>>>>>>>>>>>>>> error.
> > > >>>>>>>>>>>>>>>>>>>>>>> What currently happens on downgrade is that it
> > > >>>>>> attempts to
> > > >>>>>>>>>>>>>>>> open the
> > > >>>>>>>>>>>>>>>>>>>>>> store,
> > > >>>>>>>>>>>>>>>>>>>>>>> throws an error about the offsets column
> > > >> family not
> > > >>>>>> being
> > > >>>>>>>>>>>>>>>> opened,
> > > >>>>>>>>>>>>>>>>>>>> which
> > > >>>>>>>>>>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given
> > > >> that
> > > >>>>>>>>>> downgrades
> > > >>>>>>>>>>>>>>>>>> should
> > > >>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>> uncommon, I think this is acceptable
> > > >> behaviour, as
> > > >>>> the
> > > >>>>>>>>>>>>>>>> end-state is
> > > >>>>>>>>>>>>>>>>>>>>>>> consistent, even if it results in an
> > > >> undesirable
> > > >>>> state
> > > >>>>>>>>>>>> restore.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Should I document the upgrade/downgrade
> > > >> behaviour
> > > >>>>>>>> explicitly
> > > >>>>>>>>>>>>>>>> in the
> > > >>>>>>>>>>>>>>>>>>>> KIP?
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Nick!
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates!
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> > > >>>>>>>>>>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > > >>>>>>>>>>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore
> > > >>>> after
> > > >>>>>> this
> > > >>>>>>>> KIP
> > > >>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>> released, I would rather throw an unsupported
> > > >>>>>> operation
> > > >>>>>>>>>>>>>>>> exception
> > > >>>>>>>>>>>>>>>>>> by
> > > >>>>>>>>>>>>>>>>>>>>>>>> default.
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>>>>>>>>>>> When would a state store return -1 from
> > > >>>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
> > > >> while
> > > >>>>>> being
> > > >>>>>>>>>>>>>>>>>>>> transactional?
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Wouldn't
> > > >>>> StateStore#approximateNumUncommittedBytes()
> > > >>>>>> also
> > > >>>>>>>>>>>>>>>> return 0
> > > >>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>> the state store is transactional but nothing
> > > >> has
> > > >>>> been
> > > >>>>>>>>>> written
> > > >>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> state store yet?
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this
> > > >> KIP
> > > >>>> really
> > > >>>>>>>> need
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> introduce
> > > >>>>>>>>>>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> > > >>>> StateStoreContext
> > > >>>>>> has
> > > >>>>>>>>>>>>>>>> already
> > > >>>>>>>>>>>>>>>>>>>>>>>> appConfigs() which basically exposes the same
> > > >>>>>> information,
> > > >>>>>>>>>>>>>>>> i.e., if
> > > >>>>>>>>>>>>>>>>>>>> EOS
> > > >>>>>>>>>>>>>>>>>>>>>>>> is enabled or not.
> > > >>>>>>>>>>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore
> > > >>>> interface
> > > >>>>>> as
> > > >>>>>>>>>>>>>>>> loosely
> > > >>>>>>>>>>>>>>>>>>>> coupled
> > > >>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > > >>>>>> implementers
> > > >>>>>>>>>>>> more
> > > >>>>>>>>>>>>>>>>>>>> freedom,
> > > >>>>>>>>>>>>>>>>>>>>>>>> and reduce the amount of internal knowledge
> > > >>>> required."
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> While I understand the intent, I doubt that it
> > > >>>>>> decreases
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> coupling of
> > > >>>>>>>>>>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> > > >>>>>>>>>> READ_COMMITTED
> > > >>>>>>>>>>>>>>>> only
> > > >>>>>>>>>>>>>>>>>>>>>>>> applies to IQ but not to reads by processors.
> > > >>>> Thus,
> > > >>>>>>>>>>>>>>>> implementers
> > > >>>>>>>>>>>>>>>>>>>> need to
> > > >>>>>>>>>>>>>>>>>>>>>>>> understand how Streams accesses the state
> > > >> stores.
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> I would like to hear what others think about
> > > >> this.
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> 4.
> > > >>>>>>>>>>>>>>>>>>>>>>>> Great exposing new metrics for transactional
> > > >> state
> > > >>>>>> stores!
> > > >>>>>>>>>>>>>>>>>> However, I
> > > >>>>>>>>>>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate
> > > >> (in
> > > >>>> the
> > > >>>>>>>> docs)
> > > >>>>>>>>>>>>>>>> the 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 also need to adapt/add
> > > >> integration
> > > >>>>>> tests
> > > >>>>>>>>>>>>>>>> besides
> > > >>>>>>>>>>>>>>>>>> unit
> > > >>>>>>>>>>>>>>>>>>>>>>>> tests. Additionally, we probably need
> > > >> integration
> > > >>>> or
> > > >>>>>>>> system
> > > >>>>>>>>>>>>>>>> tests
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>> verify that upgrades and downgrades between
> > > >>>>>> transactional
> > > >>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>> non-transactional state stores work as
> > > >> expected.
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in
> > > >> the
> > > >>>> KIP,
> > > >>>>>>>> under
> > > >>>>>>>>>>>>>>>>>>>> "Rejected
> > > >>>>>>>>>>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea
> > > >> worth
> > > >>>>>> pursuing,
> > > >>>>>>>>>>>>>>>> but I
> > > >>>>>>>>>>>>>>>>>>>> believe
> > > >>>>>>>>>>>>>>>>>>>>>>>>> that it's out of the scope of this KIP,
> > > >> because
> > > >>>> it
> > > >>>>>>>> solves a
> > > >>>>>>>>>>>>>>>>>>>> different
> > > >>>>>>>>>>>>>>>>>>>>>> set
> > > >>>>>>>>>>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of
> > > >> this
> > > >>>> one
> > > >>>>>> has
> > > >>>>>>>>>>>>>>>> already
> > > >>>>>>>>>>>>>>>>>> grown
> > > >>>>>>>>>>>>>>>>>>>>>>>> quite
> > > >>>>>>>>>>>>>>>>>>>>>>>>> large!
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP (
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>
> > > >>>>
> > > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > >>>>>>>>>>>>>>>>>>>>>>>> )
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing
> > > >> back
> > > >>>>>> "Atomic
> > > >>>>>>>>>>>>>>>>>>>> Checkpointing"
> > > >>>>>>>>>>>>>>>>>>>>>>>> (for
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think
> > > >> the one
> > > >>>>>> thing
> > > >>>>>>>>>>>>>>>> missing is
> > > >>>>>>>>>>>>>>>>>>>> some
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> changes to metrics (notably the store
> > > >> "flush"
> > > >>>>>> metrics
> > > >>>>>>>> will
> > > >>>>>>>>>>>>>>>> need
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> renamed to "commit").
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> The reason I brought back Atomic
> > > >> Checkpointing
> > > >>>> was
> > > >>>>>> to
> > > >>>>>>>>>>>>>>>> decouple
> > > >>>>>>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>>>>>> flush
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> from store commit. This is important,
> > > >> because
> > > >>>> with
> > > >>>>>>>>>>>>>>>> Transactional
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on
> > > >>>> *every*
> > > >>>>>> Task
> > > >>>>>>>>>>>>>>>> commit,
> > > >>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> just when the StateStore is closing,
> > > >> otherwise
> > > >>>> our
> > > >>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>> buffer
> > > >>>>>>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> never be written and persisted, instead
> > > >> growing
> > > >>>>>>>> unbounded!
> > > >>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>>>>>> experimented
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a
> > > >> store
> > > >>>>>> flush
> > > >>>>>>>>>>>>>>>> whenever
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its
> > > >>>>>> configured
> > > >>>>>>>>>>>>>>>> size, but
> > > >>>>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer
> > > >>>> from
> > > >>>>>> being
> > > >>>>>>>>>>>>>>>>>> configured
> > > >>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> unbounded, and it still would have required
> > > >>>> explicit
> > > >>>>>>>>>>>>>>>> flushes of
> > > >>>>>>>>>>>>>>>>>>>>>> RocksDB,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory
> > > >>>>>> utilization.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the
> > > >> "right"
> > > >>>> way
> > > >>>>>> to
> > > >>>>>>>>>>>>>>>> resolve
> > > >>>>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog
> > > >> offsets
> > > >>>> that
> > > >>>>>>>>>>>>>>>> correspond
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> most
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> recently written records are always
> > > >> atomically
> > > >>>>>> written
> > > >>>>>>>> to
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> StateStore
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> (by writing them to the same transaction
> > > >>>> buffer),
> > > >>>>>> we can
> > > >>>>>>>>>>>>>>>> avoid
> > > >>>>>>>>>>>>>>>>>>>>>> forcibly
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk,
> > > >> letting
> > > >>>>>> RocksDB
> > > >>>>>>>>>>>>>>>> flush
> > > >>>>>>>>>>>>>>>>>> them
> > > >>>>>>>>>>>>>>>>>>>>>> only
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> when necessary, without losing any of our
> > > >>>>>> consistency
> > > >>>>>>>>>>>>>>>> guarantees.
> > > >>>>>>>>>>>>>>>>>>>> See
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> updated KIP for more info.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> I have fully implemented these changes,
> > > >>>> although I'm
> > > >>>>>>>> still
> > > >>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>> entirely
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> happy with the implementation for segmented
> > > >>>>>> StateStores,
> > > >>>>>>>>>> so
> > > >>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>> plan
> > > >>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests
> > > >> pass. If
> > > >>>>>> you'd
> > > >>>>>>>> like
> > > >>>>>>>>>>>>>>>> to try
> > > >>>>>>>>>>>>>>>>>>>> out
> > > >>>>>>>>>>>>>>>>>>>>>> or
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> review this highly experimental and
> > > >> incomplete
> > > >>>>>> branch,
> > > >>>>>>>>>> it's
> > > >>>>>>>>>>>>>>>>>>>> available
> > > >>>>>>>>>>>>>>>>>>>>>>>> here:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > > >>>>>>>> .
> > > >>>>>>>>>>>>>>>> Note:
> > > >>>>>>>>>>>>>>>>>>>> it's
> > > >>>>>>>>>>>>>>>>>>>>>>>> built
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable
> > > >> base
> > > >>>> to
> > > >>>>>> build
> > > >>>>>>>>>>>>>>>> and test
> > > >>>>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>> on,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples
> > > >> comparisons
> > > >>>> in a
> > > >>>>>>>> live
> > > >>>>>>>>>>>>>>>>>>>>>> environment. I
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's
> > > >> nearer
> > > >>>>>>>>>> completion
> > > >>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>> has
> > > >>>>>>>>>>>>>>>>>>>>>> been
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> proven on our main application.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing
> > > >> and
> > > >>>>>> testing:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window)
> > > >>>> stores
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> - Global stores
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> As I do not currently use either of these,
> > > >> so my
> > > >>>>>> primary
> > > >>>>>>>>>>>>>>>> test
> > > >>>>>>>>>>>>>>>>>>>>>>>> environment
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> doesn't test these areas.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next
> > > >> week
> > > >>>> for
> > > >>>>>> a few
> > > >>>>>>>>>>>>>>>> weeks,
> > > >>>>>>>>>>>>>>>>>> so
> > > >>>>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> not have time to move this forward until
> > > >> late
> > > >>>>>> August.
> > > >>>>>>>> That
> > > >>>>>>>>>>>>>>>> said,
> > > >>>>>>>>>>>>>>>>>>>> your
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just
> > > >>>> won't be
> > > >>>>>>>> able
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> respond
> > > >>>>>>>>>>>>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> quickly as usual.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact
> > > >> on IQ
> > > >>>> is
> > > >>>>>> not
> > > >>>>>>>>>>>>>>>> something
> > > >>>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>> had
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> considered.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the state
> > > >> store
> > > >>>>>> from the
> > > >>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> > > >> the
> > > >>>>>>>> checkpoint
> > > >>>>>>>>>>>>>>>> (thus,
> > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> > > >> data
> > > >>>>>> and/or
> > > >>>>>>>>>>>>>>>> number of
> > > >>>>>>>>>>>>>>>>>>>>>> commit
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you
> > > >> suggesting
> > > >>>>>> that we
> > > >>>>>>>>>>>>>>>> add an
> > > >>>>>>>>>>>>>>>>>>>>>>>> additional
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> config for the max number of commit
> > > >> intervals
> > > >>>>>> between
> > > >>>>>>>>>>>>>>>>>> checkpoints?
> > > >>>>>>>>>>>>>>>>>>>>>> That
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the
> > > >>>>>> transaction
> > > >>>>>>>>>>>>>>>> buffers
> > > >>>>>>>>>>>>>>>>>> are
> > > >>>>>>>>>>>>>>>>>>>>>>>> nearly
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of
> > > >> commit
> > > >>>>>>>> intervals
> > > >>>>>>>>>>>>>>>> have
> > > >>>>>>>>>>>>>>>>>>>>>> elapsed,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> whichever comes first?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although
> > > >> this
> > > >>>>>>>> re-ignites
> > > >>>>>>>>>>>>>>>> an
> > > >>>>>>>>>>>>>>>>>>>> earlier
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> debate about whether a config should be
> > > >>>> measured in
> > > >>>>>>>>>>>>>>>> "number of
> > > >>>>>>>>>>>>>>>>>>>> commit
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute
> > > >> time.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the
> > > >> reason
> > > >>>> I
> > > >>>>>> was
> > > >>>>>>>>>>>>>>>> pursuing
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> Atomic
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable
> > > >> flush
> > > >>>> from
> > > >>>>>>>>>>>>>>>> checkpointing,
> > > >>>>>>>>>>>>>>>>>>>>>> which
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every
> > > >> commit
> > > >>>>>> without
> > > >>>>>>>> any
> > > >>>>>>>>>>>>>>>>>>>> performance
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely
> > > >> the
> > > >>>>>> "best"
> > > >>>>>>>>>>>>>>>> solution,
> > > >>>>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>>>>>> I'm not
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back
> > > >> into
> > > >>>> this
> > > >>>>>> KIP.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
> > > >>>>>> transactional
> > > >>>>>>>>>>>> logic
> > > >>>>>>>>>>>>>>>>>>>> directly
> > > >>>>>>>>>>>>>>>>>>>>>>>> into
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with
> > > >> the
> > > >>>>>>>>>>>>>>>>>>>>>> StateStore#newTransaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> method, and reduces the number of new
> > > >> classes
> > > >>>>>>>> introduced,
> > > >>>>>>>>>>>>>>>>>>>>>> significantly
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and
> > > >> the
> > > >>>>>>>> complexity
> > > >>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>> drastically
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic
> > > >>>> Checkpoints
> > > >>>>>>>> into
> > > >>>>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>> KIP.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna
> > > >> <
> > > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> As far as I understand, you want to
> > > >> atomically
> > > >>>>>> update
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the
> > > >>>> memtable
> > > >>>>>> of a
> > > >>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit
> > > >> time
> > > >>>>>> elapsed
> > > >>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>> after
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that
> > > >> would
> > > >>>> lead
> > > >>>>>> to
> > > >>>>>>>>>>>>>>>> exceeding
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > >>>> before the
> > > >>>>>>>> next
> > > >>>>>>>>>>>>>>>> commit
> > > >>>>>>>>>>>>>>>>>>>>>>>> interval
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> ends.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would
> > > >> commit
> > > >>>>>> every
> > > >>>>>>>>>>>>>>>> commit
> > > >>>>>>>>>>>>>>>>>>>> interval
> > > >>>>>>>>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store will only be atomically
> > > >>>> updated
> > > >>>>>>>> roughly
> > > >>>>>>>>>>>>>>>> every
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of
> > > >>>> data.
> > > >>>>>> Also
> > > >>>>>>>> IQ
> > > >>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>> then
> > > >>>>>>>>>>>>>>>>>>>>>>>> only
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see new data roughly every
> > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> After a failure the state store needs to
> > > >>>> restore
> > > >>>>>> up to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Is this correct?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the state
> > > >> store
> > > >>>>>> from
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> > > >> the
> > > >>>>>>>> checkpoint
> > > >>>>>>>>>>>>>>>> (thus,
> > > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> > > >> data
> > > >>>>>> and/or
> > > >>>>>>>>>>>>>>>> number of
> > > >>>>>>>>>>>>>>>>>>>>>> commit
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have
> > > >> the
> > > >>>> same
> > > >>>>>> delay
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>>>>>>>>> records
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because
> > > >> both
> > > >>>>>> would
> > > >>>>>>>>>>>>>>>> appear
> > > >>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However,
> > > >>>> after a
> > > >>>>>>>> failure
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it might restore data that is already in
> > > >> the
> > > >>>> state
> > > >>>>>>>> store
> > > >>>>>>>>>>>>>>>>>> because
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable
> > > >> offset
> > > >>>>>> (i.e.
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>>> last
> > > >>>>>>>>>>>>>>>>>>>>>> committed
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring
> > > >>>> data
> > > >>>>>> that
> > > >>>>>>>> is
> > > >>>>>>>>>>>>>>>> already
> > > >>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should
> > > >> not
> > > >>>>>> violated.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> This solution needs at least one new
> > > >> config to
> > > >>>>>> specify
> > > >>>>>>>>>>>>>>>> when a
> > > >>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> should be written.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail
> > > >>>> that
> > > >>>>>> does
> > > >>>>>>>> not
> > > >>>>>>>>>>>>>>>> change
> > > >>>>>>>>>>>>>>>>>>>>>>>> anything
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit
> > > >>>> interval
> > > >>>>>> is 30
> > > >>>>>>>>>>>>>>>> seconds,
> > > >>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>>> five
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> seconds.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch
> > > >> on
> > > >>>> our
> > > >>>>>>>> staging
> > > >>>>>>>>>>>>>>>>>>>>>> environment,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
> > > >>>> non-trivial
> > > >>>>>>>>>>>>>>>> application.
> > > >>>>>>>>>>>>>>>>>> I'm
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate
> > > >> that
> > > >>>> we
> > > >>>>>> saw
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>>> last
> > > >>>>>>>>>>>>>>>>>>>> time
> > > >>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP,
> > > >> but
> > > >>>> this
> > > >>>>>>>> time,
> > > >>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>> think I
> > > >>>>>>>>>>>>>>>>>>>>>> know
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> why.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit,
> > > >> which is
> > > >>>>>> called
> > > >>>>>>>> at
> > > >>>>>>>>>>>>>>>> the end
> > > >>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Task
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit process, has the following
> > > >> behaviour:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under ALOS: checkpoint the
> > > >> state
> > > >>>>>> stores.
> > > >>>>>>>>>> This
> > > >>>>>>>>>>>>>>>>>> includes
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            flushing memtables in RocksDB.
> > > >>>> This is
> > > >>>>>>>>>>>> acceptable
> > > >>>>>>>>>>>>>>>>>>>> because the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> default
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit.interval.ms is 5
> > > >> seconds,
> > > >>>> so
> > > >>>>>>>> forcibly
> > > >>>>>>>>>>>>>>>> flushing
> > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> every 5
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            seconds is acceptable for most
> > > >>>>>>>> applications.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under EOS: checkpointing is
> > > >> not
> > > >>>> done,
> > > >>>>>>>>>> *unless*
> > > >>>>>>>>>>>>>>>> it's
> > > >>>>>>>>>>>>>>>>>>>> being
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> forced, due
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to e.g. the Task closing or
> > > >> being
> > > >>>>>> revoked.
> > > >>>>>>>>>> This
> > > >>>>>>>>>>>>>>>> means
> > > >>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>> under
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> normal
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing conditions, the
> > > >> state
> > > >>>> stores
> > > >>>>>>>> will
> > > >>>>>>>>>> not
> > > >>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>> checkpointed,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            not have memtables flushed at
> > > >> all ,
> > > >>>>>> unless
> > > >>>>>>>>>>>> RocksDB
> > > >>>>>>>>>>>>>>>>>>>> decides to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush them on
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            its own. Checkpointing stores
> > > >> and
> > > >>>>>>>>>> force-flushing
> > > >>>>>>>>>>>>>>>> their
> > > >>>>>>>>>>>>>>>>>>>>>> memtables
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            done when a Task is being
> > > >> closed.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint
> > > >>>> stores on
> > > >>>>>> at
> > > >>>>>>>>>>>> least
> > > >>>>>>>>>>>>>>>>>> *some*
> > > >>>>>>>>>>>>>>>>>>>>>>>> normal
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the
> > > >> RocksDB
> > > >>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>> buffers
> > > >>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets
> > > >> are
> > > >>>>>> synced to
> > > >>>>>>>>>>>>>>>> disk to
> > > >>>>>>>>>>>>>>>>>>>>>> prevent
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
> > > >>>> Consequently,
> > > >>>>>> my
> > > >>>>>>>>>>>>>>>> current
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task
> > > >> commit,
> > > >>>>>> which
> > > >>>>>>>> is
> > > >>>>>>>>>>>>>>>> far too
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> frequent.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000
> > > >> records,
> > > >>>>>> which
> > > >>>>>>>> is
> > > >>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>> change
> > > >>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
> > > >> performance
> > > >>>>>> problems
> > > >>>>>>>> for
> > > >>>>>>>>>>>>>>>> some
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> applications.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and
> > > >> I'm
> > > >>>>>>>> currently
> > > >>>>>>>>>>>>>>>> leaning
> > > >>>>>>>>>>>>>>>>>>>>>>>> towards
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> using the
> > > >>>> statestore.transaction.buffer.max.bytes
> > > >>>>>>>>>>>>>>>>>> configuration
> > > >>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to
> > > >>>> exceed it.
> > > >>>>>>>> This
> > > >>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> complement the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
> > > >> functionality
> > > >>>> that
> > > >>>>>> this
> > > >>>>>>>>>>>>>>>>>>>> configuration
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Currently, we use
> > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > >>>>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> force an
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            early Task commit if
> processing
> > > >>>> more
> > > >>>>>>>> records
> > > >>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>> cause
> > > >>>>>>>>>>>>>>>>>>>> our
> > > >>>>>>>>>>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            transactions to exceed the
> > > >> memory
> > > >>>>>> assigned
> > > >>>>>>>> to
> > > >>>>>>>>>>>>>>>> them.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - New functionality: when a
> > > >> Task
> > > >>>> *does*
> > > >>>>>>>>>> commit,
> > > >>>>>>>>>>>>>>>> we will
> > > >>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the stores (and hence flush
> the
> > > >>>>>> transaction
> > > >>>>>>>>>>>>>>>> buffers)
> > > >>>>>>>>>>>>>>>>>>>> unless
> > > >>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> expect to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            cross the
> > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > >>>>>>>>>>>>>>>>>>>> threshold
> > > >>>>>>>>>>>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the next
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick
> > > >> Telford <
> > > >>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I
> > > >> meant in
> > > >>>>>> terms of
> > > >>>>>>>>>>>>>>>>>>>> understanding
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance profile under various
> > > >>>>>> circumstances. The
> > > >>>>>>>>>>>>>>>> more
> > > >>>>>>>>>>>>>>>>>>>> complex
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would
> > > >> be for
> > > >>>>>> users
> > > >>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> understand
> > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance they see. For example,
> > > >> spilling
> > > >>>>>> records
> > > >>>>>>>> to
> > > >>>>>>>>>>>>>>>> disk
> > > >>>>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold
> > > >>>> would, I
> > > >>>>>>>>>> expect,
> > > >>>>>>>>>>>>>>>>>> reduce
> > > >>>>>>>>>>>>>>>>>>>>>> write
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
> > > >>>> throughput
> > > >>>>>> could
> > > >>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>> unexpected,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> > > >>>> diagnose/understand for
> > > >>>>>>>>>> users.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early
> > > >> commit"
> > > >>>>>> concept is
> > > >>>>>>>>>>>>>>>>>> relatively
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document,
> > > >> and
> > > >>>>>>>>>>>> conceptually
> > > >>>>>>>>>>>>>>>>>> fairly
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> obvious to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to
> > > >>>> make it
> > > >>>>>>>>>> easier
> > > >>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>> understand
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it happens though.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an
> > > >>>> indirect
> > > >>>>>> way
> > > >>>>>>>> of
> > > >>>>>>>>>>>>>>>>>> telling
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
> > > >>>>>> READ_COMMITTED
> > > >>>>>>>>>>>>>>>>>>>> essentially
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> requires
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that
> > > >> two
> > > >>>>>> threads
> > > >>>>>>>>>>>>>>>> calling
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes
> > > >>>> from
> > > >>>>>> the
> > > >>>>>>>>>> other
> > > >>>>>>>>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> until
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> > > >>>>>> READ_UNCOMMITTED, all
> > > >>>>>>>>>>>>>>>> bets are
> > > >>>>>>>>>>>>>>>>>>>> off,
> > > >>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe
> > > >> written
> > > >>>>>> records
> > > >>>>>>>> at
> > > >>>>>>>>>>>>>>>> any
> > > >>>>>>>>>>>>>>>>>>>> time,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> which is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That
> > > >> said,
> > > >>>>>>>> StateStores
> > > >>>>>>>>>>>>>>>> are
> > > >>>>>>>>>>>>>>>>>> free
> > > >>>>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implement
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these guarantees however they can,
> > > >> which is
> > > >>>> a
> > > >>>>>> bit
> > > >>>>>>>> more
> > > >>>>>>>>>>>>>>>>>> relaxed
> > > >>>>>>>>>>>>>>>>>>>>>> than
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions".
> > > >> For
> > > >>>>>> example,
> > > >>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>> RocksDB
> > > >>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED ==
> > > >>>> WBWI-based
> > > >>>>>>>>>>>>>>>>>> "transactions",
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the
> > > >>>>>> database.
> > > >>>>>>>> But
> > > >>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>> other
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> storage
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to
> > > >> *always*
> > > >>>> use
> > > >>>>>>>>>>>>>>>> transactions,
> > > >>>>>>>>>>>>>>>>>>>> even
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage
> > > >> engines
> > > >>>>>> that
> > > >>>>>>>>>> don't
> > > >>>>>>>>>>>>>>>>>> provide
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation
> > > >> guarantees
> > > >>>> can
> > > >>>>>> be
> > > >>>>>>>> met
> > > >>>>>>>>>>>>>>>> using a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> different
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> technique.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the
> > > >> StateStore
> > > >>>>>> interface
> > > >>>>>>>> as
> > > >>>>>>>>>>>>>>>>>> loosely
> > > >>>>>>>>>>>>>>>>>>>>>>>> coupled
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to
> > > >> give
> > > >>>>>>>>>>>>>>>> implementers
> > > >>>>>>>>>>>>>>>>>> more
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> freedom, and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge
> > > >>>>>> required.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> > > >>>> "IsolationLevel"
> > > >>>>>> might
> > > >>>>>>>>>> not
> > > >>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> right
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it
> > > >> much
> > > >>>> more
> > > >>>>>>>>>>>>>>>> explicit if
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> required, e.g.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7-8.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I can make these changes either later
> > > >> today
> > > >>>> or
> > > >>>>>>>>>>>> tomorrow.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Small update:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and
> > > >> fixed a
> > > >>>>>> bunch of
> > > >>>>>>>>>>>>>>>> issues
> > > >>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests
> > > >> pass,
> > > >>>>>> which is
> > > >>>>>>>>>>>>>>>>>> promising,
> > > >>>>>>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to undergo some performance
> > > >> testing. I
> > > >>>>>> haven't
> > > >>>>>>>>>>>>>>>> (yet)
> > > >>>>>>>>>>>>>>>>>>>> worked
> > > >>>>>>>>>>>>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff,
> > > >> but I
> > > >>>>>> would
> > > >>>>>>>>>>>>>>>> expect
> > > >>>>>>>>>>>>>>>>>> that,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
> > > >>>> difference. The
> > > >>>>>>>>>> branch
> > > >>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>> available
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> at
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> > > >>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>> anyone
> > > >>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno
> > > >> Cadonna
> > > >>>> <
> > > >>>>>>>>>>>>>>>>>>>> cadonna@apache.org>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was
> > > >> actually
> > > >>>> also
> > > >>>>>> my
> > > >>>>>>>>>>>>>>>> point. I
> > > >>>>>>>>>>>>>>>>>>>>>>>> understood
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion
> > > >> path
> > > >>>> as
> > > >>>>>> a way
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>>> avoid
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> early
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the
> > > >>>> intent.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually
> > > >> it is
> > > >>>>>> public
> > > >>>>>>>>>>>>>>>> API. My
> > > >>>>>>>>>>>>>>>>>>>>>> question
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal
> > > >>>>>> processing.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the
> > > >> size
> > > >>>> of the
> > > >>>>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>>>> buffer
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit
> > > >> affects
> > > >>>> the
> > > >>>>>> whole
> > > >>>>>>>>>>>>>>>>>> processing
> > > >>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a
> > > >> state
> > > >>>> store
> > > >>>>>> is
> > > >>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>> confined to
> > > >>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and
> > > >>>>>> changes the
> > > >>>>>>>>>>>>>>>> behavior
> > > >>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>> other
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you
> > > >> that
> > > >>>> it
> > > >>>>>> is a
> > > >>>>>>>>>>>>>>>> decent
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compromise. I
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides
> > > >> and
> > > >>>> list
> > > >>>>>> the
> > > >>>>>>>>>>>>>>>> options to
> > > >>>>>>>>>>>>>>>>>>>>>>>> overcome
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all
> > > >>>> options
> > > >>>>>> seem
> > > >>>>>>>>>>>> quite
> > > >>>>>>>>>>>>>>>>>> heavy
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compared
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand
> > > >> what you
> > > >>>>>> mean
> > > >>>>>>>> with
> > > >>>>>>>>>>>>>>>> "less
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> predictable
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
> > > >>>> alternatives
> > > >>>>>>>> really
> > > >>>>>>>>>>>>>>>>>>>>>> interesting.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> But I
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good
> > > >> and
> > > >>>> we
> > > >>>>>> should
> > > >>>>>>>>>>>>>>>> continue
> > > >>>>>>>>>>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>>>>>>>>>> it!
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my
> > > >> e-mail on
> > > >>>>>> June
> > > >>>>>>>>>> 20th:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning
> > > >> behind
> > > >>>>>> putting
> > > >>>>>>>>>>>>>>>> isolation
> > > >>>>>>>>>>>>>>>>>>>>>> level
> > > >>>>>>>>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should
> > > >>>> that
> > > >>>>>> also
> > > >>>>>>>> be
> > > >>>>>>>>>> a
> > > >>>>>>>>>>>>>>>> way
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>> give
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to
> > > >> decide
> > > >>>>>> whether
> > > >>>>>>>> to
> > > >>>>>>>>>>>>>>>> turn on
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned
> > > >> about
> > > >>>>>> how do
> > > >>>>>>>>>> you
> > > >>>>>>>>>>>>>>>> know
> > > >>>>>>>>>>>>>>>>>>>> if a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written
> > > >> under
> > > >>>> EOS,
> > > >>>>>> if
> > > >>>>>>>> you
> > > >>>>>>>>>>>>>>>> do not
> > > >>>>>>>>>>>>>>>>>>>>>> have a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> way
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
> > > >>>> transactional or
> > > >>>>>> not.
> > > >>>>>>>>>> If
> > > >>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can
> > > >> be
> > > >>>>>> written
> > > >>>>>>>>>>>>>>>> during
> > > >>>>>>>>>>>>>>>>>>>> normal
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state
> > > >> store
> > > >>>> is not
> > > >>>>>>>>>>>>>>>>>> transactional,
> > > >>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written
> > > >> under
> > > >>>> EOS.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My point was about not only
> > > >> considering the
> > > >>>>>> bytes
> > > >>>>>>>> in
> > > >>>>>>>>>>>>>>>> memory
> > > >>>>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>>> config
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but
> > > >> also
> > > >>>>>> bytes
> > > >>>>>>>> that
> > > >>>>>>>>>>>>>>>> might
> > > >>>>>>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> spilled
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering
> > > >>>> whether you
> > > >>>>>>>>>> should
> > > >>>>>>>>>>>>>>>>>> remove
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory
> > > >>>> bytes to
> > > >>>>>> be
> > > >>>>>>>>>> used
> > > >>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store
> > > >> records." My
> > > >>>>>>>> thinking
> > > >>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>> even
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to
> > > >>>> disk,
> > > >>>>>>>>>> limiting
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> overall
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> bytes
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it
> > > >> again
> > > >>>> and
> > > >>>>>>>>>>>>>>>> considering
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>> recent
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much
> > > >>>> sense
> > > >>>>>>>> anymore.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I like the name
> > > >>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes that
> > > >>>>>>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 8.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> > > >>>>>> implementation
> > > >>>>>>>>>>>>>>>> details) of
> > > >>>>>>>>>>>>>>>>>>>> how
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of
> > > >> changelog
> > > >>>>>>>>>>>>>>>> transactions,
> > > >>>>>>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be
> > > >>>> great.
> > > >>>>>>>> Would
> > > >>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>> great
> > > >>>>>>>>>>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the
> > > >>>>>> behavior in
> > > >>>>>>>>>>>>>>>> case of
> > > >>>>>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> failure.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional
> > > >> state
> > > >>>>>> store
> > > >>>>>>>>>>>>>>>> recover
> > > >>>>>>>>>>>>>>>>>>>> after a
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
> > > >>>> transaction
> > > >>>>>>>> buffer,
> > > >>>>>>>>>>>>>>>> etc.
> > > >>>>>>>>>>>>>>>>>>>> (that
> > > >>>>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> what
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> > > >>>>>>>>>>>>>>>> WriteBatchWithIndex
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to
> > > >>>> exceed)
> > > >>>>>>>>>>>>>>>> configured
> > > >>>>>>>>>>>>>>>>>>>> memory
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needs to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
> > > >>>> Ultimately,
> > > >>>>>> any
> > > >>>>>>>>>>>>>>>> approach
> > > >>>>>>>>>>>>>>>>>>>> based
> > > >>>>>>>>>>>>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cracking
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in
> > > >>>> ways
> > > >>>>>> it's
> > > >>>>>>>> not
> > > >>>>>>>>>>>>>>>> really
> > > >>>>>>>>>>>>>>>>>>>>>>>> designed
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for is
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen
> > > >> performance
> > > >>>> or
> > > >>>>>>>>>>>>>>>> consistency
> > > >>>>>>>>>>>>>>>>>>>> issues.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing
> > > >> these
> > > >>>>>> early
> > > >>>>>>>>>>>>>>>> commits?
> > > >>>>>>>>>>>>>>>>>>>> While
> > > >>>>>>>>>>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to
> > > >>>> ensure
> > > >>>>>>>>>>>>>>>> consistency
> > > >>>>>>>>>>>>>>>>>>>> whilst
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maintaining
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem
> > > >> *very*
> > > >>>>>>>>>>>>>>>> complicated, and
> > > >>>>>>>>>>>>>>>>>>>> might
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for
> > > >> users
> > > >>>> as a
> > > >>>>>>>>>>>>>>>> consequence.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of
> > > >> this
> > > >>>>>> KIP is
> > > >>>>>>>>>>>>>>>> growing a
> > > >>>>>>>>>>>>>>>>>>>> bit
> > > >>>>>>>>>>>>>>>>>>>>>>>> out
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss
> > > >> ideas
> > > >>>> for
> > > >>>>>>>> future
> > > >>>>>>>>>>>>>>>>>>>>>>>> improvements, I
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > > >>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope
> > > >> down
> > > >>>> to a
> > > >>>>>>>> design
> > > >>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>> achieves
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> most
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
> > > >>>>>> restorations
> > > >>>>>>>>>>>>>>>> during
> > > >>>>>>>>>>>>>>>>>> dirty
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design
> > > >> that
> > > >>>>>> this KIP
> > > >>>>>>>>>>>>>>>> produces
> > > >>>>>>>>>>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ultimately
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially
> > > >> if
> > > >>>> the
> > > >>>>>> bulk
> > > >>>>>>>> of
> > > >>>>>>>>>>>>>>>> it is
> > > >>>>>>>>>>>>>>>>>>>>>> internal
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week
> > > >>>> trying
> > > >>>>>> to
> > > >>>>>>>>>>>>>>>> re-work
> > > >>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove
> > > >> the
> > > >>>>>>>>>>>>>>>> newTransaction()
> > > >>>>>>>>>>>>>>>>>>>>>> method,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> such
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
> > > >>>>>>>> RocksDBStore.
> > > >>>>>>>>>>>>>>>> That
> > > >>>>>>>>>>>>>>>>>>>> way, if
> > > >>>>>>>>>>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the
> > > >> future,
> > > >>>>>> like
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>>> SST
> > > >>>>>>>>>>>>>>>>>> file
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>> management
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so
> > > >> with
> > > >>>>>> little/no
> > > >>>>>>>>>>>> API
> > > >>>>>>>>>>>>>>>>>>>> changes.
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>
> > > >>
> > > >
> >
>

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

Posted by Colt McNealy <co...@littlehorse.io>.
Guozhang—I agree, I am in favor of moving forward with the KIP now that the
Transactional State Stores will be behind a feature flag.

Nick—I just did a bit more light testing of your branch `KIP-892-3.5.0`
with your most recent changes. I couldn't detect a performance difference
versus trunk (in the past there was a slight degradation of performance on
the restoration path, but that has been fixed). I don't believe that your
branch has the state updater thread enabled, so I didn't test that path too
heavily.

As expected, however, our internal correctness tests failed due to the IQ
read-your-own-writes issue we discussed previously. The community as a
whole would vastly benefit from this KIP getting over the finish line in
3.7.0, and so long as it is behind a feature flag so that we at LittleHorse
can still guarantee RYOW for our users, I think it's purely a win 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 <gu...@gmail.com>
wrote:

> I'd agree with you guys that as long as we are in agreement about the
> configuration semantics, that would be a big win to move forward for
> this KIP. As for the TaskCorruptedException handling like wiping state
> stores, we can discuss that in the PR rather than in the KIP.
>
> Just to clarify, I'm onboard with the latest proposal, and probably we
> can move on for voting on this KIP now?
>
> Guozhang
>
> On Thu, Oct 19, 2023 at 5:33 AM Bruno Cadonna <ca...@apache.org> wrote:
> >
> > Hi Nick,
> >
> > What you and Lucas wrote about the different configurations of ALOS/EOS
> > and READ_COMMITTED/READ_UNCOMMITTED make sense to me. My earlier
> > concerns about changelogs diverging from the content of the local state
> > stores turned out to not apply. So I think, we can move on with those
> > configurations.
> >
> > Regarding the TaskCorruptedException and wiping out the state stores
> > under EOS, couldn't we abort the transaction on the state store and
> > close the task dirty? If the Kafka transaction was indeed committed, the
> > store would restore the missing part from the changelog topic. If the
> > Kafka transaction was not committed, changelog topic and state store are
> > in-sync.
> >
> > In any case, IMO those are implementation details that we do not need to
> > discuss and solve in the KIP discussion. We can solve them on the PR.
> > The important thing is that the processing guarantees hold.
> >
> > Best,
> > Bruno
> >
> > On 10/18/23 3:56 PM, Nick Telford wrote:
> > > Hi Lucas,
> > >
> > > TaskCorruptedException is how Streams signals that the Task state
> needs to
> > > be wiped, so we can't retain that exception without also wiping state
> on
> > > timeouts.
> > >
> > > Regards,
> > > Nick
> > >
> > > On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <lbrutschy@confluent.io
> .invalid>
> > > wrote:
> > >
> > >> Hi Nick,
> > >>
> > >> I think indeed the better behavior would be to retry commitTransaction
> > >> until we risk running out of time to meet `max.poll.interval.ms`.
> > >>
> > >> However, if it's handled as a `TaskCorruptedException` at the moment,
> > >> I would do the same in this KIP, and leave exception handling
> > >> improvements to future work. 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 <ni...@gmail.com>
> > >> wrote:
> > >>>
> > >>> Hi Lucas,
> > >>>
> > >>> Yeah, this is pretty much the direction I'm thinking of going in
> now. You
> > >>> make an interesting point about committing on-error under
> > >>> ALOS/READ_COMMITTED, although I haven't had a chance to think
> through the
> > >>> implications yet.
> > >>>
> > >>> Something that I ran into earlier this week is an issue with the new
> > >>> handling of TimeoutException. Without TX stores, TimeoutException
> under
> > >> EOS
> > >>> throws a TaskCorruptedException, which wipes the stores. However,
> with TX
> > >>> stores, TimeoutException is now just bubbled up and dealt with as it
> is
> > >>> under ALOS. The problem arises when the Producer#commitTransaction
> call
> > >>> times out: Streams attempts to ignore the error and continue
> producing,
> > >>> which causes the next call to Producer#send to throw
> > >>> "IllegalStateException: Cannot attempt operation `send` because the
> > >>> previous call to `commitTransaction` timed out and must be retried".
> > >>>
> > >>> I'm not sure what we should do here: retrying the commitTransaction
> seems
> > >>> logical, but what if it times out again? Where do we draw the line
> and
> > >>> shutdown the instance?
> > >>>
> > >>> Regards,
> > >>> Nick
> > >>>
> > >>> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <lbrutschy@confluent.io
> > >> .invalid>
> > >>> wrote:
> > >>>
> > >>>> Hi all,
> > >>>>
> > >>>> I think I liked your suggestion of allowing EOS with
> READ_UNCOMMITTED,
> > >>>> but keep wiping the state on error, and I'd vote for this solution
> > >>>> when introducing `default.state.isolation.level`. This way, we'd
> have
> > >>>> the most low-risk roll-out of this feature (no behavior change
> without
> > >>>> reconfiguration), with the possibility of switching to the most
> sane /
> > >>>> battle-tested default settings in 4.0. Essentially, we'd have a
> > >>>> feature flag but call it `default.state.isolation.level` and don't
> > >>>> have to deprecate it later.
> > >>>>
> > >>>> So the possible configurations would then be this:
> > >>>>
> > >>>> 1. ALOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB,
> IQ
> > >>>> reads from DB.
> > >>>> 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> > >>>> WriteBatch/DB. Flush on error (see note below).
> > >>>> 3. EOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
> > >>>> reads from DB. Wipe state on error.
> > >>>> 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> > >>>> WriteBatch/DB.
> > >>>>
> > >>>> I believe the feature is important enough that we will see good
> > >>>> adoption even without changing the default. In 4.0, when we have
> seen
> > >>>> this being adopted and is battle-tested, we make READ_COMMITTED the
> > >>>> default for EOS, or even READ_COMITTED always the default, depending
> > >>>> on our experiences. And we could add a clever implementation of
> > >>>> READ_UNCOMITTED with WriteBatches later.
> > >>>>
> > >>>> The only smell here is that `default.state.isolation.level` wouldn't
> > >>>> be purely an IQ setting, but it would also (slightly) change the
> > >>>> behavior of the processing, but that seems unavoidable as long as we
> > >>>> haven't solve READ_UNCOMITTED IQ with WriteBatches.
> > >>>>
> > >>>> Minor: As for Bruno's point 4, I think if we are concerned about
> this
> > >>>> 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).
> > >>>>
> > >>>> Cheers,
> > >>>> Lucas
> > >>>>
> > >>>> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <
> nick.telford@gmail.com>
> > >>>> wrote:
> > >>>>>
> > >>>>> Hi Guozhang,
> > >>>>>
> > >>>>> The KIP as it stands introduces a new configuration,
> > >>>>> default.state.isolation.level, which is independent of
> > >> processing.mode.
> > >>>>> It's intended that this new configuration be used to configure a
> > >> global
> > >>>> IQ
> > >>>>> isolation level in the short term, with a future KIP introducing
> the
> > >>>>> capability to change the isolation level on a per-query basis,
> > >> falling
> > >>>> back
> > >>>>> to the "default" defined by this config. That's why I called it
> > >>>> "default",
> > >>>>> for future-proofing.
> > >>>>>
> > >>>>> However, it currently includes the caveat that READ_UNCOMMITTED is
> > >> not
> > >>>>> available under EOS. I think this is the coupling you are alluding
> > >> to?
> > >>>>>
> > >>>>> This isn't intended to be a restriction of the API, but is
> currently
> > >> a
> > >>>>> technical limitation. However, after discussing with some users
> about
> > >>>>> use-cases that would require READ_UNCOMMITTED under EOS, I'm
> > >> inclined to
> > >>>>> remove that clause and put in the necessary work to make that
> > >> combination
> > >>>>> possible now.
> > >>>>>
> > >>>>> I currently see two possible approaches:
> > >>>>>
> > >>>>>     1. Disable TX StateStores internally when the IsolationLevel is
> > >>>>>     READ_UNCOMMITTED and the processing.mode is EOS. This is more
> > >>>> difficult
> > >>>>>     than it sounds, as there are many assumptions being made
> > >> throughout
> > >>>> the
> > >>>>>     internals about the guarantees StateStores provide. It would
> > >>>> definitely add
> > >>>>>     a lot of extra "if (read_uncommitted && eos)" branches,
> > >> complicating
> > >>>>>     maintenance and testing.
> > >>>>>     2. Invest the time *now* to make READ_UNCOMMITTED of EOS
> > >> StateStores
> > >>>>>     possible. I have some ideas on how this could be achieved, but
> > >> they
> > >>>> would
> > >>>>>     need testing and could introduce some additional issues. The
> > >> benefit
> > >>>> of
> > >>>>>     this approach is that it would make query-time IsolationLevels
> > >> much
> > >>>> simpler
> > >>>>>     to implement in the future.
> > >>>>>
> > >>>>> Unfortunately, both will require considerable work that will
> further
> > >>>> delay
> > >>>>> this KIP, which was the reason I placed the restriction in the KIP
> > >> in the
> > >>>>> first place.
> > >>>>>
> > >>>>> Regards,
> > >>>>> Nick
> > >>>>>
> > >>>>> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
> > >> guozhang.wang.us@gmail.com>
> > >>>>> wrote:
> > >>>>>
> > >>>>>> Hello Nick,
> > >>>>>>
> > >>>>>> First of all, thanks a lot for the great effort you've put in
> > >> driving
> > >>>>>> this KIP! I really like it coming through finally, as many people
> > >> in
> > >>>>>> the community have raised this. At the same time I honestly feel a
> > >> bit
> > >>>>>> ashamed for not putting enough of my time supporting it and
> > >> pushing it
> > >>>>>> through the finish line (you raised this KIP almost a year ago).
> > >>>>>>
> > >>>>>> I briefly passed through the DISCUSS thread so far, not sure I've
> > >> 100
> > >>>>>> percent digested all the bullet points. But with the goal of
> > >> trying to
> > >>>>>> help take it through the finish line in mind, I'd want to throw
> > >>>>>> thoughts on top of my head only on the point #4 above which I felt
> > >> may
> > >>>>>> be the main hurdle for the current KIP to drive to a consensus
> now.
> > >>>>>>
> > >>>>>> The general question I asked myself is, whether we want to couple
> > >> "IQ
> > >>>>>> reading mode" with "processing mode". While technically I tend to
> > >>>>>> agree with you that, it's feels like a bug if some single user
> > >> chose
> > >>>>>> "EOS" for processing mode while choosing "read uncommitted" for IQ
> > >>>>>> reading mode, at the same time, I'm thinking if it's possible that
> > >>>>>> there could be two different persons (or even two teams) that
> > >> would be
> > >>>>>> using the stream API to build the app, and the IQ API to query the
> > >>>>>> running state of the app. I know this is less of a technical thing
> > >> but
> > >>>>>> rather a more design stuff, but if it could be ever the case, I'm
> > >>>>>> wondering if the personale using the IQ API knows about the risks
> > >> of
> > >>>>>> using read uncommitted but still chose so for the favor of
> > >>>>>> performance, no matter if the underlying stream processing mode
> > >>>>>> configured by another personale is EOS or not. In that regard, I'm
> > >>>>>> leaning towards a "leaving the door open, and close it later if we
> > >>>>>> found it's a bad idea" aspect with a configuration 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 can still enforce
> > >>>>>> coupling them.
> > >>>>>>
> > >>>>>> Again, just my 2c here. Thanks again for the great patience and
> > >>>>>> diligence on this KIP.
> > >>>>>>
> > >>>>>>
> > >>>>>> Guozhang
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
> > >> nick.telford@gmail.com>
> > >>>>>> wrote:
> > >>>>>>>
> > >>>>>>> Hi Bruno,
> > >>>>>>>
> > >>>>>>> 4.
> > >>>>>>> I'll hold off on making that change until we have a consensus as
> > >> to
> > >>>> what
> > >>>>>>> configuration to use to control all of this, as it'll be
> > >> affected by
> > >>>> the
> > >>>>>>> decision on EOS isolation levels.
> > >>>>>>>
> > >>>>>>> 5.
> > >>>>>>> Done. I've chosen "committedOffsets".
> > >>>>>>>
> > >>>>>>> Regards,
> > >>>>>>> Nick
> > >>>>>>>
> > >>>>>>> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <ca...@apache.org>
> > >>>> wrote:
> > >>>>>>>
> > >>>>>>>> Hi Nick,
> > >>>>>>>>
> > >>>>>>>> 1.
> > >>>>>>>> Yeah, you are probably right that it does not make too much
> > >> sense.
> > >>>>>>>> Thanks for the clarification!
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> 4.
> > >>>>>>>> Yes, sorry for the back and forth, but I think for the sake of
> > >> the
> > >>>> KIP
> > >>>>>>>> it is better to let the ALOS behavior as it is for now due to
> > >> the
> > >>>>>>>> possible issues you would run into. Maybe we can find a
> > >> solution
> > >>>> in the
> > >>>>>>>> future. Now the question returns to whether we really need
> > >>>>>>>> default.state.isolation.level. Maybe the config could be the
> > >>>> feature
> > >>>>>>>> flag Sophie requested.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> 5.
> > >>>>>>>> There 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 committedOffsetFor().
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> Best,
> > >>>>>>>> Bruno
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> On 10/13/23 3:21 PM, Nick Telford wrote:
> > >>>>>>>>> Hi Bruno,
> > >>>>>>>>>
> > >>>>>>>>> Thanks for getting back to me.
> > >>>>>>>>>
> > >>>>>>>>> 1.
> > >>>>>>>>> I think this should be possible. Are you thinking of the
> > >>>> situation
> > >>>>>> where
> > >>>>>>>> a
> > >>>>>>>>> user may downgrade to a previous version of Kafka Streams? In
> > >>>> that
> > >>>>>> case,
> > >>>>>>>>> sadly, the RocksDBStore would get wiped by the older version
> > >> of
> > >>>> Kafka
> > >>>>>>>>> Streams anyway, because that version wouldn't understand the
> > >>>> extra
> > >>>>>> column
> > >>>>>>>>> family (that holds offsets), so the missing Position file
> > >> would
> > >>>>>>>>> automatically get rebuilt when the store is rebuilt from the
> > >>>>>> changelog.
> > >>>>>>>>> Are there other situations than downgrade where a
> > >> transactional
> > >>>> store
> > >>>>>>>> could
> > >>>>>>>>> be replaced by a non-transactional one? I can't think of any.
> > >>>>>>>>>
> > >>>>>>>>> 2.
> > >>>>>>>>> Ahh yes, the Test Plan - my Kryptonite! This section
> > >> definitely
> > >>>>>> needs to
> > >>>>>>>> be
> > >>>>>>>>> fleshed out. I'll work on that. How much detail do you need?
> > >>>>>>>>>
> > >>>>>>>>> 3.
> > >>>>>>>>> See my previous email discussing this.
> > >>>>>>>>>
> > >>>>>>>>> 4.
> > >>>>>>>>> Hmm, this is an interesting point. Are you suggesting that
> > >> under
> > >>>> ALOS
> > >>>>>>>>> READ_COMMITTED should not be supported?
> > >>>>>>>>>
> > >>>>>>>>> Regards,
> > >>>>>>>>> Nick
> > >>>>>>>>>
> > >>>>>>>>> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
> > >> cadonna@apache.org>
> > >>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> Hi Nick,
> > >>>>>>>>>>
> > >>>>>>>>>> I think the KIP is converging!
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> 1.
> > >>>>>>>>>> I am wondering whether it makes sense to write the position
> > >> file
> > >>>>>> during
> > >>>>>>>>>> close as we do for the checkpoint file, so that in case the
> > >>>> state
> > >>>>>> store
> > >>>>>>>>>> is replaced with a non-transactional state store the
> > >>>>>> non-transactional
> > >>>>>>>>>> state store finds the position file. I think, this is not
> > >>>> strictly
> > >>>>>>>>>> needed, but would be a nice behavior instead of just
> > >> deleting
> > >>>> the
> > >>>>>>>>>> position file.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> 2.
> > >>>>>>>>>> The test plan does not mention integration tests. Do you not
> > >>>> need to
> > >>>>>>>>>> extend existing ones and add new ones. Also for upgrading
> > >> and
> > >>>>>>>>>> downgrading you might need integration and/or system tests.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> 3.
> > >>>>>>>>>> I think Sophie made a point. Although, IQ reading from
> > >>>> uncommitted
> > >>>>>> data
> > >>>>>>>>>> under EOS might be considered a bug by some people. Thus,
> > >> your
> > >>>> KIP
> > >>>>>> would
> > >>>>>>>>>> fix a bug rather than changing the intended behavior.
> > >> However, I
> > >>>>>> also
> > >>>>>>>>>> see that a feature flag would help users that rely on this
> > >> buggy
> > >>>>>>>>>> behavior (at least until AK 4.0).
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> 4.
> > >>>>>>>>>> This is related to the previous point. I assume that the
> > >>>> difference
> > >>>>>>>>>> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is
> > >> that in
> > >>>> the
> > >>>>>>>>>> former you enable transactions on the state store and in the
> > >>>> latter
> > >>>>>> you
> > >>>>>>>>>> disable them. If my assumption is correct, I think that is
> > >> an
> > >>>> issue.
> > >>>>>>>>>> Let's assume under ALOS Streams fails over a couple of times
> > >>>> more or
> > >>>>>>>>>> less at the same step in processing after value 3 is added
> > >> to an
> > >>>>>>>>>> aggregation but the offset of the corresponding input record
> > >>>> was not
> > >>>>>>>>>> committed. Without transactions disabled, the aggregation
> > >> value
> > >>>>>> would
> > >>>>>>>>>> increase by 3 for each failover. With transactions enabled,
> > >>>> value 3
> > >>>>>>>>>> would only be added to the aggregation once when the offset
> > >> of
> > >>>> the
> > >>>>>> input
> > >>>>>>>>>> record is committed and the transaction finally completes.
> > >> So
> > >>>> the
> > >>>>>>>>>> content of the state store would change depending on the
> > >>>>>> configuration
> > >>>>>>>>>> for IQ. IMO, the content of the state store should be
> > >>>> independent
> > >>>>>> from
> > >>>>>>>>>> IQ. Given this issue, I propose to not use transactions with
> > >>>> ALOS at
> > >>>>>>>>>> all. I was a big proponent of using transactions with ALOS,
> > >> but
> > >>>> I
> > >>>>>>>>>> realized that transactions with ALOS is not as easy as
> > >> enabling
> > >>>>>>>>>> transactions on state stores. Another aspect that is
> > >>>> problematic is
> > >>>>>> that
> > >>>>>>>>>> the changelog topic which actually replicates the state
> > >> store
> > >>>> is not
> > >>>>>>>>>> transactional under ALOS. Thus, it might happen that the
> > >> state
> > >>>>>> store and
> > >>>>>>>>>> the changelog differ in their content. All of this is maybe
> > >>>> solvable
> > >>>>>>>>>> somehow, but for the sake of this KIP, I would leave it for
> > >> the
> > >>>>>> future.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> Best,
> > >>>>>>>>>> Bruno
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > >>>>>>>>>>> Hey Nick! First of all thanks for taking up this awesome
> > >>>> feature,
> > >>>>>> I'm
> > >>>>>>>>>> sure
> > >>>>>>>>>>> every single
> > >>>>>>>>>>> Kafka Streams user and dev would agree that it is sorely
> > >>>> needed.
> > >>>>>>>>>>>
> > >>>>>>>>>>> I've just been catching up on the KIP and surrounding
> > >>>> discussion,
> > >>>>>> so
> > >>>>>>>>>> please
> > >>>>>>>>>>> forgive me
> > >>>>>>>>>>> for any misunderstandings or misinterpretations of the
> > >> current
> > >>>>>> plan and
> > >>>>>>>>>>> don't hesitate to
> > >>>>>>>>>>> correct me.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Before I jump in, I just want to say that having seen this
> > >>>> drag on
> > >>>>>> for
> > >>>>>>>> so
> > >>>>>>>>>>> long, my singular
> > >>>>>>>>>>> goal in responding is to help this KIP past a perceived
> > >>>> impasse so
> > >>>>>> we
> > >>>>>>>> can
> > >>>>>>>>>>> finally move on
> > >>>>>>>>>>> to voting and implementing it. Long discussions are to be
> > >>>> expected
> > >>>>>> for
> > >>>>>>>>>>> major features like
> > >>>>>>>>>>> this but it's completely on us as the Streams devs to make
> > >> sure
> > >>>>>> there
> > >>>>>>>> is
> > >>>>>>>>>> an
> > >>>>>>>>>>> end in sight
> > >>>>>>>>>>> for any ongoing discussion.
> > >>>>>>>>>>>
> > >>>>>>>>>>> With that said, it's my understanding that the KIP as
> > >> currently
> > >>>>>>>> proposed
> > >>>>>>>>>> is
> > >>>>>>>>>>> just not tenable
> > >>>>>>>>>>> for Kafka Streams, and would prevent some EOS users from
> > >>>> upgrading
> > >>>>>> to
> > >>>>>>>> the
> > >>>>>>>>>>> version it
> > >>>>>>>>>>> first appears in. Given that we can't predict or guarantee
> > >>>> whether
> > >>>>>> any
> > >>>>>>>> of
> > >>>>>>>>>>> the followup KIPs
> > >>>>>>>>>>> would be completed in the same release cycle as this one,
> > >> we
> > >>>> need
> > >>>>>> to
> > >>>>>>>> make
> > >>>>>>>>>>> sure that the
> > >>>>>>>>>>> feature is either compatible with all current users or else
> > >>>>>>>>>> feature-flagged
> > >>>>>>>>>>> so that they may
> > >>>>>>>>>>> opt in/out.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Therefore, IIUC we need to have either (or both) of these
> > >> as
> > >>>>>>>>>>> fully-implemented config options:
> > >>>>>>>>>>> 1. default.state.isolation.level
> > >>>>>>>>>>> 2. enable.transactional.state.stores
> > >>>>>>>>>>>
> > >>>>>>>>>>> This way EOS users for whom read_committed semantics are
> > >> not
> > >>>>>> viable can
> > >>>>>>>>>>> still upgrade,
> > >>>>>>>>>>> and either use the isolation.level config to leverage the
> > >> new
> > >>>> txn
> > >>>>>> state
> > >>>>>>>>>>> stores without sacrificing
> > >>>>>>>>>>> their application semantics, or else simply keep the
> > >>>> transactional
> > >>>>>>>> state
> > >>>>>>>>>>> stores disabled until we
> > >>>>>>>>>>> are able to fully implement the isolation level
> > >> configuration
> > >>>> at
> > >>>>>> either
> > >>>>>>>>>> an
> > >>>>>>>>>>> application or query level.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Frankly you are the expert here and know much more about
> > >> the
> > >>>>>> tradeoffs
> > >>>>>>>> in
> > >>>>>>>>>>> both semantics and
> > >>>>>>>>>>> effort level of implementing one of these configs vs the
> > >>>> other. In
> > >>>>>> my
> > >>>>>>>>>>> opinion, either option would
> > >>>>>>>>>>> be fine and I would leave the decision of which one to
> > >> include
> > >>>> in
> > >>>>>> this
> > >>>>>>>>>> KIP
> > >>>>>>>>>>> completely up to you.
> > >>>>>>>>>>> I just don't see a way for the KIP to proceed without some
> > >>>>>> variation of
> > >>>>>>>>>> the
> > >>>>>>>>>>> above that would allow
> > >>>>>>>>>>> EOS users to opt-out of read_committed.
> > >>>>>>>>>>>
> > >>>>>>>>>>> (If it's all the same to you, I would recommend 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 that gets released.
> > >>>>>>>>>>> So personally, my choice would be to add the feature flag
> > >> and
> > >>>>>> leave it
> > >>>>>>>>>> off
> > >>>>>>>>>>> by default. If all 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
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > >>>>>> nick.telford@gmail.com>
> > >>>>>>>>>> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>>> Hi Bruno,
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Agreed, I can live with that for now.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> In an effort to keep the scope of this KIP from
> > >> expanding, I'm
> > >>>>>> leaning
> > >>>>>>>>>>>> towards just providing a configurable
> > >>>>>> default.state.isolation.level
> > >>>>>>>> and
> > >>>>>>>>>>>> removing IsolationLevel from the StateStoreContext. This
> > >>>> would be
> > >>>>>>>>>>>> compatible with adding support for query-time
> > >> IsolationLevels
> > >>>> in
> > >>>>>> the
> > >>>>>>>>>>>> future, whilst providing a way for users to select an
> > >>>> isolation
> > >>>>>> level
> > >>>>>>>>>> now.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> The big problem with this, however, is that if a user
> > >> selects
> > >>>>>>>>>>>> processing.mode
> > >>>>>>>>>>>> = "exactly-once(-v2|-beta)", and
> > >>>> default.state.isolation.level =
> > >>>>>>>>>>>> "READ_UNCOMMITTED", we need to guarantee that the data
> > >> isn't
> > >>>>>> written
> > >>>>>>>> to
> > >>>>>>>>>>>> disk until commit() is called, but we also need to permit
> > >> IQ
> > >>>>>> threads
> > >>>>>>>> to
> > >>>>>>>>>>>> read from the ongoing transaction.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> A simple solution would be to (temporarily) forbid this
> > >>>>>> combination of
> > >>>>>>>>>>>> configuration, and have default.state.isolation.level
> > >>>>>> automatically
> > >>>>>>>>>> switch
> > >>>>>>>>>>>> to READ_COMMITTED when processing.mode is anything other
> > >> than
> > >>>>>>>>>>>> at-least-once. Do you think this would be acceptable?
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> In a later KIP, we can add support for query-time
> > >> isolation
> > >>>>>> levels and
> > >>>>>>>>>>>> solve this particular problem there, which would relax
> > >> this
> > >>>>>>>> restriction.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Regards,
> > >>>>>>>>>>>> Nick
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> > >>>> cadonna@apache.org>
> > >>>>>>>>>> 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,
> > >>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > >>>>>>>>>>>>>> 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 <
> > >>>>>> nick.telford@gmail.com
> > >>>>>>>>>
> > >>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Hi everyone,
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> I agree that having IsolationLevel be determined at
> > >>>> query-time
> > >>>>>> is
> > >>>>>>>> the
> > >>>>>>>>>>>>>>> ideal design, but there are a few sticking points:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>> There needs to be some way to communicate the
> > >>>> IsolationLevel
> > >>>>>> down
> > >>>>>>>> to
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>> RocksDBStore itself, so that the query can respect it.
> > >>>> Since
> > >>>>>> stores
> > >>>>>>>>>>>> are
> > >>>>>>>>>>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
> > >>>>>> MeteredStore,
> > >>>>>>>>>>>>> etc.),
> > >>>>>>>>>>>>>>> we need some way to deliver that information to the
> > >> bottom
> > >>>>>> layer.
> > >>>>>>>> For
> > >>>>>>>>>>>>> IQv2,
> > >>>>>>>>>>>>>>> we can use the existing State#query() method, but IQv1
> > >> has
> > >>>> no
> > >>>>>> way
> > >>>>>>>> to
> > >>>>>>>>>>>> do
> > >>>>>>>>>>>>>>> this.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> A simple approach, which would potentially open up
> > >> other
> > >>>>>> options,
> > >>>>>>>>>>>> would
> > >>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> > >>>>>>>>>>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > >>>>>>>> ReadOnlyKeyValueStore
> > >>>>>>>>>>>>> (and
> > >>>>>>>>>>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore,
> > >>>> etc.).
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>> As mentioned above, RocksDB WriteBatches are not
> > >>>> thread-safe,
> > >>>>>> which
> > >>>>>>>>>>>>> causes
> > >>>>>>>>>>>>>>> a problem if we want to provide READ_UNCOMMITTED
> > >>>> Iterators. I
> > >>>>>> also
> > >>>>>>>>>>>> had a
> > >>>>>>>>>>>>>>> look at RocksDB Transactions[1], but they solve a very
> > >>>>>> different
> > >>>>>>>>>>>>> problem,
> > >>>>>>>>>>>>>>> and have the same thread-safety issue.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> One possible approach that I mentioned is chaining
> > >>>>>> WriteBatches:
> > >>>>>>>>>> every
> > >>>>>>>>>>>>>>> time a new Interactive Query is received (i.e.
> > >>>> readOnlyView,
> > >>>>>> see
> > >>>>>>>>>>>> above,
> > >>>>>>>>>>>>>>> is called) we "freeze" the existing WriteBatch, and
> > >> start a
> > >>>>>> new one
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>> writes. The Interactive Query queries the "chain" of
> > >>>> previous
> > >>>>>>>>>>>>> WriteBatches
> > >>>>>>>>>>>>>>> + the underlying database; while the StreamThread
> > >> starts
> > >>>>>> writing to
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>> *new* WriteBatch. On-commit, the StreamThread would
> > >> write
> > >>>> *all*
> > >>>>>>>>>>>>>>> WriteBatches in the chain to the database (that have
> > >> not
> > >>>> yet
> > >>>>>> been
> > >>>>>>>>>>>>> written).
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> WriteBatches would be closed/freed only when they have
> > >> been
> > >>>>>> both
> > >>>>>>>>>>>>>>> committed, and all open Interactive Queries on them
> > >> have
> > >>>> been
> > >>>>>>>> closed.
> > >>>>>>>>>>>>> This
> > >>>>>>>>>>>>>>> would require some reference counting.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Obviously a drawback of this approach is the potential
> > >> for
> > >>>>>>>> increased
> > >>>>>>>>>>>>>>> memory usage: if an Interactive Query is long-lived,
> > >> for
> > >>>>>> example by
> > >>>>>>>>>>>>> doing a
> > >>>>>>>>>>>>>>> full scan over a large database, or even just pausing
> > >> in
> > >>>> the
> > >>>>>> middle
> > >>>>>>>>>> of
> > >>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>> iteration, then the existing chain of WriteBatches
> > >> could be
> > >>>>>> kept
> > >>>>>>>>>>>> around
> > >>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>> a long time, potentially forever.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> A.
> > >>>>>>>>>>>>>>> Going off on a tangent, it looks like in addition to
> > >>>> supporting
> > >>>>>>>>>>>>>>> READ_COMMITTED queries, we could go further and support
> > >>>>>>>>>>>> REPEATABLE_READ
> > >>>>>>>>>>>>>>> queries (i.e. where subsequent reads to the same key
> > >> in the
> > >>>>>> same
> > >>>>>>>>>>>>>>> Interactive Query are guaranteed to yield the same
> > >> value)
> > >>>> by
> > >>>>>> making
> > >>>>>>>>>>>> use
> > >>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so
> > >> the
> > >>>>>>>>>> performance
> > >>>>>>>>>>>>>>> impact is likely to be negligible, but they do require
> > >>>> that the
> > >>>>>>>>>>>>> Interactive
> > >>>>>>>>>>>>>>> Query session can be explicitly closed.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> This could be achieved if we made the above
> > >> readOnlyView
> > >>>>>> interface
> > >>>>>>>>>>>> look
> > >>>>>>>>>>>>>>> more like:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> > >>>>>>>>>>>> ReadOnlyKeyValueStore<K,
> > >>>>>>>>>>>>>>> V>, AutoCloseable {}
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > >>>>>>>>>>>>>>>         ...
> > >>>>>>>>>>>>>>>         ReadOnlyKeyValueView<K, V>
> > >>>> readOnlyView(IsolationLevel
> > >>>>>>>>>>>>> isolationLevel);
> > >>>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> But this would be a breaking change, as existing IQv1
> > >>>> queries
> > >>>>>> are
> > >>>>>>>>>>>>>>> guaranteed to never call store.close(), and therefore
> > >> these
> > >>>>>> would
> > >>>>>>>>>> leak
> > >>>>>>>>>>>>>>> memory under REPEATABLE_READ.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> B.
> > >>>>>>>>>>>>>>> One thing that's notable: MyRocks states that they
> > >> support
> > >>>>>>>>>>>>> READ_COMMITTED
> > >>>>>>>>>>>>>>> and REPEATABLE_READ, but they make no mention of
> > >>>>>>>>>>>> READ_UNCOMMITTED[3][4].
> > >>>>>>>>>>>>>>> This could be because doing so is technically
> > >>>>>> difficult/impossible
> > >>>>>>>>>>>> using
> > >>>>>>>>>>>>>>> the primitives available in RocksDB.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Lucas, to address your points:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> U1.
> > >>>>>>>>>>>>>>> It's only "SHOULD" to permit alternative (i.e.
> > >> non-RocksDB)
> > >>>>>>>>>>>>>>> implementations of StateStore that do not support
> > >> atomic
> > >>>>>> writes.
> > >>>>>>>>>>>>> Obviously
> > >>>>>>>>>>>>>>> in those cases, the guarantees Kafka Streams
> > >>>> provides/expects
> > >>>>>> would
> > >>>>>>>>>> be
> > >>>>>>>>>>>>>>> relaxed. Do you think we should require all
> > >>>> implementations to
> > >>>>>>>>>> support
> > >>>>>>>>>>>>>>> atomic writes?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> U2.
> > >>>>>>>>>>>>>>> Stores can support multiple IsolationLevels. As we've
> > >>>> discussed
> > >>>>>>>>>> above,
> > >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>> ideal scenario would be to specify the IsolationLevel
> > >> at
> > >>>>>>>> query-time.
> > >>>>>>>>>>>>>>> Failing that, I think the second-best approach is to
> > >>>> define the
> > >>>>>>>>>>>>>>> IsolationLevel for *all* queries based on the
> > >>>> processing.mode,
> > >>>>>>>> which
> > >>>>>>>>>>>> is
> > >>>>>>>>>>>>>>> what the default StateStoreContext#isolationLevel()
> > >>>> achieves.
> > >>>>>> Would
> > >>>>>>>>>>>> you
> > >>>>>>>>>>>>>>> prefer an alternative?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> While the existing implementation is equivalent to
> > >>>>>>>> READ_UNCOMMITTED,
> > >>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>> can yield unexpected results/errors under EOS, if a
> > >>>>>> transaction is
> > >>>>>>>>>>>>> rolled
> > >>>>>>>>>>>>>>> back. While this would be a change in behaviour for
> > >> users,
> > >>>> it
> > >>>>>> would
> > >>>>>>>>>>>> look
> > >>>>>>>>>>>>>>> more like a bug fix than a breaking change. That said,
> > >> we
> > >>>>>> *could*
> > >>>>>>>>>> make
> > >>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>> configurable, and default to the existing behaviour
> > >>>>>>>>>> (READ_UNCOMMITTED)
> > >>>>>>>>>>>>>>> instead of inferring it from the processing.mode?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> N1, N2.
> > >>>>>>>>>>>>>>> These were only primitives to avoid boxing costs, but
> > >> since
> > >>>>>> this is
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>> performance sensitive area, it should be fine to
> > >> change if
> > >>>>>> that's
> > >>>>>>>>>>>>> desirable.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> N3.
> > >>>>>>>>>>>>>>> It's because the store "manages its own offsets", which
> > >>>>>> includes
> > >>>>>>>> both
> > >>>>>>>>>>>>>>> committing the offset, *and providing it* via
> > >>>>>> getCommittedOffset().
> > >>>>>>>>>>>>>>> Personally, I think "managesOffsets" conveys this best,
> > >>>> but I
> > >>>>>> don't
> > >>>>>>>>>>>> mind
> > >>>>>>>>>>>>>>> changing it if the nomenclature is unclear.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Sorry for the massive emails/essays!
> > >>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> 1:
> > >> https://github.com/facebook/rocksdb/wiki/Transactions
> > >>>>>>>>>>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > >>>>>>>>>>>>>>> 3:
> > >>>>>>>>
> > >> https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > >>>>>>>>>>>>>>> 4:
> > >>>> https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > >>>>>>>>>>>>>>> <lb...@confluent.io.invalid> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Hi Nick,
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> since I last read it in April, the KIP has become much
> > >>>>>> cleaner and
> > >>>>>>>>>>>>>>>> easier to read. Great work!
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> It feels to me the last big open point is whether we
> > >> can
> > >>>>>> implement
> > >>>>>>>>>>>>>>>> isolation level as a query parameter. I understand
> > >> that
> > >>>> there
> > >>>>>> are
> > >>>>>>>>>>>>>>>> implementation concerns, but as Colt says, it would
> > >> be a
> > >>>> great
> > >>>>>>>>>>>>>>>> addition, and would also simplify the migration path
> > >> for
> > >>>> this
> > >>>>>>>>>> change.
> > >>>>>>>>>>>>>>>> Is the implementation problem you mentioned caused by
> > >> the
> > >>>>>>>> WriteBatch
> > >>>>>>>>>>>>>>>> not having a notion of a snapshot, as the underlying
> > >> DB
> > >>>>>> iterator
> > >>>>>>>>>>>> does?
> > >>>>>>>>>>>>>>>> In that case, I am not sure a chain of WriteBatches
> > >> as you
> > >>>>>> propose
> > >>>>>>>>>>>>>>>> would fully solve the problem, but maybe I didn't dig
> > >>>> enough
> > >>>>>> into
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>> details to fully understand it.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> If it's not possible to implement it now, would it be
> > >> an
> > >>>>>> option to
> > >>>>>>>>>>>>>>>> make sure in this KIP that we do not fully close the
> > >> door
> > >>>> on
> > >>>>>>>>>>>> per-query
> > >>>>>>>>>>>>>>>> isolation levels in the interface, as it may be
> > >> possible
> > >>>> to
> > >>>>>>>>>> implement
> > >>>>>>>>>>>>>>>> the missing primitives in RocksDB or Speedb in the
> > >> future.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Understanding:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to
> > >> be
> > >>>>>> persisted
> > >>>>>>>>>>>>>>>> atomically with the records?
> > >>>>>>>>>>>>>>>> * U2) Don't understand the default implementation of
> > >>>>>>>>>>>> `isolationLevel`.
> > >>>>>>>>>>>>>>>> The isolation level should be a property of the
> > >> underlying
> > >>>>>> store,
> > >>>>>>>>>> and
> > >>>>>>>>>>>>>>>> not be defined by the default config? Existing stores
> > >>>> probably
> > >>>>>>>> don't
> > >>>>>>>>>>>>>>>> guarantee READ_COMMITTED, so the default should be to
> > >>>> return
> > >>>>>>>>>>>>>>>> READ_UNCOMMITTED.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Nits:
> > >>>>>>>>>>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong`
> > >>>> return
> > >>>>>> type,
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>>> avoid the `null`?
> > >>>>>>>>>>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
> > >>>>>> `OptionalLong`
> > >>>>>>>>>>>>>>>> return type, to avoid the `-1`?
> > >>>>>>>>>>>>>>>> * N3) I don't understand why `managesOffsets` uses the
> > >>>>>> 'manage'
> > >>>>>>>>>> verb,
> > >>>>>>>>>>>>>>>> whereas all other methods use the "commits" verb. I'd
> > >>>> suggest
> > >>>>>>>>>>>>>>>> `commitsOffsets`.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Either way, it feels this KIP is very close to the
> > >> finish
> > >>>>>> line,
> > >>>>>>>> I'm
> > >>>>>>>>>>>>>>>> looking forward to seeing this in production!
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>> Lucas
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> > >>>>>> colt@littlehorse.io
> > >>>>>>>>>
> > >>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Making IsolationLevel a query-time constraint,
> > >> rather
> > >>>> than
> > >>>>>>>> linking
> > >>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>> the processing.guarantee.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> As I understand it, would this allow even a user of
> > >> EOS
> > >>>> to
> > >>>>>>>> control
> > >>>>>>>>>>>>>>>> whether
> > >>>>>>>>>>>>>>>>> reading committed or uncommitted records? If so, I am
> > >>>> highly
> > >>>>>> in
> > >>>>>>>>>>>> favor
> > >>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>> this.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> I know that I was one of the early people to point
> > >> out
> > >>>> the
> > >>>>>>>> current
> > >>>>>>>>>>>>>>>>> shortcoming that IQ reads uncommitted records, but
> > >> just
> > >>>> this
> > >>>>>>>>>>>> morning I
> > >>>>>>>>>>>>>>>>> realized a pattern we use which means that (for
> > >> certain
> > >>>>>> queries)
> > >>>>>>>>>> our
> > >>>>>>>>>>>>>>>> system
> > >>>>>>>>>>>>>>>>> needs to be able to read uncommitted records, which
> > >> is
> > >>>> the
> > >>>>>>>> current
> > >>>>>>>>>>>>>>>> behavior
> > >>>>>>>>>>>>>>>>> of Kafka Streams in EOS.***
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> If IsolationLevel being a query-time decision allows
> > >> for
> > >>>>>> this,
> > >>>>>>>> then
> > >>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>> would be amazing. I would also vote that the default
> > >>>> behavior
> > >>>>>>>>>> should
> > >>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>> reading uncommitted records, because it is totally
> > >>>> possible
> > >>>>>> for a
> > >>>>>>>>>>>>> valid
> > >>>>>>>>>>>>>>>>> application to depend on that behavior, and breaking
> > >> it
> > >>>> in a
> > >>>>>>>> minor
> > >>>>>>>>>>>>>>>> release
> > >>>>>>>>>>>>>>>>> might be a bit strong.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> *** (Note, for the curious reader....) Our
> > >> use-case/query
> > >>>>>> pattern
> > >>>>>>>>>>>> is a
> > >>>>>>>>>>>>>>>> bit
> > >>>>>>>>>>>>>>>>> complex, but reading "uncommitted" records is
> > >> actually
> > >>>> safe
> > >>>>>> in
> > >>>>>>>> our
> > >>>>>>>>>>>>> case
> > >>>>>>>>>>>>>>>>> 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 soon as the processor processes
> > >> the
> > >>>>>> result.
> > >>>>>>>> If
> > >>>>>>>>>>>>> they
> > >>>>>>>>>>>>>>>>> can't read uncommitted records, they may receive a
> > >> "201 -
> > >>>>>>>> Created"
> > >>>>>>>>>>>>>>>>> response, immediately followed by a "404 - 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 <
> > >>>>>>>>>>>> nick.telford@gmail.com>
> > >>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Addendum:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I think we would also face the same problem with the
> > >>>>>> approach
> > >>>>>>>> John
> > >>>>>>>>>>>>>>>> outlined
> > >>>>>>>>>>>>>>>>>> earlier (using the record cache as a transaction
> > >> buffer
> > >>>> and
> > >>>>>>>>>>>> flushing
> > >>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>> straight to SST files). This is because the record
> > >> cache
> > >>>>>> (the
> > >>>>>>>>>>>>>>>> ThreadCache
> > >>>>>>>>>>>>>>>>>> class) is not thread-safe, so every commit would
> > >>>> invalidate
> > >>>>>> open
> > >>>>>>>>>> IQ
> > >>>>>>>>>>>>>>>>>> Iterators in the same way that RocksDB WriteBatches
> > >> do.
> > >>>>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > >>>>>>>>>>>> nick.telford@gmail.com>
> > >>>>>>>>>>>>>>>>>> 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.
> > >>>>>>>>>>>>>>>>>>> 2. Making IsolationLevel a query-time constraint,
> > >>>> rather
> > >>>>>> than
> > >>>>>>>>>>>>>>>> linking it
> > >>>>>>>>>>>>>>>>>>> to the processing.guarantee.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> There's a wrinkle that makes this a challenge:
> > >>>> Interactive
> > >>>>>>>>>> Queries
> > >>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>> open an Iterator, when using transactions and
> > >>>>>> READ_UNCOMMITTED.
> > >>>>>>>>>>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries
> > >>>> need
> > >>>>>> to be
> > >>>>>>>>>>>> able
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>> read records from the currently uncommitted
> > >> transaction
> > >>>>>> buffer
> > >>>>>>>>>>>>>>>>>>> (WriteBatch). This includes for Iterators, which
> > >> should
> > >>>>>> iterate
> > >>>>>>>>>>>>>>>> both the
> > >>>>>>>>>>>>>>>>>>> transaction buffer and underlying database (using
> > >>>>>>>>>>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> The issue is that when the StreamThread commits, it
> > >>>> writes
> > >>>>>> the
> > >>>>>>>>>>>>>>>> current
> > >>>>>>>>>>>>>>>>>>> WriteBatch to RocksDB *and then clears the
> > >> WriteBatch*.
> > >>>>>>>> Clearing
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> WriteBatch while an Interactive Query holds an open
> > >>>>>> Iterator on
> > >>>>>>>>>> it
> > >>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>> invalidate the Iterator. Worse, it turns out that
> > >>>> Iterators
> > >>>>>>>> over
> > >>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>> WriteBatch become invalidated not just when the
> > >>>> WriteBatch
> > >>>>>> is
> > >>>>>>>>>>>>>>>> cleared,
> > >>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>> also when the Iterators' current key receives a new
> > >>>> write.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Now that I'm writing this, I remember that this is
> > >> the
> > >>>>>> major
> > >>>>>>>>>>>> reason
> > >>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>> switched the original design from having a
> > >> query-time
> > >>>>>>>>>>>>>>>> IsolationLevel to
> > >>>>>>>>>>>>>>>>>>> having the IsolationLevel linked to the
> > >>>> transactionality
> > >>>>>> of the
> > >>>>>>>>>>>>>>>> stores
> > >>>>>>>>>>>>>>>>>>> themselves.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> It *might* be possible to resolve this, by having a
> > >>>>>> "chain" of
> > >>>>>>>>>>>>>>>>>>> WriteBatches, with the StreamThread switching to a
> > >> new
> > >>>>>>>> WriteBatch
> > >>>>>>>>>>>>>>>>>> whenever
> > >>>>>>>>>>>>>>>>>>> a new Interactive Query attempts to read from the
> > >>>>>> database, but
> > >>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>> could
> > >>>>>>>>>>>>>>>>>>> cause some performance problems/memory pressure
> > >> when
> > >>>>>> subjected
> > >>>>>>>> to
> > >>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>> high
> > >>>>>>>>>>>>>>>>>>> Interactive Query load. It would also reduce the
> > >>>>>> efficiency of
> > >>>>>>>>>>>>>>>>>> WriteBatches
> > >>>>>>>>>>>>>>>>>>> on-commit, as we'd have to write N WriteBatches,
> > >> where
> > >>>> N
> > >>>>>> is the
> > >>>>>>>>>>>>>>>> number of
> > >>>>>>>>>>>>>>>>>>> Interactive Queries since the last commit.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> I realise this is getting into the weeds of the
> > >>>>>> implementation,
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>> you'd
> > >>>>>>>>>>>>>>>>>>> rather we focus on the API for now, but I think
> > >> it's
> > >>>>>> important
> > >>>>>>>> to
> > >>>>>>>>>>>>>>>>>> consider
> > >>>>>>>>>>>>>>>>>>> how to implement the desired API, in case we come
> > >> up
> > >>>> with
> > >>>>>> an
> > >>>>>>>> API
> > >>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>> cannot be implemented efficiently, or even at all!
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Thoughts?
> > >>>>>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> > >>>>>>>> cadonna@apache.org
> > >>>>>>>>>>>
> > >>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> 6.
> > >>>>>>>>>>>>>>>>>>>> Of course, you are right! My bad!
> > >>>>>>>>>>>>>>>>>>>> Wiping out the state in the downgrading case is
> > >> fine.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> 3a.
> > >>>>>>>>>>>>>>>>>>>> Focus on the public facing changes for the KIP. We
> > >>>> will
> > >>>>>> manage
> > >>>>>>>>>> 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.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > >>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Thanks for getting back to me!
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>>>>>>>> The fact that implementations can always track
> > >>>> estimated
> > >>>>>>>> memory
> > >>>>>>>>>>>>>>>> usage
> > >>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as
> > >> an
> > >>>>>> option,
> > >>>>>>>> and
> > >>>>>>>>>>>>>>>> I'll
> > >>>>>>>>>>>>>>>>>>>> clarify
> > >>>>>>>>>>>>>>>>>>>>> the JavaDoc that 0 is not just for
> > >> non-transactional
> > >>>>>> stores,
> > >>>>>>>>>>>>>>>> which is
> > >>>>>>>>>>>>>>>>>>>>> currently misleading.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> 6.
> > >>>>>>>>>>>>>>>>>>>>> The problem with catching the exception in the
> > >>>> downgrade
> > >>>>>>>>>> process
> > >>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>> would require new code in the Kafka version being
> > >>>>>> downgraded
> > >>>>>>>>>> to.
> > >>>>>>>>>>>>>>>> Since
> > >>>>>>>>>>>>>>>>>>>>> users could conceivably downgrade to almost *any*
> > >>>> older
> > >>>>>>>> version
> > >>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>> Kafka
> > >>>>>>>>>>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
> > >>>>>>>>>>>>>>>>>>>>> The only way I can think of doing it would be to
> > >>>> provide
> > >>>>>> a
> > >>>>>>>>>>>>>>>> dedicated
> > >>>>>>>>>>>>>>>>>>>>> downgrade tool, that goes through every local
> > >> store
> > >>>> and
> > >>>>>>>> removes
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> offsets column families. But that seems like an
> > >>>>>> unnecessary
> > >>>>>>>>>>>>>>>> amount of
> > >>>>>>>>>>>>>>>>>>>> extra
> > >>>>>>>>>>>>>>>>>>>>> code to maintain just to handle a somewhat niche
> > >>>>>> situation,
> > >>>>>>>>>> when
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> alternative (automatically wipe and restore
> > >> stores)
> > >>>>>> should be
> > >>>>>>>>>>>>>>>>>>>> acceptable.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've
> > >>>> requested.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> 3a.
> > >>>>>>>>>>>>>>>>>>>>> I agree that IsolationLevel makes more sense at
> > >>>>>> query-time,
> > >>>>>>>> and
> > >>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>>>>>>>> initially attempted to place the IsolationLevel
> > >> at
> > >>>>>>>> query-time,
> > >>>>>>>>>>>>>>>> but I
> > >>>>>>>>>>>>>>>>>> ran
> > >>>>>>>>>>>>>>>>>>>>> into some problems:
> > >>>>>>>>>>>>>>>>>>>>> - The key issue is that, under ALOS we're not
> > >> staging
> > >>>>>> writes
> > >>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>> transactions, so can't perform writes at the
> > >>>>>> READ_COMMITTED
> > >>>>>>>>>>>>>>>> isolation
> > >>>>>>>>>>>>>>>>>>>>> level. However, this may be addressed if we
> > >> decide to
> > >>>>>>>> *always*
> > >>>>>>>>>>>>>>>> use
> > >>>>>>>>>>>>>>>>>>>>> transactions as discussed under 3b.
> > >>>>>>>>>>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> > >>>> implementations. I
> > >>>>>>>>>> remember
> > >>>>>>>>>>>>>>>>>> having
> > >>>>>>>>>>>>>>>>>>>>> some difficulty understanding the IQv1 internals,
> > >>>> which
> > >>>>>> made
> > >>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>> difficult
> > >>>>>>>>>>>>>>>>>>>>> to determine what needed to be changed. However,
> > >> I
> > >>>>>> *think*
> > >>>>>>>> this
> > >>>>>>>>>>>>>>>> can be
> > >>>>>>>>>>>>>>>>>>>>> addressed for both implementations by wrapping
> > >> the
> > >>>>>>>> RocksDBStore
> > >>>>>>>>>>>>>>>> in an
> > >>>>>>>>>>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides
> > >> read
> > >>>>>> methods
> > >>>>>>>>>>>>>>>> (get,
> > >>>>>>>>>>>>>>>>>>>> etc.)
> > >>>>>>>>>>>>>>>>>>>>> to either read directly from the database or
> > >> from the
> > >>>>>> ongoing
> > >>>>>>>>>>>>>>>>>>>> transaction.
> > >>>>>>>>>>>>>>>>>>>>> But IQv1 might still be difficult.
> > >>>>>>>>>>>>>>>>>>>>> - If IsolationLevel becomes a query constraint,
> > >> then
> > >>>> all
> > >>>>>>>> other
> > >>>>>>>>>>>>>>>>>>>> StateStores
> > >>>>>>>>>>>>>>>>>>>>> will need to respect it, including the in-memory
> > >>>> stores.
> > >>>>>> This
> > >>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>> require
> > >>>>>>>>>>>>>>>>>>>>> us to adapt in-memory stores to stage their
> > >> writes so
> > >>>>>> they
> > >>>>>>>> can
> > >>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>> isolated
> > >>>>>>>>>>>>>>>>>>>>> from READ_COMMITTTED queries. It would also
> > >> become an
> > >>>>>>>> important
> > >>>>>>>>>>>>>>>>>>>>> consideration for third-party stores on upgrade,
> > >> as
> > >>>>>> without
> > >>>>>>>>>>>>>>>> changes,
> > >>>>>>>>>>>>>>>>>>>> they
> > >>>>>>>>>>>>>>>>>>>>> would not support READ_COMMITTED queries
> > >> correctly.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Ultimately, I may need some help making the
> > >> necessary
> > >>>>>> change
> > >>>>>>>> to
> > >>>>>>>>>>>>>>>> IQv1
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>> support this, but I don't think it's
> > >> fundamentally
> > >>>>>>>> impossible,
> > >>>>>>>>>>>>>>>> if we
> > >>>>>>>>>>>>>>>>>>>> want
> > >>>>>>>>>>>>>>>>>>>>> to pursue this route.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> 3b.
> > >>>>>>>>>>>>>>>>>>>>> The main reason I chose to keep ALOS
> > >> un-transactional
> > >>>>>> was to
> > >>>>>>>>>>>>>>>> minimize
> > >>>>>>>>>>>>>>>>>>>>> behavioural change for most users (I believe most
> > >>>> Streams
> > >>>>>>>> users
> > >>>>>>>>>>>>>>>> use
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> default configuration, which is ALOS). That said,
> > >>>> it's
> > >>>>>> clear
> > >>>>>>>>>>>>>>>> that if
> > >>>>>>>>>>>>>>>>>>>> ALOS
> > >>>>>>>>>>>>>>>>>>>>> also used transactional stores, the only change
> > >> in
> > >>>>>> behaviour
> > >>>>>>>>>>>>>>>> would be
> > >>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>> it would become *more correct*, which could be
> > >>>>>> considered a
> > >>>>>>>>>> "bug
> > >>>>>>>>>>>>>>>> fix"
> > >>>>>>>>>>>>>>>>>> by
> > >>>>>>>>>>>>>>>>>>>>> users, rather than a change they need to handle.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> I believe that performance using transactions
> > >> (aka.
> > >>>>>> RocksDB
> > >>>>>>>>>>>>>>>>>>>> WriteBatches)
> > >>>>>>>>>>>>>>>>>>>>> should actually be *better* than the un-batched
> > >>>>>> write-path
> > >>>>>>>> that
> > >>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>> currently used[1]. The only "performance"
> > >>>> consideration
> > >>>>>> will
> > >>>>>>>> be
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> increased memory usage that transactions require.
> > >>>> Given
> > >>>>>> the
> > >>>>>>>>>>>>>>>>>> mitigations
> > >>>>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>> this memory that we have in place, I would expect
> > >>>> that
> > >>>>>> this
> > >>>>>>>> is
> > >>>>>>>>>>>>>>>> not a
> > >>>>>>>>>>>>>>>>>>>>> problem for most users.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> If we're happy to do so, we can make ALOS also
> > >> use
> > >>>>>>>>>> transactions.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Link 1:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>
> > >>>>
> > >>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> > >>>>>>>>>>>> cadonna@apache.org
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Thanks for the updates and sorry for the delay
> > >> on my
> > >>>>>> side!
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>>>>>>>>> Making the default implementation for flush() a
> > >>>> no-op
> > >>>>>> sounds
> > >>>>>>>>>>>>>>>> good to
> > >>>>>>>>>>>>>>>>>>>> me.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>>>>>>>>> I think what was bugging me here is that a
> > >>>> third-party
> > >>>>>> state
> > >>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>> needs
> > >>>>>>>>>>>>>>>>>>>>>> to implement the state store interface. That
> > >> means
> > >>>> they
> > >>>>>> need
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>> implement a wrapper around the actual state
> > >> store
> > >>>> as we
> > >>>>>> do
> > >>>>>>>> for
> > >>>>>>>>>>>>>>>>>> RocksDB
> > >>>>>>>>>>>>>>>>>>>>>> with RocksDBStore. So, a third-party state
> > >> store can
> > >>>>>> always
> > >>>>>>>>>>>>>>>> estimate
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> uncommitted bytes, if it wants, because the
> > >> wrapper
> > >>>> can
> > >>>>>>>> record
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> added
> > >>>>>>>>>>>>>>>>>>>>>> bytes.
> > >>>>>>>>>>>>>>>>>>>>>> One case I can think of where returning -1 makes
> > >>>> sense
> > >>>>>> is
> > >>>>>>>> when
> > >>>>>>>>>>>>>>>>>> Streams
> > >>>>>>>>>>>>>>>>>>>>>> does not need to estimate the size of the write
> > >>>> batch
> > >>>>>> and
> > >>>>>>>>>>>>>>>> trigger
> > >>>>>>>>>>>>>>>>>>>>>> extraordinary commits, because the third-party
> > >> state
> > >>>>>> store
> > >>>>>>>>>>>>>>>> takes care
> > >>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>> memory. But in that case the method could also
> > >> just
> > >>>>>> return
> > >>>>>>>> 0.
> > >>>>>>>>>>>>>>>> Even
> > >>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>> case would be better solved with a method that
> > >>>> returns
> > >>>>>>>> whether
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>>>> store manages itself the memory used for
> > >> uncommitted
> > >>>>>> bytes
> > >>>>>>>> or
> > >>>>>>>>>>>>>>>> not.
> > >>>>>>>>>>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return
> > >>>> value,
> > >>>>>> I was
> > >>>>>>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>>>>>>>> wondering when and if it will be used.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Regarding returning 0 for transactional state
> > >> stores
> > >>>>>> when
> > >>>>>>>> the
> > >>>>>>>>>>>>>>>> batch
> > >>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>> empty, I was just wondering because you
> > >> explicitly
> > >>>>>> stated
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> "or {@code 0} if this StateStore does not
> > >> support
> > >>>>>>>>>>>> transactions."
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> So it seemed to me returning 0 could only
> > >> happen for
> > >>>>>>>>>>>>>>>>>> non-transactional
> > >>>>>>>>>>>>>>>>>>>>>> state stores.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> a) What do you think if we move the isolation
> > >> level
> > >>>> to
> > >>>>>> IQ
> > >>>>>>>> (v1
> > >>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>> v2)?
> > >>>>>>>>>>>>>>>>>>>>>> In the end this is the only component that
> > >> really
> > >>>> needs
> > >>>>>> to
> > >>>>>>>>>>>>>>>> specify
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> isolation level. It is similar to the Kafka
> > >> consumer
> > >>>>>> that
> > >>>>>>>> can
> > >>>>>>>>>>>>>>>> choose
> > >>>>>>>>>>>>>>>>>>>>>> with what isolation level to read the input
> > >> topic.
> > >>>>>>>>>>>>>>>>>>>>>> For IQv1 the isolation level should go into
> > >>>>>>>>>>>>>>>> StoreQueryParameters. For
> > >>>>>>>>>>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> b) Point a) raises the question what should
> > >> happen
> > >>>>>> during
> > >>>>>>>>>>>>>>>>>> at-least-once
> > >>>>>>>>>>>>>>>>>>>>>> processing when the state store does not use
> > >>>>>> transactions?
> > >>>>>>>>>> John
> > >>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> past proposed to also use transactions on state
> > >>>> stores
> > >>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>> at-least-once. I like that idea, because it
> > >> avoids
> > >>>>>>>> aggregating
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> same
> > >>>>>>>>>>>>>>>>>>>>>> records over and over again in the case of a
> > >>>> failure. We
> > >>>>>>>> had a
> > >>>>>>>>>>>>>>>> case
> > >>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>> the past where a Streams applications in
> > >>>> at-least-once
> > >>>>>> mode
> > >>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>> failing
> > >>>>>>>>>>>>>>>>>>>>>> continuously for some reasons I do not remember
> > >>>> before
> > >>>>>>>>>>>>>>>> committing the
> > >>>>>>>>>>>>>>>>>>>>>> offsets. After each failover, the app aggregated
> > >>>> again
> > >>>>>> and
> > >>>>>>>>>>>>>>>> again the
> > >>>>>>>>>>>>>>>>>>>>>> same records. Of course the aggregate increased
> > >> to
> > >>>> very
> > >>>>>>>> wrong
> > >>>>>>>>>>>>>>>> values
> > >>>>>>>>>>>>>>>>>>>>>> just because of the failover. With transactions
> > >> on
> > >>>> the
> > >>>>>> state
> > >>>>>>>>>>>>>>>> stores
> > >>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>> could have avoided this. The app would have
> > >> output
> > >>>> the
> > >>>>>> same
> > >>>>>>>>>>>>>>>> aggregate
> > >>>>>>>>>>>>>>>>>>>>>> multiple times (i.e., after each failover) but
> > >> at
> > >>>> least
> > >>>>>> the
> > >>>>>>>>>>>>>>>> value of
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> aggregate would not depend on the number of
> > >>>> failovers.
> > >>>>>>>>>>>>>>>> Outputting the
> > >>>>>>>>>>>>>>>>>>>>>> same aggregate multiple times would be incorrect
> > >>>> under
> > >>>>>>>>>>>>>>>> exactly-once
> > >>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>>>> it is OK for at-least-once.
> > >>>>>>>>>>>>>>>>>>>>>> If it makes sense to add a config to turn on
> > >> and off
> > >>>>>>>>>>>>>>>> transactions on
> > >>>>>>>>>>>>>>>>>>>>>> state stores under at-least-once or just use
> > >>>>>> transactions in
> > >>>>>>>>>>>>>>>> any case
> > >>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>> a question we should also discuss in this KIP.
> > >> It
> > >>>>>> depends a
> > >>>>>>>>>> bit
> > >>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would
> > >>>> add a
> > >>>>>>>> config.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 4.
> > >>>>>>>>>>>>>>>>>>>>>> Your points are all valid. I tend to say to
> > >> keep the
> > >>>>>> metrics
> > >>>>>>>>>>>>>>>> around
> > >>>>>>>>>>>>>>>>>>>>>> flush() until we remove flush() completely from
> > >> the
> > >>>>>>>> interface.
> > >>>>>>>>>>>>>>>> Calls
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>> flush() might still exist since existing
> > >> processors
> > >>>>>> might
> > >>>>>>>>>> still
> > >>>>>>>>>>>>>>>> call
> > >>>>>>>>>>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For
> > >>>> sure, we
> > >>>>>> need
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> document
> > >>>>>>>>>>>>>>>>>>>>>> how the metrics change due to the transactions
> > >> in
> > >>>> the
> > >>>>>>>> upgrade
> > >>>>>>>>>>>>>>>> notes.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 5.
> > >>>>>>>>>>>>>>>>>>>>>> I see. Then you should describe how the
> > >> .position
> > >>>> files
> > >>>>>> are
> > >>>>>>>>>>>>>>>> handled
> > >>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>> a dedicated section of the KIP or incorporate
> > >> the
> > >>>>>>>> description
> > >>>>>>>>>>>>>>>> in the
> > >>>>>>>>>>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only
> > >>>>>> mentioning it
> > >>>>>>>>>> in
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> "Compatibility, 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:
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>
> > >>>>>>
> > >>>>
> > >>
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> It is a bit brittle, but it works.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > >>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Thanks for taking the time to review the KIP.
> > >> I'm
> > >>>> back
> > >>>>>> from
> > >>>>>>>>>>>>>>>> leave
> > >>>>>>>>>>>>>>>>>> now
> > >>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>> intend to move this forwards as quickly as I
> > >> can.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Addressing your points:
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>>>>>>>>>> Because flush() is part of the StateStore API,
> > >> it's
> > >>>>>> exposed
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>> custom
> > >>>>>>>>>>>>>>>>>>>>>>> Processors, which might be making calls to
> > >> flush().
> > >>>>>> This
> > >>>>>>>> was
> > >>>>>>>>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> case in a few integration tests.
> > >>>>>>>>>>>>>>>>>>>>>>> To maintain as much compatibility as possible,
> > >> I'd
> > >>>>>> prefer
> > >>>>>>>> not
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>> make
> > >>>>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>> an UnsupportedOperationException, as it will
> > >> cause
> > >>>>>>>> previously
> > >>>>>>>>>>>>>>>>>> working
> > >>>>>>>>>>>>>>>>>>>>>>> Processors to start throwing exceptions at
> > >> runtime.
> > >>>>>>>>>>>>>>>>>>>>>>> I agree that it doesn't make sense for it to
> > >> proxy
> > >>>>>>>> commit(),
> > >>>>>>>>>>>>>>>> though,
> > >>>>>>>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>>>>>> that would cause it to violate the "StateStores
> > >>>> commit
> > >>>>>> only
> > >>>>>>>>>>>>>>>> when the
> > >>>>>>>>>>>>>>>>>>>> Task
> > >>>>>>>>>>>>>>>>>>>>>>> commits" rule.
> > >>>>>>>>>>>>>>>>>>>>>>> Instead, I think we should make this a no-op.
> > >> That
> > >>>> way,
> > >>>>>>>>>>>>>>>> existing
> > >>>>>>>>>>>>>>>>>> user
> > >>>>>>>>>>>>>>>>>>>>>>> Processors will continue to work as-before,
> > >> without
> > >>>>>>>> violation
> > >>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>>>> consistency that would be caused by premature
> > >>>>>> flush/commit
> > >>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>> StateStore
> > >>>>>>>>>>>>>>>>>>>>>>> data to disk.
> > >>>>>>>>>>>>>>>>>>>>>>> What do you think?
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
> > >>>>>> implementation
> > >>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>> transactional, but is unable to estimate the
> > >>>>>> uncommitted
> > >>>>>>>>>>>> memory
> > >>>>>>>>>>>>>>>>>> usage,
> > >>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> method will return -1.
> > >>>>>>>>>>>>>>>>>>>>>>> The intention here is to permit third-party
> > >>>>>> implementations
> > >>>>>>>>>>>>>>>> that may
> > >>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>> able to estimate memory usage.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been
> > >> written to
> > >>>> the
> > >>>>>>>> store
> > >>>>>>>>>>>>>>>> yet. I
> > >>>>>>>>>>>>>>>>>>>>>> thought
> > >>>>>>>>>>>>>>>>>>>>>>> that was implied by "This method will return an
> > >>>>>>>> approximation
> > >>>>>>>>>>>>>>>> of the
> > >>>>>>>>>>>>>>>>>>>>>> memory
> > >>>>>>>>>>>>>>>>>>>>>>> would be freed by the next call to {@link
> > >>>>>> #commit(Map)}"
> > >>>>>>>> and
> > >>>>>>>>>>>>>>>>>> "@return
> > >>>>>>>>>>>>>>>>>>>> The
> > >>>>>>>>>>>>>>>>>>>>>>> approximate size of all records awaiting {@link
> > >>>>>>>>>>>> #commit(Map)}",
> > >>>>>>>>>>>>>>>>>>>> however,
> > >>>>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you
> > >> think
> > >>>> this
> > >>>>>> is
> > >>>>>>>>>>>>>>>> unclear?
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>>>>>> I realise this is probably the most contentious
> > >>>> point
> > >>>>>> in my
> > >>>>>>>>>>>>>>>> design,
> > >>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>> I'm
> > >>>>>>>>>>>>>>>>>>>>>>> open to changing it if I'm unable to convince
> > >> you
> > >>>> of
> > >>>>>> the
> > >>>>>>>>>>>>>>>> benefits.
> > >>>>>>>>>>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > >>>>>>>>>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly
> > >>>> provided
> > >>>>>>>>>>>>>>>> StateStores
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>> query, and it may be important for users to
> > >>>>>>>> programmatically
> > >>>>>>>>>>>>>>>> know
> > >>>>>>>>>>>>>>>>>>>> which
> > >>>>>>>>>>>>>>>>>>>>>>> mode the StateStore is operating under. If we
> > >>>> simply
> > >>>>>>>> provide
> > >>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the
> > >>>> internal
> > >>>>>>>> streams
> > >>>>>>>>>>>>>>>>>>>> engine), or
> > >>>>>>>>>>>>>>>>>>>>>>> similar, then users will need to understand the
> > >>>>>> operation
> > >>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>> consequences
> > >>>>>>>>>>>>>>>>>>>>>>> of each available processing mode and how it
> > >>>> pertains
> > >>>>>> to
> > >>>>>>>>>> their
> > >>>>>>>>>>>>>>>>>>>>>> StateStore.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Interactive Query users aren't the only people
> > >> that
> > >>>>>> care
> > >>>>>>>>>> about
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
> > >>>>>>>> implementers
> > >>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>> custom
> > >>>>>>>>>>>>>>>>>>>>>>> StateStores also need to understand the
> > >> behaviour
> > >>>>>> expected
> > >>>>>>>> of
> > >>>>>>>>>>>>>>>> their
> > >>>>>>>>>>>>>>>>>>>>>>> implementation. KIP-892 introduces some
> > >> assumptions
> > >>>>>> into
> > >>>>>>>> the
> > >>>>>>>>>>>>>>>> Streams
> > >>>>>>>>>>>>>>>>>>>>>> Engine
> > >>>>>>>>>>>>>>>>>>>>>>> about how StateStores operate under each
> > >> processing
> > >>>>>> mode,
> > >>>>>>>> and
> > >>>>>>>>>>>>>>>> it's
> > >>>>>>>>>>>>>>>>>>>>>>> important that custom implementations adhere to
> > >>>> those
> > >>>>>>>>>>>>>>>> assumptions in
> > >>>>>>>>>>>>>>>>>>>>>> order
> > >>>>>>>>>>>>>>>>>>>>>>> to maintain the consistency guarantees.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> IsolationLevels provide a high-level contract
> > >> on
> > >>>> the
> > >>>>>>>>>> behaviour
> > >>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> StateStore: a user knows that under
> > >> READ_COMMITTED,
> > >>>>>> they
> > >>>>>>>> will
> > >>>>>>>>>>>>>>>> see
> > >>>>>>>>>>>>>>>>>>>> writes
> > >>>>>>>>>>>>>>>>>>>>>>> only after the Task has committed, and under
> > >>>>>>>> READ_UNCOMMITTED
> > >>>>>>>>>>>>>>>> they
> > >>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>> see
> > >>>>>>>>>>>>>>>>>>>>>>> writes immediately. No understanding of the
> > >>>> details of
> > >>>>>> each
> > >>>>>>>>>>>>>>>>>>>>>> processing.mode
> > >>>>>>>>>>>>>>>>>>>>>>> is required, either for IQ users or StateStore
> > >>>>>>>> implementers.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> An argument can be made that these contractual
> > >>>>>> guarantees
> > >>>>>>>> can
> > >>>>>>>>>>>>>>>> simply
> > >>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>> documented for the processing.mode (i.e. that
> > >>>>>> exactly-once
> > >>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
> > >>>>>>>> at-least-once
> > >>>>>>>>>>>>>>>> behaves
> > >>>>>>>>>>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small
> > >>>> issues
> > >>>>>> with
> > >>>>>>>>>>>>>>>> this I'd
> > >>>>>>>>>>>>>>>>>>>>>> prefer
> > >>>>>>>>>>>>>>>>>>>>>>> to avoid:
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>          - Where would we document these
> > >> contracts,
> > >>>> in
> > >>>>>> a way
> > >>>>>>>>>> that
> > >>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>> difficult
> > >>>>>>>>>>>>>>>>>>>>>>>          for users/implementers to miss/ignore?
> > >>>>>>>>>>>>>>>>>>>>>>>          - It's not clear to users that the
> > >>>> processing
> > >>>>>> mode
> > >>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>> communicating
> > >>>>>>>>>>>>>>>>>>>>>>>          an expectation of read isolation,
> > >> unless
> > >>>> they
> > >>>>>> read
> > >>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> documentation. Users
> > >>>>>>>>>>>>>>>>>>>>>>>          rarely consult documentation unless
> > >> they
> > >>>> feel
> > >>>>>> they
> > >>>>>>>>>> need
> > >>>>>>>>>>>>>>>> to, so
> > >>>>>>>>>>>>>>>>>>>> it's
> > >>>>>>>>>>>>>>>>>>>>>> likely
> > >>>>>>>>>>>>>>>>>>>>>>>          this detail would get missed by many
> > >> users.
> > >>>>>>>>>>>>>>>>>>>>>>>          - It tightly couples processing modes
> > >> to
> > >>>> read
> > >>>>>>>>>> isolation.
> > >>>>>>>>>>>>>>>> Adding
> > >>>>>>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>>>>>          processing modes, or changing the read
> > >>>>>> isolation of
> > >>>>>>>>>>>>>>>> existing
> > >>>>>>>>>>>>>>>>>>>>>> processing
> > >>>>>>>>>>>>>>>>>>>>>>>          modes would be difficult/impossible.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Ultimately, the cost of introducing
> > >>>> IsolationLevels is
> > >>>>>>>> just a
> > >>>>>>>>>>>>>>>> single
> > >>>>>>>>>>>>>>>>>>>>>>> method, since we re-use the existing
> > >> IsolationLevel
> > >>>>>> enum
> > >>>>>>>> from
> > >>>>>>>>>>>>>>>> Kafka.
> > >>>>>>>>>>>>>>>>>>>> This
> > >>>>>>>>>>>>>>>>>>>>>>> gives us a clear place to document the
> > >> contractual
> > >>>>>>>> guarantees
> > >>>>>>>>>>>>>>>>>> expected
> > >>>>>>>>>>>>>>>>>>>>>>> of/provided by StateStores, that is accessible
> > >>>> both by
> > >>>>>> the
> > >>>>>>>>>>>>>>>>>> StateStore
> > >>>>>>>>>>>>>>>>>>>>>>> itself, and by IQ users.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> (Writing this I've just realised that the
> > >>>> StateStore
> > >>>>>> and IQ
> > >>>>>>>>>>>>>>>> APIs
> > >>>>>>>>>>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>>>>>>>>>> don't provide access to StateStoreContext that
> > >> IQ
> > >>>> users
> > >>>>>>>> would
> > >>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>> direct
> > >>>>>>>>>>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> > >>>>>>>>>> isolationLevel()
> > >>>>>>>>>>>>>>>>>> itself
> > >>>>>>>>>>>>>>>>>>>>>> too?)
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 4.
> > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics
> > >>>> in-place
> > >>>>>>>>>>>>>>>> either, as
> > >>>>>>>>>>>>>>>>>>>> it's a
> > >>>>>>>>>>>>>>>>>>>>>>> backwards incompatible change. My concern is
> > >> that,
> > >>>> if
> > >>>>>> we
> > >>>>>>>>>> leave
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> existing
> > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics in place, they will be
> > >> confusing to
> > >>>>>> users.
> > >>>>>>>>>>>>>>>> Right
> > >>>>>>>>>>>>>>>>>> now,
> > >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics record explicit flushes to
> > >> disk,
> > >>>> but
> > >>>>>> under
> > >>>>>>>>>>>>>>>> KIP-892,
> > >>>>>>>>>>>>>>>>>>>> even
> > >>>>>>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>> commit() will not explicitly flush data to
> > >> disk -
> > >>>>>> RocksDB
> > >>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>> decide
> > >>>>>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>>>> when to flush memtables to disk itself.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd
> > >> have
> > >>>> two
> > >>>>>>>>>>>> options,
> > >>>>>>>>>>>>>>>>>> which
> > >>>>>>>>>>>>>>>>>>>>>> both
> > >>>>>>>>>>>>>>>>>>>>>>> seem pretty bad to me:
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>          1. Have them record calls to commit(),
> > >>>> which
> > >>>>>> would
> > >>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>> misleading, as
> > >>>>>>>>>>>>>>>>>>>>>>>          data is no longer explicitly "flushed"
> > >> to
> > >>>> disk
> > >>>>>> by
> > >>>>>>>> this
> > >>>>>>>>>>>>>>>> call.
> > >>>>>>>>>>>>>>>>>>>>>>>          2. Have them record nothing at all,
> > >> which
> > >>>> is
> > >>>>>>>>>> equivalent
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> removing
> > >>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>          metrics, except that users will see the
> > >>>> metric
> > >>>>>>>> still
> > >>>>>>>>>>>>>>>> exists and
> > >>>>>>>>>>>>>>>>>>>> so
> > >>>>>>>>>>>>>>>>>>>>>> assume
> > >>>>>>>>>>>>>>>>>>>>>>>          that the metric is correct, and that
> > >>>> there's a
> > >>>>>>>> problem
> > >>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>> their
> > >>>>>>>>>>>>>>>>>>>>>> system
> > >>>>>>>>>>>>>>>>>>>>>>>          when there isn't.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> I agree that removing them is also a bad
> > >> solution,
> > >>>> and
> > >>>>>> I'd
> > >>>>>>>>>>>>>>>> like some
> > >>>>>>>>>>>>>>>>>>>>>>> guidance on the best path forward here.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 5.
> > >>>>>>>>>>>>>>>>>>>>>>> Position files are updated on every write to a
> > >>>>>> StateStore.
> > >>>>>>>>>>>>>>>> Since our
> > >>>>>>>>>>>>>>>>>>>>>> writes
> > >>>>>>>>>>>>>>>>>>>>>>> are now buffered until commit(), we can't
> > >> update
> > >>>> the
> > >>>>>>>> Position
> > >>>>>>>>>>>>>>>> file
> > >>>>>>>>>>>>>>>>>>>> until
> > >>>>>>>>>>>>>>>>>>>>>>> commit() has been called, otherwise it would be
> > >>>>>>>> inconsistent
> > >>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> data
> > >>>>>>>>>>>>>>>>>>>>>>> in the event of a rollback. Consequently, we
> > >> need
> > >>>> to
> > >>>>>> manage
> > >>>>>>>>>>>>>>>> these
> > >>>>>>>>>>>>>>>>>>>> offsets
> > >>>>>>>>>>>>>>>>>>>>>>> the same way we manage the checkpoint offsets,
> > >> and
> > >>>>>> ensure
> > >>>>>>>>>>>>>>>> they're
> > >>>>>>>>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>>>>>>> written on commit().
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 6.
> > >>>>>>>>>>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what
> > >>>> tests to
> > >>>>>>>>>> write.
> > >>>>>>>>>>>>>>>> How
> > >>>>>>>>>>>>>>>>>>>>>> explicit
> > >>>>>>>>>>>>>>>>>>>>>>> do we need to be here in the KIP?
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed
> > >> to be
> > >>>>>>>> seamless,
> > >>>>>>>>>>>>>>>> and we
> > >>>>>>>>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>>>>> definitely add some tests around that.
> > >> Downgrade,
> > >>>> it
> > >>>>>>>>>>>>>>>> transpires,
> > >>>>>>>>>>>>>>>>>> isn't
> > >>>>>>>>>>>>>>>>>>>>>>> currently possible, as the extra column family
> > >> for
> > >>>>>> offset
> > >>>>>>>>>>>>>>>> storage is
> > >>>>>>>>>>>>>>>>>>>>>>> incompatible with the pre-KIP-892
> > >> implementation:
> > >>>> when
> > >>>>>> you
> > >>>>>>>>>>>>>>>> open a
> > >>>>>>>>>>>>>>>>>>>> RocksDB
> > >>>>>>>>>>>>>>>>>>>>>>> database, you must open all available column
> > >>>> families
> > >>>>>> or
> > >>>>>>>>>>>>>>>> receive an
> > >>>>>>>>>>>>>>>>>>>>>> error.
> > >>>>>>>>>>>>>>>>>>>>>>> What currently happens on downgrade is that it
> > >>>>>> attempts to
> > >>>>>>>>>>>>>>>> open the
> > >>>>>>>>>>>>>>>>>>>>>> store,
> > >>>>>>>>>>>>>>>>>>>>>>> throws an error about the offsets column
> > >> family not
> > >>>>>> being
> > >>>>>>>>>>>>>>>> opened,
> > >>>>>>>>>>>>>>>>>>>> which
> > >>>>>>>>>>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given
> > >> that
> > >>>>>>>>>> downgrades
> > >>>>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>> uncommon, I think this is acceptable
> > >> behaviour, as
> > >>>> the
> > >>>>>>>>>>>>>>>> end-state is
> > >>>>>>>>>>>>>>>>>>>>>>> consistent, even if it results in an
> > >> undesirable
> > >>>> state
> > >>>>>>>>>>>> restore.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Should I document the upgrade/downgrade
> > >> behaviour
> > >>>>>>>> explicitly
> > >>>>>>>>>>>>>>>> in the
> > >>>>>>>>>>>>>>>>>>>> KIP?
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > >>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Hi Nick!
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates!
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> > >>>>>>>>>>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > >>>>>>>>>>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore
> > >>>> after
> > >>>>>> this
> > >>>>>>>> KIP
> > >>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>> released, I would rather throw an unsupported
> > >>>>>> operation
> > >>>>>>>>>>>>>>>> exception
> > >>>>>>>>>>>>>>>>>> by
> > >>>>>>>>>>>>>>>>>>>>>>>> default.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>>>>>>>>>>> When would a state store return -1 from
> > >>>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
> > >> while
> > >>>>>> being
> > >>>>>>>>>>>>>>>>>>>> transactional?
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Wouldn't
> > >>>> StateStore#approximateNumUncommittedBytes()
> > >>>>>> also
> > >>>>>>>>>>>>>>>> return 0
> > >>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>> the state store is transactional but nothing
> > >> has
> > >>>> been
> > >>>>>>>>>> written
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> state store yet?
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this
> > >> KIP
> > >>>> really
> > >>>>>>>> need
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> introduce
> > >>>>>>>>>>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> > >>>> StateStoreContext
> > >>>>>> has
> > >>>>>>>>>>>>>>>> already
> > >>>>>>>>>>>>>>>>>>>>>>>> appConfigs() which basically exposes the same
> > >>>>>> information,
> > >>>>>>>>>>>>>>>> i.e., if
> > >>>>>>>>>>>>>>>>>>>> EOS
> > >>>>>>>>>>>>>>>>>>>>>>>> is enabled or not.
> > >>>>>>>>>>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore
> > >>>> interface
> > >>>>>> as
> > >>>>>>>>>>>>>>>> loosely
> > >>>>>>>>>>>>>>>>>>>> coupled
> > >>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > >>>>>> implementers
> > >>>>>>>>>>>> more
> > >>>>>>>>>>>>>>>>>>>> freedom,
> > >>>>>>>>>>>>>>>>>>>>>>>> and reduce the amount of internal knowledge
> > >>>> required."
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> While I understand the intent, I doubt that it
> > >>>>>> decreases
> > >>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> coupling of
> > >>>>>>>>>>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> > >>>>>>>>>> READ_COMMITTED
> > >>>>>>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>>>>>>>> applies to IQ but not to reads by processors.
> > >>>> Thus,
> > >>>>>>>>>>>>>>>> implementers
> > >>>>>>>>>>>>>>>>>>>> need to
> > >>>>>>>>>>>>>>>>>>>>>>>> understand how Streams accesses the state
> > >> stores.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> I would like to hear what others think about
> > >> this.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> 4.
> > >>>>>>>>>>>>>>>>>>>>>>>> Great exposing new metrics for transactional
> > >> state
> > >>>>>> stores!
> > >>>>>>>>>>>>>>>>>> However, I
> > >>>>>>>>>>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate
> > >> (in
> > >>>> the
> > >>>>>>>> docs)
> > >>>>>>>>>>>>>>>> the 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 also need to adapt/add
> > >> integration
> > >>>>>> tests
> > >>>>>>>>>>>>>>>> besides
> > >>>>>>>>>>>>>>>>>> unit
> > >>>>>>>>>>>>>>>>>>>>>>>> tests. Additionally, we probably need
> > >> integration
> > >>>> or
> > >>>>>>>> system
> > >>>>>>>>>>>>>>>> tests
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>> verify that upgrades and downgrades between
> > >>>>>> transactional
> > >>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>> non-transactional state stores work as
> > >> expected.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in
> > >> the
> > >>>> KIP,
> > >>>>>>>> under
> > >>>>>>>>>>>>>>>>>>>> "Rejected
> > >>>>>>>>>>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea
> > >> worth
> > >>>>>> pursuing,
> > >>>>>>>>>>>>>>>> but I
> > >>>>>>>>>>>>>>>>>>>> believe
> > >>>>>>>>>>>>>>>>>>>>>>>>> that it's out of the scope of this KIP,
> > >> because
> > >>>> it
> > >>>>>>>> solves a
> > >>>>>>>>>>>>>>>>>>>> different
> > >>>>>>>>>>>>>>>>>>>>>> set
> > >>>>>>>>>>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of
> > >> this
> > >>>> one
> > >>>>>> has
> > >>>>>>>>>>>>>>>> already
> > >>>>>>>>>>>>>>>>>> grown
> > >>>>>>>>>>>>>>>>>>>>>>>> quite
> > >>>>>>>>>>>>>>>>>>>>>>>>> large!
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP (
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>
> > >>>>>>
> > >>>>
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > >>>>>>>>>>>>>>>>>>>>>>>> )
> > >>>>>>>>>>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing
> > >> back
> > >>>>>> "Atomic
> > >>>>>>>>>>>>>>>>>>>> Checkpointing"
> > >>>>>>>>>>>>>>>>>>>>>>>> (for
> > >>>>>>>>>>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think
> > >> the one
> > >>>>>> thing
> > >>>>>>>>>>>>>>>> missing is
> > >>>>>>>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>>>>>>>> changes to metrics (notably the store
> > >> "flush"
> > >>>>>> metrics
> > >>>>>>>> will
> > >>>>>>>>>>>>>>>> need
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>> renamed to "commit").
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> The reason I brought back Atomic
> > >> Checkpointing
> > >>>> was
> > >>>>>> to
> > >>>>>>>>>>>>>>>> decouple
> > >>>>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>>>>> flush
> > >>>>>>>>>>>>>>>>>>>>>>>>>> from store commit. This is important,
> > >> because
> > >>>> with
> > >>>>>>>>>>>>>>>> Transactional
> > >>>>>>>>>>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on
> > >>>> *every*
> > >>>>>> Task
> > >>>>>>>>>>>>>>>> commit,
> > >>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>> just when the StateStore is closing,
> > >> otherwise
> > >>>> our
> > >>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>> buffer
> > >>>>>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>>>>> never be written and persisted, instead
> > >> growing
> > >>>>>>>> unbounded!
> > >>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>> experimented
> > >>>>>>>>>>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a
> > >> store
> > >>>>>> flush
> > >>>>>>>>>>>>>>>> whenever
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its
> > >>>>>> configured
> > >>>>>>>>>>>>>>>> size, but
> > >>>>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer
> > >>>> from
> > >>>>>> being
> > >>>>>>>>>>>>>>>>>> configured
> > >>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>> unbounded, and it still would have required
> > >>>> explicit
> > >>>>>>>>>>>>>>>> flushes of
> > >>>>>>>>>>>>>>>>>>>>>> RocksDB,
> > >>>>>>>>>>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory
> > >>>>>> utilization.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the
> > >> "right"
> > >>>> way
> > >>>>>> to
> > >>>>>>>>>>>>>>>> resolve
> > >>>>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog
> > >> offsets
> > >>>> that
> > >>>>>>>>>>>>>>>> correspond
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> most
> > >>>>>>>>>>>>>>>>>>>>>>>>>> recently written records are always
> > >> atomically
> > >>>>>> written
> > >>>>>>>> to
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> StateStore
> > >>>>>>>>>>>>>>>>>>>>>>>>>> (by writing them to the same transaction
> > >>>> buffer),
> > >>>>>> we can
> > >>>>>>>>>>>>>>>> avoid
> > >>>>>>>>>>>>>>>>>>>>>> forcibly
> > >>>>>>>>>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk,
> > >> letting
> > >>>>>> RocksDB
> > >>>>>>>>>>>>>>>> flush
> > >>>>>>>>>>>>>>>>>> them
> > >>>>>>>>>>>>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>>>>>>>>>> when necessary, without losing any of our
> > >>>>>> consistency
> > >>>>>>>>>>>>>>>> guarantees.
> > >>>>>>>>>>>>>>>>>>>> See
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>> updated KIP for more info.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> I have fully implemented these changes,
> > >>>> although I'm
> > >>>>>>>> still
> > >>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>> entirely
> > >>>>>>>>>>>>>>>>>>>>>>>>>> happy with the implementation for segmented
> > >>>>>> StateStores,
> > >>>>>>>>>> so
> > >>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>> plan
> > >>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests
> > >> pass. If
> > >>>>>> you'd
> > >>>>>>>> like
> > >>>>>>>>>>>>>>>> to try
> > >>>>>>>>>>>>>>>>>>>> out
> > >>>>>>>>>>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>>>>>>>>>>> review this highly experimental and
> > >> incomplete
> > >>>>>> branch,
> > >>>>>>>>>> it's
> > >>>>>>>>>>>>>>>>>>>> available
> > >>>>>>>>>>>>>>>>>>>>>>>> here:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > >>>>>>>> .
> > >>>>>>>>>>>>>>>> Note:
> > >>>>>>>>>>>>>>>>>>>> it's
> > >>>>>>>>>>>>>>>>>>>>>>>> built
> > >>>>>>>>>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable
> > >> base
> > >>>> to
> > >>>>>> build
> > >>>>>>>>>>>>>>>> and test
> > >>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>> on,
> > >>>>>>>>>>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples
> > >> comparisons
> > >>>> in a
> > >>>>>>>> live
> > >>>>>>>>>>>>>>>>>>>>>> environment. I
> > >>>>>>>>>>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's
> > >> nearer
> > >>>>>>>>>> completion
> > >>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>> has
> > >>>>>>>>>>>>>>>>>>>>>> been
> > >>>>>>>>>>>>>>>>>>>>>>>>>> proven on our main application.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing
> > >> and
> > >>>>>> testing:
> > >>>>>>>>>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window)
> > >>>> stores
> > >>>>>>>>>>>>>>>>>>>>>>>>>> - Global stores
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> As I do not currently use either of these,
> > >> so my
> > >>>>>> primary
> > >>>>>>>>>>>>>>>> test
> > >>>>>>>>>>>>>>>>>>>>>>>> environment
> > >>>>>>>>>>>>>>>>>>>>>>>>>> doesn't test these areas.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next
> > >> week
> > >>>> for
> > >>>>>> a few
> > >>>>>>>>>>>>>>>> weeks,
> > >>>>>>>>>>>>>>>>>> so
> > >>>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>>>>> not have time to move this forward until
> > >> late
> > >>>>>> August.
> > >>>>>>>> That
> > >>>>>>>>>>>>>>>> said,
> > >>>>>>>>>>>>>>>>>>>> your
> > >>>>>>>>>>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just
> > >>>> won't be
> > >>>>>>>> able
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> respond
> > >>>>>>>>>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>>>>>>>>> quickly as usual.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact
> > >> on IQ
> > >>>> is
> > >>>>>> not
> > >>>>>>>>>>>>>>>> something
> > >>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>> had
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> considered.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the state
> > >> store
> > >>>>>> from the
> > >>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> > >> the
> > >>>>>>>> checkpoint
> > >>>>>>>>>>>>>>>> (thus,
> > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> > >> data
> > >>>>>> and/or
> > >>>>>>>>>>>>>>>> number of
> > >>>>>>>>>>>>>>>>>>>>>> commit
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you
> > >> suggesting
> > >>>>>> that we
> > >>>>>>>>>>>>>>>> add an
> > >>>>>>>>>>>>>>>>>>>>>>>> additional
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> config for the max number of commit
> > >> intervals
> > >>>>>> between
> > >>>>>>>>>>>>>>>>>> checkpoints?
> > >>>>>>>>>>>>>>>>>>>>>> That
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the
> > >>>>>> transaction
> > >>>>>>>>>>>>>>>> buffers
> > >>>>>>>>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>>>>>>>>> nearly
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of
> > >> commit
> > >>>>>>>> intervals
> > >>>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>>>>>> elapsed,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> whichever comes first?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although
> > >> this
> > >>>>>>>> re-ignites
> > >>>>>>>>>>>>>>>> an
> > >>>>>>>>>>>>>>>>>>>> earlier
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> debate about whether a config should be
> > >>>> measured in
> > >>>>>>>>>>>>>>>> "number of
> > >>>>>>>>>>>>>>>>>>>> commit
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute
> > >> time.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the
> > >> reason
> > >>>> I
> > >>>>>> was
> > >>>>>>>>>>>>>>>> pursuing
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> Atomic
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable
> > >> flush
> > >>>> from
> > >>>>>>>>>>>>>>>> checkpointing,
> > >>>>>>>>>>>>>>>>>>>>>> which
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every
> > >> commit
> > >>>>>> without
> > >>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>> performance
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely
> > >> the
> > >>>>>> "best"
> > >>>>>>>>>>>>>>>> solution,
> > >>>>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>>>>>> I'm not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back
> > >> into
> > >>>> this
> > >>>>>> KIP.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
> > >>>>>> transactional
> > >>>>>>>>>>>> logic
> > >>>>>>>>>>>>>>>>>>>> directly
> > >>>>>>>>>>>>>>>>>>>>>>>> into
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with
> > >> the
> > >>>>>>>>>>>>>>>>>>>>>> StateStore#newTransaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> method, and reduces the number of new
> > >> classes
> > >>>>>>>> introduced,
> > >>>>>>>>>>>>>>>>>>>>>> significantly
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and
> > >> the
> > >>>>>>>> complexity
> > >>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>> drastically
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic
> > >>>> Checkpoints
> > >>>>>>>> into
> > >>>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>> KIP.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna
> > >> <
> > >>>>>>>>>>>>>>>> cadonna@apache.org>
> > >>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> As far as I understand, you want to
> > >> atomically
> > >>>>>> update
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the
> > >>>> memtable
> > >>>>>> of a
> > >>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit
> > >> time
> > >>>>>> elapsed
> > >>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>> after
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that
> > >> would
> > >>>> lead
> > >>>>>> to
> > >>>>>>>>>>>>>>>> exceeding
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > >>>> before the
> > >>>>>>>> next
> > >>>>>>>>>>>>>>>> commit
> > >>>>>>>>>>>>>>>>>>>>>>>> interval
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> ends.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would
> > >> commit
> > >>>>>> every
> > >>>>>>>>>>>>>>>> commit
> > >>>>>>>>>>>>>>>>>>>> interval
> > >>>>>>>>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store will only be atomically
> > >>>> updated
> > >>>>>>>> roughly
> > >>>>>>>>>>>>>>>> every
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of
> > >>>> data.
> > >>>>>> Also
> > >>>>>>>> IQ
> > >>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>> then
> > >>>>>>>>>>>>>>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> see new data roughly every
> > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> After a failure the state store needs to
> > >>>> restore
> > >>>>>> up to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Is this correct?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the state
> > >> store
> > >>>>>> from
> > >>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> > >> the
> > >>>>>>>> checkpoint
> > >>>>>>>>>>>>>>>> (thus,
> > >>>>>>>>>>>>>>>>>>>>>>>> flushing
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> > >> data
> > >>>>>> and/or
> > >>>>>>>>>>>>>>>> number of
> > >>>>>>>>>>>>>>>>>>>>>> commit
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have
> > >> the
> > >>>> same
> > >>>>>> delay
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>> records
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because
> > >> both
> > >>>>>> would
> > >>>>>>>>>>>>>>>> appear
> > >>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However,
> > >>>> after a
> > >>>>>>>> failure
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > >>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> it might restore data that is already in
> > >> the
> > >>>> state
> > >>>>>>>> store
> > >>>>>>>>>>>>>>>>>> because
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable
> > >> offset
> > >>>>>> (i.e.
> > >>>>>>>> the
> > >>>>>>>>>>>>>>>> last
> > >>>>>>>>>>>>>>>>>>>>>> committed
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring
> > >>>> data
> > >>>>>> that
> > >>>>>>>> is
> > >>>>>>>>>>>>>>>> already
> > >>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should
> > >> not
> > >>>>>> violated.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> This solution needs at least one new
> > >> config to
> > >>>>>> specify
> > >>>>>>>>>>>>>>>> when a
> > >>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> should be written.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail
> > >>>> that
> > >>>>>> does
> > >>>>>>>> not
> > >>>>>>>>>>>>>>>> change
> > >>>>>>>>>>>>>>>>>>>>>>>> anything
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit
> > >>>> interval
> > >>>>>> is 30
> > >>>>>>>>>>>>>>>> seconds,
> > >>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>> five
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> seconds.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch
> > >> on
> > >>>> our
> > >>>>>>>> staging
> > >>>>>>>>>>>>>>>>>>>>>> environment,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
> > >>>> non-trivial
> > >>>>>>>>>>>>>>>> application.
> > >>>>>>>>>>>>>>>>>> I'm
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> already
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate
> > >> that
> > >>>> we
> > >>>>>> saw
> > >>>>>>>> the
> > >>>>>>>>>>>>>>>> last
> > >>>>>>>>>>>>>>>>>>>> time
> > >>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP,
> > >> but
> > >>>> this
> > >>>>>>>> time,
> > >>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>> think I
> > >>>>>>>>>>>>>>>>>>>>>> know
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> why.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit,
> > >> which is
> > >>>>>> called
> > >>>>>>>> at
> > >>>>>>>>>>>>>>>> the end
> > >>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Task
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit process, has the following
> > >> behaviour:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under ALOS: checkpoint the
> > >> state
> > >>>>>> stores.
> > >>>>>>>>>> This
> > >>>>>>>>>>>>>>>>>> includes
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            flushing memtables in RocksDB.
> > >>>> This is
> > >>>>>>>>>>>> acceptable
> > >>>>>>>>>>>>>>>>>>>> because the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> default
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit.interval.ms is 5
> > >> seconds,
> > >>>> so
> > >>>>>>>> forcibly
> > >>>>>>>>>>>>>>>> flushing
> > >>>>>>>>>>>>>>>>>>>>>> memtables
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> every 5
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            seconds is acceptable for most
> > >>>>>>>> applications.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under EOS: checkpointing is
> > >> not
> > >>>> done,
> > >>>>>>>>>> *unless*
> > >>>>>>>>>>>>>>>> it's
> > >>>>>>>>>>>>>>>>>>>> being
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> forced, due
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to e.g. the Task closing or
> > >> being
> > >>>>>> revoked.
> > >>>>>>>>>> This
> > >>>>>>>>>>>>>>>> means
> > >>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>> under
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> normal
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing conditions, the
> > >> state
> > >>>> stores
> > >>>>>>>> will
> > >>>>>>>>>> not
> > >>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>> checkpointed,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            not have memtables flushed at
> > >> all ,
> > >>>>>> unless
> > >>>>>>>>>>>> RocksDB
> > >>>>>>>>>>>>>>>>>>>> decides to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush them on
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            its own. Checkpointing stores
> > >> and
> > >>>>>>>>>> force-flushing
> > >>>>>>>>>>>>>>>> their
> > >>>>>>>>>>>>>>>>>>>>>> memtables
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            done when a Task is being
> > >> closed.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint
> > >>>> stores on
> > >>>>>> at
> > >>>>>>>>>>>> least
> > >>>>>>>>>>>>>>>>>> *some*
> > >>>>>>>>>>>>>>>>>>>>>>>> normal
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the
> > >> RocksDB
> > >>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>> buffers
> > >>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets
> > >> are
> > >>>>>> synced to
> > >>>>>>>>>>>>>>>> disk to
> > >>>>>>>>>>>>>>>>>>>>>> prevent
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
> > >>>> Consequently,
> > >>>>>> my
> > >>>>>>>>>>>>>>>> current
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task
> > >> commit,
> > >>>>>> which
> > >>>>>>>> is
> > >>>>>>>>>>>>>>>> far too
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> frequent.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000
> > >> records,
> > >>>>>> which
> > >>>>>>>> is
> > >>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>> change
> > >>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
> > >> performance
> > >>>>>> problems
> > >>>>>>>> for
> > >>>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> applications.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and
> > >> I'm
> > >>>>>>>> currently
> > >>>>>>>>>>>>>>>> leaning
> > >>>>>>>>>>>>>>>>>>>>>>>> towards
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> using the
> > >>>> statestore.transaction.buffer.max.bytes
> > >>>>>>>>>>>>>>>>>> configuration
> > >>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to
> > >>>> exceed it.
> > >>>>>>>> This
> > >>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> complement the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
> > >> functionality
> > >>>> that
> > >>>>>> this
> > >>>>>>>>>>>>>>>>>>>> configuration
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Currently, we use
> > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > >>>>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> force an
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            early Task commit if processing
> > >>>> more
> > >>>>>>>> records
> > >>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>> cause
> > >>>>>>>>>>>>>>>>>>>> our
> > >>>>>>>>>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            transactions to exceed the
> > >> memory
> > >>>>>> assigned
> > >>>>>>>> to
> > >>>>>>>>>>>>>>>> them.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - New functionality: when a
> > >> Task
> > >>>> *does*
> > >>>>>>>>>> commit,
> > >>>>>>>>>>>>>>>> we will
> > >>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the stores (and hence flush the
> > >>>>>> transaction
> > >>>>>>>>>>>>>>>> buffers)
> > >>>>>>>>>>>>>>>>>>>> unless
> > >>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> expect to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            cross the
> > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > >>>>>>>>>>>>>>>>>>>> threshold
> > >>>>>>>>>>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the next
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick
> > >> Telford <
> > >>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I
> > >> meant in
> > >>>>>> terms of
> > >>>>>>>>>>>>>>>>>>>> understanding
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance profile under various
> > >>>>>> circumstances. The
> > >>>>>>>>>>>>>>>> more
> > >>>>>>>>>>>>>>>>>>>> complex
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would
> > >> be for
> > >>>>>> users
> > >>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> understand
> > >>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance they see. For example,
> > >> spilling
> > >>>>>> records
> > >>>>>>>> to
> > >>>>>>>>>>>>>>>> disk
> > >>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold
> > >>>> would, I
> > >>>>>>>>>> expect,
> > >>>>>>>>>>>>>>>>>> reduce
> > >>>>>>>>>>>>>>>>>>>>>> write
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
> > >>>> throughput
> > >>>>>> could
> > >>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>> unexpected,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> > >>>> diagnose/understand for
> > >>>>>>>>>> users.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early
> > >> commit"
> > >>>>>> concept is
> > >>>>>>>>>>>>>>>>>> relatively
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document,
> > >> and
> > >>>>>>>>>>>> conceptually
> > >>>>>>>>>>>>>>>>>> fairly
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> obvious to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to
> > >>>> make it
> > >>>>>>>>>> easier
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>> understand
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it happens though.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an
> > >>>> indirect
> > >>>>>> way
> > >>>>>>>> of
> > >>>>>>>>>>>>>>>>>> telling
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
> > >>>>>> READ_COMMITTED
> > >>>>>>>>>>>>>>>>>>>> essentially
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> requires
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that
> > >> two
> > >>>>>> threads
> > >>>>>>>>>>>>>>>> calling
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes
> > >>>> from
> > >>>>>> the
> > >>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> until
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> > >>>>>> READ_UNCOMMITTED, all
> > >>>>>>>>>>>>>>>> bets are
> > >>>>>>>>>>>>>>>>>>>> off,
> > >>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe
> > >> written
> > >>>>>> records
> > >>>>>>>> at
> > >>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>> time,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> which is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That
> > >> said,
> > >>>>>>>> StateStores
> > >>>>>>>>>>>>>>>> are
> > >>>>>>>>>>>>>>>>>> free
> > >>>>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> implement
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these guarantees however they can,
> > >> which is
> > >>>> a
> > >>>>>> bit
> > >>>>>>>> more
> > >>>>>>>>>>>>>>>>>> relaxed
> > >>>>>>>>>>>>>>>>>>>>>> than
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions".
> > >> For
> > >>>>>> example,
> > >>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>> RocksDB
> > >>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED ==
> > >>>> WBWI-based
> > >>>>>>>>>>>>>>>>>> "transactions",
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the
> > >>>>>> database.
> > >>>>>>>> But
> > >>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> storage
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to
> > >> *always*
> > >>>> use
> > >>>>>>>>>>>>>>>> transactions,
> > >>>>>>>>>>>>>>>>>>>> even
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage
> > >> engines
> > >>>>>> that
> > >>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>> provide
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation
> > >> guarantees
> > >>>> can
> > >>>>>> be
> > >>>>>>>> met
> > >>>>>>>>>>>>>>>> using a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> different
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> technique.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the
> > >> StateStore
> > >>>>>> interface
> > >>>>>>>> as
> > >>>>>>>>>>>>>>>>>> loosely
> > >>>>>>>>>>>>>>>>>>>>>>>> coupled
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to
> > >> give
> > >>>>>>>>>>>>>>>> implementers
> > >>>>>>>>>>>>>>>>>> more
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> freedom, and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge
> > >>>>>> required.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> > >>>> "IsolationLevel"
> > >>>>>> might
> > >>>>>>>>>> not
> > >>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> right
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it
> > >> much
> > >>>> more
> > >>>>>>>>>>>>>>>> explicit if
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> required, e.g.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7-8.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I can make these changes either later
> > >> today
> > >>>> or
> > >>>>>>>>>>>> tomorrow.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Small update:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and
> > >> fixed a
> > >>>>>> bunch of
> > >>>>>>>>>>>>>>>> issues
> > >>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests
> > >> pass,
> > >>>>>> which is
> > >>>>>>>>>>>>>>>>>> promising,
> > >>>>>>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to undergo some performance
> > >> testing. I
> > >>>>>> haven't
> > >>>>>>>>>>>>>>>> (yet)
> > >>>>>>>>>>>>>>>>>>>> worked
> > >>>>>>>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff,
> > >> but I
> > >>>>>> would
> > >>>>>>>>>>>>>>>> expect
> > >>>>>>>>>>>>>>>>>> that,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
> > >>>> difference. The
> > >>>>>>>>>> branch
> > >>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>> available
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> at
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> > >>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>> anyone
> > >>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno
> > >> Cadonna
> > >>>> <
> > >>>>>>>>>>>>>>>>>>>> cadonna@apache.org>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was
> > >> actually
> > >>>> also
> > >>>>>> my
> > >>>>>>>>>>>>>>>> point. I
> > >>>>>>>>>>>>>>>>>>>>>>>> understood
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion
> > >> path
> > >>>> as
> > >>>>>> a way
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>>> avoid
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> early
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the
> > >>>> intent.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually
> > >> it is
> > >>>>>> public
> > >>>>>>>>>>>>>>>> API. My
> > >>>>>>>>>>>>>>>>>>>>>> question
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal
> > >>>>>> processing.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the
> > >> size
> > >>>> of the
> > >>>>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>>>> buffer
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit
> > >> affects
> > >>>> the
> > >>>>>> whole
> > >>>>>>>>>>>>>>>>>> processing
> > >>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a
> > >> state
> > >>>> store
> > >>>>>> is
> > >>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>> confined to
> > >>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and
> > >>>>>> changes the
> > >>>>>>>>>>>>>>>> behavior
> > >>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you
> > >> that
> > >>>> it
> > >>>>>> is a
> > >>>>>>>>>>>>>>>> decent
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compromise. I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides
> > >> and
> > >>>> list
> > >>>>>> the
> > >>>>>>>>>>>>>>>> options to
> > >>>>>>>>>>>>>>>>>>>>>>>> overcome
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all
> > >>>> options
> > >>>>>> seem
> > >>>>>>>>>>>> quite
> > >>>>>>>>>>>>>>>>>> heavy
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> compared
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand
> > >> what you
> > >>>>>> mean
> > >>>>>>>> with
> > >>>>>>>>>>>>>>>> "less
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> predictable
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
> > >>>> alternatives
> > >>>>>>>> really
> > >>>>>>>>>>>>>>>>>>>>>> interesting.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> But I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good
> > >> and
> > >>>> we
> > >>>>>> should
> > >>>>>>>>>>>>>>>> continue
> > >>>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>>> it!
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my
> > >> e-mail on
> > >>>>>> June
> > >>>>>>>>>> 20th:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning
> > >> behind
> > >>>>>> putting
> > >>>>>>>>>>>>>>>> isolation
> > >>>>>>>>>>>>>>>>>>>>>> level
> > >>>>>>>>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should
> > >>>> that
> > >>>>>> also
> > >>>>>>>> be
> > >>>>>>>>>> a
> > >>>>>>>>>>>>>>>> way
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>> give
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to
> > >> decide
> > >>>>>> whether
> > >>>>>>>> to
> > >>>>>>>>>>>>>>>> turn on
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned
> > >> about
> > >>>>>> how do
> > >>>>>>>>>> you
> > >>>>>>>>>>>>>>>> know
> > >>>>>>>>>>>>>>>>>>>> if a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written
> > >> under
> > >>>> EOS,
> > >>>>>> if
> > >>>>>>>> you
> > >>>>>>>>>>>>>>>> do not
> > >>>>>>>>>>>>>>>>>>>>>> have a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> way
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
> > >>>> transactional or
> > >>>>>> not.
> > >>>>>>>>>> If
> > >>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can
> > >> be
> > >>>>>> written
> > >>>>>>>>>>>>>>>> during
> > >>>>>>>>>>>>>>>>>>>> normal
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state
> > >> store
> > >>>> is not
> > >>>>>>>>>>>>>>>>>> transactional,
> > >>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written
> > >> under
> > >>>> EOS.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My point was about not only
> > >> considering the
> > >>>>>> bytes
> > >>>>>>>> in
> > >>>>>>>>>>>>>>>> memory
> > >>>>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>>> config
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but
> > >> also
> > >>>>>> bytes
> > >>>>>>>> that
> > >>>>>>>>>>>>>>>> might
> > >>>>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> spilled
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering
> > >>>> whether you
> > >>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>> remove
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory
> > >>>> bytes to
> > >>>>>> be
> > >>>>>>>>>> used
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store
> > >> records." My
> > >>>>>>>> thinking
> > >>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>> even
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to
> > >>>> disk,
> > >>>>>>>>>> limiting
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> overall
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> bytes
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it
> > >> again
> > >>>> and
> > >>>>>>>>>>>>>>>> considering
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> recent
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much
> > >>>> sense
> > >>>>>>>> anymore.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I like the name
> > >>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes that
> > >>>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 8.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> > >>>>>> implementation
> > >>>>>>>>>>>>>>>> details) of
> > >>>>>>>>>>>>>>>>>>>> how
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of
> > >> changelog
> > >>>>>>>>>>>>>>>> transactions,
> > >>>>>>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be
> > >>>> great.
> > >>>>>>>> Would
> > >>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>> great
> > >>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the
> > >>>>>> behavior in
> > >>>>>>>>>>>>>>>> case of
> > >>>>>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> failure.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional
> > >> state
> > >>>>>> store
> > >>>>>>>>>>>>>>>> recover
> > >>>>>>>>>>>>>>>>>>>> after a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
> > >>>> transaction
> > >>>>>>>> buffer,
> > >>>>>>>>>>>>>>>> etc.
> > >>>>>>>>>>>>>>>>>>>> (that
> > >>>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> what
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> > >>>>>>>>>>>>>>>> WriteBatchWithIndex
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to
> > >>>> exceed)
> > >>>>>>>>>>>>>>>> configured
> > >>>>>>>>>>>>>>>>>>>> memory
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> needs to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
> > >>>> Ultimately,
> > >>>>>> any
> > >>>>>>>>>>>>>>>> approach
> > >>>>>>>>>>>>>>>>>>>> based
> > >>>>>>>>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cracking
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in
> > >>>> ways
> > >>>>>> it's
> > >>>>>>>> not
> > >>>>>>>>>>>>>>>> really
> > >>>>>>>>>>>>>>>>>>>>>>>> designed
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for is
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen
> > >> performance
> > >>>> or
> > >>>>>>>>>>>>>>>> consistency
> > >>>>>>>>>>>>>>>>>>>> issues.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing
> > >> these
> > >>>>>> early
> > >>>>>>>>>>>>>>>> commits?
> > >>>>>>>>>>>>>>>>>>>> While
> > >>>>>>>>>>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to
> > >>>> ensure
> > >>>>>>>>>>>>>>>> consistency
> > >>>>>>>>>>>>>>>>>>>> whilst
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maintaining
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem
> > >> *very*
> > >>>>>>>>>>>>>>>> complicated, and
> > >>>>>>>>>>>>>>>>>>>> might
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for
> > >> users
> > >>>> as a
> > >>>>>>>>>>>>>>>> consequence.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of
> > >> this
> > >>>>>> KIP is
> > >>>>>>>>>>>>>>>> growing a
> > >>>>>>>>>>>>>>>>>>>> bit
> > >>>>>>>>>>>>>>>>>>>>>>>> out
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss
> > >> ideas
> > >>>> for
> > >>>>>>>> future
> > >>>>>>>>>>>>>>>>>>>>>>>> improvements, I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> > >>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope
> > >> down
> > >>>> to a
> > >>>>>>>> design
> > >>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>> achieves
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> most
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
> > >>>>>> restorations
> > >>>>>>>>>>>>>>>> during
> > >>>>>>>>>>>>>>>>>> dirty
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design
> > >> that
> > >>>>>> this KIP
> > >>>>>>>>>>>>>>>> produces
> > >>>>>>>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ultimately
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially
> > >> if
> > >>>> the
> > >>>>>> bulk
> > >>>>>>>> of
> > >>>>>>>>>>>>>>>> it is
> > >>>>>>>>>>>>>>>>>>>>>> internal
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week
> > >>>> trying
> > >>>>>> to
> > >>>>>>>>>>>>>>>> re-work
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove
> > >> the
> > >>>>>>>>>>>>>>>> newTransaction()
> > >>>>>>>>>>>>>>>>>>>>>> method,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> such
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
> > >>>>>>>> RocksDBStore.
> > >>>>>>>>>>>>>>>> That
> > >>>>>>>>>>>>>>>>>>>> way, if
> > >>>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the
> > >> future,
> > >>>>>> like
> > >>>>>>>> the
> > >>>>>>>>>>>>>>>> SST
> > >>>>>>>>>>>>>>>>>> file
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> management
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so
> > >> with
> > >>>>>> little/no
> > >>>>>>>>>>>> API
> > >>>>>>>>>>>>>>>>>>>> changes.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>
> > >>
> > >
>

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

Posted by Guozhang Wang <gu...@gmail.com>.
I'd agree with you guys that as long as we are in agreement about the
configuration semantics, that would be a big win to move forward for
this KIP. As for the TaskCorruptedException handling like wiping state
stores, we can discuss that in the PR rather than in the KIP.

Just to clarify, I'm onboard with the latest proposal, and probably we
can move on for voting on this KIP now?

Guozhang

On Thu, Oct 19, 2023 at 5:33 AM Bruno Cadonna <ca...@apache.org> wrote:
>
> Hi Nick,
>
> What you and Lucas wrote about the different configurations of ALOS/EOS
> and READ_COMMITTED/READ_UNCOMMITTED make sense to me. My earlier
> concerns about changelogs diverging from the content of the local state
> stores turned out to not apply. So I think, we can move on with those
> configurations.
>
> Regarding the TaskCorruptedException and wiping out the state stores
> under EOS, couldn't we abort the transaction on the state store and
> close the task dirty? If the Kafka transaction was indeed committed, the
> store would restore the missing part from the changelog topic. If the
> Kafka transaction was not committed, changelog topic and state store are
> in-sync.
>
> In any case, IMO those are implementation details that we do not need to
> discuss and solve in the KIP discussion. We can solve them on the PR.
> The important thing is that the processing guarantees hold.
>
> Best,
> Bruno
>
> On 10/18/23 3:56 PM, Nick Telford wrote:
> > Hi Lucas,
> >
> > TaskCorruptedException is how Streams signals that the Task state needs to
> > be wiped, so we can't retain that exception without also wiping state on
> > timeouts.
> >
> > Regards,
> > Nick
> >
> > On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <lb...@confluent.io.invalid>
> > wrote:
> >
> >> Hi Nick,
> >>
> >> I think indeed the better behavior would be to retry commitTransaction
> >> until we risk running out of time to meet `max.poll.interval.ms`.
> >>
> >> However, if it's handled as a `TaskCorruptedException` at the moment,
> >> I would do the same in this KIP, and leave exception handling
> >> improvements to future work. 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 <ni...@gmail.com>
> >> wrote:
> >>>
> >>> Hi Lucas,
> >>>
> >>> Yeah, this is pretty much the direction I'm thinking of going in now. You
> >>> make an interesting point about committing on-error under
> >>> ALOS/READ_COMMITTED, although I haven't had a chance to think through the
> >>> implications yet.
> >>>
> >>> Something that I ran into earlier this week is an issue with the new
> >>> handling of TimeoutException. Without TX stores, TimeoutException under
> >> EOS
> >>> throws a TaskCorruptedException, which wipes the stores. However, with TX
> >>> stores, TimeoutException is now just bubbled up and dealt with as it is
> >>> under ALOS. The problem arises when the Producer#commitTransaction call
> >>> times out: Streams attempts to ignore the error and continue producing,
> >>> which causes the next call to Producer#send to throw
> >>> "IllegalStateException: Cannot attempt operation `send` because the
> >>> previous call to `commitTransaction` timed out and must be retried".
> >>>
> >>> I'm not sure what we should do here: retrying the commitTransaction seems
> >>> logical, but what if it times out again? Where do we draw the line and
> >>> shutdown the instance?
> >>>
> >>> Regards,
> >>> Nick
> >>>
> >>> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <lbrutschy@confluent.io
> >> .invalid>
> >>> wrote:
> >>>
> >>>> Hi all,
> >>>>
> >>>> I think I liked your suggestion of allowing EOS with READ_UNCOMMITTED,
> >>>> but keep wiping the state on error, and I'd vote for this solution
> >>>> when introducing `default.state.isolation.level`. This way, we'd have
> >>>> the most low-risk roll-out of this feature (no behavior change without
> >>>> reconfiguration), with the possibility of switching to the most sane /
> >>>> battle-tested default settings in 4.0. Essentially, we'd have a
> >>>> feature flag but call it `default.state.isolation.level` and don't
> >>>> have to deprecate it later.
> >>>>
> >>>> So the possible configurations would then be this:
> >>>>
> >>>> 1. ALOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
> >>>> reads from DB.
> >>>> 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> >>>> WriteBatch/DB. Flush on error (see note below).
> >>>> 3. EOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
> >>>> reads from DB. Wipe state on error.
> >>>> 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> >>>> WriteBatch/DB.
> >>>>
> >>>> I believe the feature is important enough that we will see good
> >>>> adoption even without changing the default. In 4.0, when we have seen
> >>>> this being adopted and is battle-tested, we make READ_COMMITTED the
> >>>> default for EOS, or even READ_COMITTED always the default, depending
> >>>> on our experiences. And we could add a clever implementation of
> >>>> READ_UNCOMITTED with WriteBatches later.
> >>>>
> >>>> The only smell here is that `default.state.isolation.level` wouldn't
> >>>> be purely an IQ setting, but it would also (slightly) change the
> >>>> behavior of the processing, but that seems unavoidable as long as we
> >>>> haven't solve READ_UNCOMITTED IQ with WriteBatches.
> >>>>
> >>>> Minor: As for Bruno's point 4, I think if we are concerned about this
> >>>> 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).
> >>>>
> >>>> Cheers,
> >>>> Lucas
> >>>>
> >>>> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <ni...@gmail.com>
> >>>> wrote:
> >>>>>
> >>>>> Hi Guozhang,
> >>>>>
> >>>>> The KIP as it stands introduces a new configuration,
> >>>>> default.state.isolation.level, which is independent of
> >> processing.mode.
> >>>>> It's intended that this new configuration be used to configure a
> >> global
> >>>> IQ
> >>>>> isolation level in the short term, with a future KIP introducing the
> >>>>> capability to change the isolation level on a per-query basis,
> >> falling
> >>>> back
> >>>>> to the "default" defined by this config. That's why I called it
> >>>> "default",
> >>>>> for future-proofing.
> >>>>>
> >>>>> However, it currently includes the caveat that READ_UNCOMMITTED is
> >> not
> >>>>> available under EOS. I think this is the coupling you are alluding
> >> to?
> >>>>>
> >>>>> This isn't intended to be a restriction of the API, but is currently
> >> a
> >>>>> technical limitation. However, after discussing with some users about
> >>>>> use-cases that would require READ_UNCOMMITTED under EOS, I'm
> >> inclined to
> >>>>> remove that clause and put in the necessary work to make that
> >> combination
> >>>>> possible now.
> >>>>>
> >>>>> I currently see two possible approaches:
> >>>>>
> >>>>>     1. Disable TX StateStores internally when the IsolationLevel is
> >>>>>     READ_UNCOMMITTED and the processing.mode is EOS. This is more
> >>>> difficult
> >>>>>     than it sounds, as there are many assumptions being made
> >> throughout
> >>>> the
> >>>>>     internals about the guarantees StateStores provide. It would
> >>>> definitely add
> >>>>>     a lot of extra "if (read_uncommitted && eos)" branches,
> >> complicating
> >>>>>     maintenance and testing.
> >>>>>     2. Invest the time *now* to make READ_UNCOMMITTED of EOS
> >> StateStores
> >>>>>     possible. I have some ideas on how this could be achieved, but
> >> they
> >>>> would
> >>>>>     need testing and could introduce some additional issues. The
> >> benefit
> >>>> of
> >>>>>     this approach is that it would make query-time IsolationLevels
> >> much
> >>>> simpler
> >>>>>     to implement in the future.
> >>>>>
> >>>>> Unfortunately, both will require considerable work that will further
> >>>> delay
> >>>>> this KIP, which was the reason I placed the restriction in the KIP
> >> in the
> >>>>> first place.
> >>>>>
> >>>>> Regards,
> >>>>> Nick
> >>>>>
> >>>>> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
> >> guozhang.wang.us@gmail.com>
> >>>>> wrote:
> >>>>>
> >>>>>> Hello Nick,
> >>>>>>
> >>>>>> First of all, thanks a lot for the great effort you've put in
> >> driving
> >>>>>> this KIP! I really like it coming through finally, as many people
> >> in
> >>>>>> the community have raised this. At the same time I honestly feel a
> >> bit
> >>>>>> ashamed for not putting enough of my time supporting it and
> >> pushing it
> >>>>>> through the finish line (you raised this KIP almost a year ago).
> >>>>>>
> >>>>>> I briefly passed through the DISCUSS thread so far, not sure I've
> >> 100
> >>>>>> percent digested all the bullet points. But with the goal of
> >> trying to
> >>>>>> help take it through the finish line in mind, I'd want to throw
> >>>>>> thoughts on top of my head only on the point #4 above which I felt
> >> may
> >>>>>> be the main hurdle for the current KIP to drive to a consensus now.
> >>>>>>
> >>>>>> The general question I asked myself is, whether we want to couple
> >> "IQ
> >>>>>> reading mode" with "processing mode". While technically I tend to
> >>>>>> agree with you that, it's feels like a bug if some single user
> >> chose
> >>>>>> "EOS" for processing mode while choosing "read uncommitted" for IQ
> >>>>>> reading mode, at the same time, I'm thinking if it's possible that
> >>>>>> there could be two different persons (or even two teams) that
> >> would be
> >>>>>> using the stream API to build the app, and the IQ API to query the
> >>>>>> running state of the app. I know this is less of a technical thing
> >> but
> >>>>>> rather a more design stuff, but if it could be ever the case, I'm
> >>>>>> wondering if the personale using the IQ API knows about the risks
> >> of
> >>>>>> using read uncommitted but still chose so for the favor of
> >>>>>> performance, no matter if the underlying stream processing mode
> >>>>>> configured by another personale is EOS or not. In that regard, I'm
> >>>>>> leaning towards a "leaving the door open, and close it later if we
> >>>>>> found it's a bad idea" aspect with a configuration 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 can still enforce
> >>>>>> coupling them.
> >>>>>>
> >>>>>> Again, just my 2c here. Thanks again for the great patience and
> >>>>>> diligence on this KIP.
> >>>>>>
> >>>>>>
> >>>>>> Guozhang
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
> >> nick.telford@gmail.com>
> >>>>>> wrote:
> >>>>>>>
> >>>>>>> Hi Bruno,
> >>>>>>>
> >>>>>>> 4.
> >>>>>>> I'll hold off on making that change until we have a consensus as
> >> to
> >>>> what
> >>>>>>> configuration to use to control all of this, as it'll be
> >> affected by
> >>>> the
> >>>>>>> decision on EOS isolation levels.
> >>>>>>>
> >>>>>>> 5.
> >>>>>>> Done. I've chosen "committedOffsets".
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>> Nick
> >>>>>>>
> >>>>>>> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <ca...@apache.org>
> >>>> wrote:
> >>>>>>>
> >>>>>>>> Hi Nick,
> >>>>>>>>
> >>>>>>>> 1.
> >>>>>>>> Yeah, you are probably right that it does not make too much
> >> sense.
> >>>>>>>> Thanks for the clarification!
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 4.
> >>>>>>>> Yes, sorry for the back and forth, but I think for the sake of
> >> the
> >>>> KIP
> >>>>>>>> it is better to let the ALOS behavior as it is for now due to
> >> the
> >>>>>>>> possible issues you would run into. Maybe we can find a
> >> solution
> >>>> in the
> >>>>>>>> future. Now the question returns to whether we really need
> >>>>>>>> default.state.isolation.level. Maybe the config could be the
> >>>> feature
> >>>>>>>> flag Sophie requested.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> 5.
> >>>>>>>> There 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 committedOffsetFor().
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Best,
> >>>>>>>> Bruno
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 10/13/23 3:21 PM, Nick Telford wrote:
> >>>>>>>>> Hi Bruno,
> >>>>>>>>>
> >>>>>>>>> Thanks for getting back to me.
> >>>>>>>>>
> >>>>>>>>> 1.
> >>>>>>>>> I think this should be possible. Are you thinking of the
> >>>> situation
> >>>>>> where
> >>>>>>>> a
> >>>>>>>>> user may downgrade to a previous version of Kafka Streams? In
> >>>> that
> >>>>>> case,
> >>>>>>>>> sadly, the RocksDBStore would get wiped by the older version
> >> of
> >>>> Kafka
> >>>>>>>>> Streams anyway, because that version wouldn't understand the
> >>>> extra
> >>>>>> column
> >>>>>>>>> family (that holds offsets), so the missing Position file
> >> would
> >>>>>>>>> automatically get rebuilt when the store is rebuilt from the
> >>>>>> changelog.
> >>>>>>>>> Are there other situations than downgrade where a
> >> transactional
> >>>> store
> >>>>>>>> could
> >>>>>>>>> be replaced by a non-transactional one? I can't think of any.
> >>>>>>>>>
> >>>>>>>>> 2.
> >>>>>>>>> Ahh yes, the Test Plan - my Kryptonite! This section
> >> definitely
> >>>>>> needs to
> >>>>>>>> be
> >>>>>>>>> fleshed out. I'll work on that. How much detail do you need?
> >>>>>>>>>
> >>>>>>>>> 3.
> >>>>>>>>> See my previous email discussing this.
> >>>>>>>>>
> >>>>>>>>> 4.
> >>>>>>>>> Hmm, this is an interesting point. Are you suggesting that
> >> under
> >>>> ALOS
> >>>>>>>>> READ_COMMITTED should not be supported?
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>> Nick
> >>>>>>>>>
> >>>>>>>>> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
> >> cadonna@apache.org>
> >>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi Nick,
> >>>>>>>>>>
> >>>>>>>>>> I think the KIP is converging!
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 1.
> >>>>>>>>>> I am wondering whether it makes sense to write the position
> >> file
> >>>>>> during
> >>>>>>>>>> close as we do for the checkpoint file, so that in case the
> >>>> state
> >>>>>> store
> >>>>>>>>>> is replaced with a non-transactional state store the
> >>>>>> non-transactional
> >>>>>>>>>> state store finds the position file. I think, this is not
> >>>> strictly
> >>>>>>>>>> needed, but would be a nice behavior instead of just
> >> deleting
> >>>> the
> >>>>>>>>>> position file.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 2.
> >>>>>>>>>> The test plan does not mention integration tests. Do you not
> >>>> need to
> >>>>>>>>>> extend existing ones and add new ones. Also for upgrading
> >> and
> >>>>>>>>>> downgrading you might need integration and/or system tests.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 3.
> >>>>>>>>>> I think Sophie made a point. Although, IQ reading from
> >>>> uncommitted
> >>>>>> data
> >>>>>>>>>> under EOS might be considered a bug by some people. Thus,
> >> your
> >>>> KIP
> >>>>>> would
> >>>>>>>>>> fix a bug rather than changing the intended behavior.
> >> However, I
> >>>>>> also
> >>>>>>>>>> see that a feature flag would help users that rely on this
> >> buggy
> >>>>>>>>>> behavior (at least until AK 4.0).
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 4.
> >>>>>>>>>> This is related to the previous point. I assume that the
> >>>> difference
> >>>>>>>>>> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is
> >> that in
> >>>> the
> >>>>>>>>>> former you enable transactions on the state store and in the
> >>>> latter
> >>>>>> you
> >>>>>>>>>> disable them. If my assumption is correct, I think that is
> >> an
> >>>> issue.
> >>>>>>>>>> Let's assume under ALOS Streams fails over a couple of times
> >>>> more or
> >>>>>>>>>> less at the same step in processing after value 3 is added
> >> to an
> >>>>>>>>>> aggregation but the offset of the corresponding input record
> >>>> was not
> >>>>>>>>>> committed. Without transactions disabled, the aggregation
> >> value
> >>>>>> would
> >>>>>>>>>> increase by 3 for each failover. With transactions enabled,
> >>>> value 3
> >>>>>>>>>> would only be added to the aggregation once when the offset
> >> of
> >>>> the
> >>>>>> input
> >>>>>>>>>> record is committed and the transaction finally completes.
> >> So
> >>>> the
> >>>>>>>>>> content of the state store would change depending on the
> >>>>>> configuration
> >>>>>>>>>> for IQ. IMO, the content of the state store should be
> >>>> independent
> >>>>>> from
> >>>>>>>>>> IQ. Given this issue, I propose to not use transactions with
> >>>> ALOS at
> >>>>>>>>>> all. I was a big proponent of using transactions with ALOS,
> >> but
> >>>> I
> >>>>>>>>>> realized that transactions with ALOS is not as easy as
> >> enabling
> >>>>>>>>>> transactions on state stores. Another aspect that is
> >>>> problematic is
> >>>>>> that
> >>>>>>>>>> the changelog topic which actually replicates the state
> >> store
> >>>> is not
> >>>>>>>>>> transactional under ALOS. Thus, it might happen that the
> >> state
> >>>>>> store and
> >>>>>>>>>> the changelog differ in their content. All of this is maybe
> >>>> solvable
> >>>>>>>>>> somehow, but for the sake of this KIP, I would leave it for
> >> the
> >>>>>> future.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Bruno
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> >>>>>>>>>>> Hey Nick! First of all thanks for taking up this awesome
> >>>> feature,
> >>>>>> I'm
> >>>>>>>>>> sure
> >>>>>>>>>>> every single
> >>>>>>>>>>> Kafka Streams user and dev would agree that it is sorely
> >>>> needed.
> >>>>>>>>>>>
> >>>>>>>>>>> I've just been catching up on the KIP and surrounding
> >>>> discussion,
> >>>>>> so
> >>>>>>>>>> please
> >>>>>>>>>>> forgive me
> >>>>>>>>>>> for any misunderstandings or misinterpretations of the
> >> current
> >>>>>> plan and
> >>>>>>>>>>> don't hesitate to
> >>>>>>>>>>> correct me.
> >>>>>>>>>>>
> >>>>>>>>>>> Before I jump in, I just want to say that having seen this
> >>>> drag on
> >>>>>> for
> >>>>>>>> so
> >>>>>>>>>>> long, my singular
> >>>>>>>>>>> goal in responding is to help this KIP past a perceived
> >>>> impasse so
> >>>>>> we
> >>>>>>>> can
> >>>>>>>>>>> finally move on
> >>>>>>>>>>> to voting and implementing it. Long discussions are to be
> >>>> expected
> >>>>>> for
> >>>>>>>>>>> major features like
> >>>>>>>>>>> this but it's completely on us as the Streams devs to make
> >> sure
> >>>>>> there
> >>>>>>>> is
> >>>>>>>>>> an
> >>>>>>>>>>> end in sight
> >>>>>>>>>>> for any ongoing discussion.
> >>>>>>>>>>>
> >>>>>>>>>>> With that said, it's my understanding that the KIP as
> >> currently
> >>>>>>>> proposed
> >>>>>>>>>> is
> >>>>>>>>>>> just not tenable
> >>>>>>>>>>> for Kafka Streams, and would prevent some EOS users from
> >>>> upgrading
> >>>>>> to
> >>>>>>>> the
> >>>>>>>>>>> version it
> >>>>>>>>>>> first appears in. Given that we can't predict or guarantee
> >>>> whether
> >>>>>> any
> >>>>>>>> of
> >>>>>>>>>>> the followup KIPs
> >>>>>>>>>>> would be completed in the same release cycle as this one,
> >> we
> >>>> need
> >>>>>> to
> >>>>>>>> make
> >>>>>>>>>>> sure that the
> >>>>>>>>>>> feature is either compatible with all current users or else
> >>>>>>>>>> feature-flagged
> >>>>>>>>>>> so that they may
> >>>>>>>>>>> opt in/out.
> >>>>>>>>>>>
> >>>>>>>>>>> Therefore, IIUC we need to have either (or both) of these
> >> as
> >>>>>>>>>>> fully-implemented config options:
> >>>>>>>>>>> 1. default.state.isolation.level
> >>>>>>>>>>> 2. enable.transactional.state.stores
> >>>>>>>>>>>
> >>>>>>>>>>> This way EOS users for whom read_committed semantics are
> >> not
> >>>>>> viable can
> >>>>>>>>>>> still upgrade,
> >>>>>>>>>>> and either use the isolation.level config to leverage the
> >> new
> >>>> txn
> >>>>>> state
> >>>>>>>>>>> stores without sacrificing
> >>>>>>>>>>> their application semantics, or else simply keep the
> >>>> transactional
> >>>>>>>> state
> >>>>>>>>>>> stores disabled until we
> >>>>>>>>>>> are able to fully implement the isolation level
> >> configuration
> >>>> at
> >>>>>> either
> >>>>>>>>>> an
> >>>>>>>>>>> application or query level.
> >>>>>>>>>>>
> >>>>>>>>>>> Frankly you are the expert here and know much more about
> >> the
> >>>>>> tradeoffs
> >>>>>>>> in
> >>>>>>>>>>> both semantics and
> >>>>>>>>>>> effort level of implementing one of these configs vs the
> >>>> other. In
> >>>>>> my
> >>>>>>>>>>> opinion, either option would
> >>>>>>>>>>> be fine and I would leave the decision of which one to
> >> include
> >>>> in
> >>>>>> this
> >>>>>>>>>> KIP
> >>>>>>>>>>> completely up to you.
> >>>>>>>>>>> I just don't see a way for the KIP to proceed without some
> >>>>>> variation of
> >>>>>>>>>> the
> >>>>>>>>>>> above that would allow
> >>>>>>>>>>> EOS users to opt-out of read_committed.
> >>>>>>>>>>>
> >>>>>>>>>>> (If it's all the same to you, I would recommend 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 that gets released.
> >>>>>>>>>>> So personally, my choice would be to add the feature flag
> >> and
> >>>>>> leave it
> >>>>>>>>>> off
> >>>>>>>>>>> by default. If all 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
> >>>>>>>>>>>
> >>>>>>>>>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> >>>>>> nick.telford@gmail.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Agreed, I can live with that for now.
> >>>>>>>>>>>>
> >>>>>>>>>>>> In an effort to keep the scope of this KIP from
> >> expanding, I'm
> >>>>>> leaning
> >>>>>>>>>>>> towards just providing a configurable
> >>>>>> default.state.isolation.level
> >>>>>>>> and
> >>>>>>>>>>>> removing IsolationLevel from the StateStoreContext. This
> >>>> would be
> >>>>>>>>>>>> compatible with adding support for query-time
> >> IsolationLevels
> >>>> in
> >>>>>> the
> >>>>>>>>>>>> future, whilst providing a way for users to select an
> >>>> isolation
> >>>>>> level
> >>>>>>>>>> now.
> >>>>>>>>>>>>
> >>>>>>>>>>>> The big problem with this, however, is that if a user
> >> selects
> >>>>>>>>>>>> processing.mode
> >>>>>>>>>>>> = "exactly-once(-v2|-beta)", and
> >>>> default.state.isolation.level =
> >>>>>>>>>>>> "READ_UNCOMMITTED", we need to guarantee that the data
> >> isn't
> >>>>>> written
> >>>>>>>> to
> >>>>>>>>>>>> disk until commit() is called, but we also need to permit
> >> IQ
> >>>>>> threads
> >>>>>>>> to
> >>>>>>>>>>>> read from the ongoing transaction.
> >>>>>>>>>>>>
> >>>>>>>>>>>> A simple solution would be to (temporarily) forbid this
> >>>>>> combination of
> >>>>>>>>>>>> configuration, and have default.state.isolation.level
> >>>>>> automatically
> >>>>>>>>>> switch
> >>>>>>>>>>>> to READ_COMMITTED when processing.mode is anything other
> >> than
> >>>>>>>>>>>> at-least-once. Do you think this would be acceptable?
> >>>>>>>>>>>>
> >>>>>>>>>>>> In a later KIP, we can add support for query-time
> >> isolation
> >>>>>> levels and
> >>>>>>>>>>>> solve this particular problem there, which would relax
> >> this
> >>>>>>>> restriction.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Regards,
> >>>>>>>>>>>> Nick
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> >>>> cadonna@apache.org>
> >>>>>>>>>> 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,
> >>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> >>>>>>>>>>>>>> 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 <
> >>>>>> nick.telford@gmail.com
> >>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I agree that having IsolationLevel be determined at
> >>>> query-time
> >>>>>> is
> >>>>>>>> the
> >>>>>>>>>>>>>>> ideal design, but there are a few sticking points:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>> There needs to be some way to communicate the
> >>>> IsolationLevel
> >>>>>> down
> >>>>>>>> to
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>> RocksDBStore itself, so that the query can respect it.
> >>>> Since
> >>>>>> stores
> >>>>>>>>>>>> are
> >>>>>>>>>>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
> >>>>>> MeteredStore,
> >>>>>>>>>>>>> etc.),
> >>>>>>>>>>>>>>> we need some way to deliver that information to the
> >> bottom
> >>>>>> layer.
> >>>>>>>> For
> >>>>>>>>>>>>> IQv2,
> >>>>>>>>>>>>>>> we can use the existing State#query() method, but IQv1
> >> has
> >>>> no
> >>>>>> way
> >>>>>>>> to
> >>>>>>>>>>>> do
> >>>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> A simple approach, which would potentially open up
> >> other
> >>>>>> options,
> >>>>>>>>>>>> would
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> >>>>>>>>>>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> >>>>>>>> ReadOnlyKeyValueStore
> >>>>>>>>>>>>> (and
> >>>>>>>>>>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore,
> >>>> etc.).
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>> As mentioned above, RocksDB WriteBatches are not
> >>>> thread-safe,
> >>>>>> which
> >>>>>>>>>>>>> causes
> >>>>>>>>>>>>>>> a problem if we want to provide READ_UNCOMMITTED
> >>>> Iterators. I
> >>>>>> also
> >>>>>>>>>>>> had a
> >>>>>>>>>>>>>>> look at RocksDB Transactions[1], but they solve a very
> >>>>>> different
> >>>>>>>>>>>>> problem,
> >>>>>>>>>>>>>>> and have the same thread-safety issue.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> One possible approach that I mentioned is chaining
> >>>>>> WriteBatches:
> >>>>>>>>>> every
> >>>>>>>>>>>>>>> time a new Interactive Query is received (i.e.
> >>>> readOnlyView,
> >>>>>> see
> >>>>>>>>>>>> above,
> >>>>>>>>>>>>>>> is called) we "freeze" the existing WriteBatch, and
> >> start a
> >>>>>> new one
> >>>>>>>>>>>> for
> >>>>>>>>>>>>> new
> >>>>>>>>>>>>>>> writes. The Interactive Query queries the "chain" of
> >>>> previous
> >>>>>>>>>>>>> WriteBatches
> >>>>>>>>>>>>>>> + the underlying database; while the StreamThread
> >> starts
> >>>>>> writing to
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>> *new* WriteBatch. On-commit, the StreamThread would
> >> write
> >>>> *all*
> >>>>>>>>>>>>>>> WriteBatches in the chain to the database (that have
> >> not
> >>>> yet
> >>>>>> been
> >>>>>>>>>>>>> written).
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> WriteBatches would be closed/freed only when they have
> >> been
> >>>>>> both
> >>>>>>>>>>>>>>> committed, and all open Interactive Queries on them
> >> have
> >>>> been
> >>>>>>>> closed.
> >>>>>>>>>>>>> This
> >>>>>>>>>>>>>>> would require some reference counting.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Obviously a drawback of this approach is the potential
> >> for
> >>>>>>>> increased
> >>>>>>>>>>>>>>> memory usage: if an Interactive Query is long-lived,
> >> for
> >>>>>> example by
> >>>>>>>>>>>>> doing a
> >>>>>>>>>>>>>>> full scan over a large database, or even just pausing
> >> in
> >>>> the
> >>>>>> middle
> >>>>>>>>>> of
> >>>>>>>>>>>>> an
> >>>>>>>>>>>>>>> iteration, then the existing chain of WriteBatches
> >> could be
> >>>>>> kept
> >>>>>>>>>>>> around
> >>>>>>>>>>>>> for
> >>>>>>>>>>>>>>> a long time, potentially forever.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> A.
> >>>>>>>>>>>>>>> Going off on a tangent, it looks like in addition to
> >>>> supporting
> >>>>>>>>>>>>>>> READ_COMMITTED queries, we could go further and support
> >>>>>>>>>>>> REPEATABLE_READ
> >>>>>>>>>>>>>>> queries (i.e. where subsequent reads to the same key
> >> in the
> >>>>>> same
> >>>>>>>>>>>>>>> Interactive Query are guaranteed to yield the same
> >> value)
> >>>> by
> >>>>>> making
> >>>>>>>>>>>> use
> >>>>>>>>>>>>> of
> >>>>>>>>>>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so
> >> the
> >>>>>>>>>> performance
> >>>>>>>>>>>>>>> impact is likely to be negligible, but they do require
> >>>> that the
> >>>>>>>>>>>>> Interactive
> >>>>>>>>>>>>>>> Query session can be explicitly closed.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> This could be achieved if we made the above
> >> readOnlyView
> >>>>>> interface
> >>>>>>>>>>>> look
> >>>>>>>>>>>>>>> more like:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> >>>>>>>>>>>> ReadOnlyKeyValueStore<K,
> >>>>>>>>>>>>>>> V>, AutoCloseable {}
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> >>>>>>>>>>>>>>>         ...
> >>>>>>>>>>>>>>>         ReadOnlyKeyValueView<K, V>
> >>>> readOnlyView(IsolationLevel
> >>>>>>>>>>>>> isolationLevel);
> >>>>>>>>>>>>>>> }
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> But this would be a breaking change, as existing IQv1
> >>>> queries
> >>>>>> are
> >>>>>>>>>>>>>>> guaranteed to never call store.close(), and therefore
> >> these
> >>>>>> would
> >>>>>>>>>> leak
> >>>>>>>>>>>>>>> memory under REPEATABLE_READ.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> B.
> >>>>>>>>>>>>>>> One thing that's notable: MyRocks states that they
> >> support
> >>>>>>>>>>>>> READ_COMMITTED
> >>>>>>>>>>>>>>> and REPEATABLE_READ, but they make no mention of
> >>>>>>>>>>>> READ_UNCOMMITTED[3][4].
> >>>>>>>>>>>>>>> This could be because doing so is technically
> >>>>>> difficult/impossible
> >>>>>>>>>>>> using
> >>>>>>>>>>>>>>> the primitives available in RocksDB.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Lucas, to address your points:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> U1.
> >>>>>>>>>>>>>>> It's only "SHOULD" to permit alternative (i.e.
> >> non-RocksDB)
> >>>>>>>>>>>>>>> implementations of StateStore that do not support
> >> atomic
> >>>>>> writes.
> >>>>>>>>>>>>> Obviously
> >>>>>>>>>>>>>>> in those cases, the guarantees Kafka Streams
> >>>> provides/expects
> >>>>>> would
> >>>>>>>>>> be
> >>>>>>>>>>>>>>> relaxed. Do you think we should require all
> >>>> implementations to
> >>>>>>>>>> support
> >>>>>>>>>>>>>>> atomic writes?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> U2.
> >>>>>>>>>>>>>>> Stores can support multiple IsolationLevels. As we've
> >>>> discussed
> >>>>>>>>>> above,
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> ideal scenario would be to specify the IsolationLevel
> >> at
> >>>>>>>> query-time.
> >>>>>>>>>>>>>>> Failing that, I think the second-best approach is to
> >>>> define the
> >>>>>>>>>>>>>>> IsolationLevel for *all* queries based on the
> >>>> processing.mode,
> >>>>>>>> which
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>> what the default StateStoreContext#isolationLevel()
> >>>> achieves.
> >>>>>> Would
> >>>>>>>>>>>> you
> >>>>>>>>>>>>>>> prefer an alternative?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> While the existing implementation is equivalent to
> >>>>>>>> READ_UNCOMMITTED,
> >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>> can yield unexpected results/errors under EOS, if a
> >>>>>> transaction is
> >>>>>>>>>>>>> rolled
> >>>>>>>>>>>>>>> back. While this would be a change in behaviour for
> >> users,
> >>>> it
> >>>>>> would
> >>>>>>>>>>>> look
> >>>>>>>>>>>>>>> more like a bug fix than a breaking change. That said,
> >> we
> >>>>>> *could*
> >>>>>>>>>> make
> >>>>>>>>>>>>> it
> >>>>>>>>>>>>>>> configurable, and default to the existing behaviour
> >>>>>>>>>> (READ_UNCOMMITTED)
> >>>>>>>>>>>>>>> instead of inferring it from the processing.mode?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> N1, N2.
> >>>>>>>>>>>>>>> These were only primitives to avoid boxing costs, but
> >> since
> >>>>>> this is
> >>>>>>>>>>>> not
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>>> performance sensitive area, it should be fine to
> >> change if
> >>>>>> that's
> >>>>>>>>>>>>> desirable.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> N3.
> >>>>>>>>>>>>>>> It's because the store "manages its own offsets", which
> >>>>>> includes
> >>>>>>>> both
> >>>>>>>>>>>>>>> committing the offset, *and providing it* via
> >>>>>> getCommittedOffset().
> >>>>>>>>>>>>>>> Personally, I think "managesOffsets" conveys this best,
> >>>> but I
> >>>>>> don't
> >>>>>>>>>>>> mind
> >>>>>>>>>>>>>>> changing it if the nomenclature is unclear.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Sorry for the massive emails/essays!
> >>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 1:
> >> https://github.com/facebook/rocksdb/wiki/Transactions
> >>>>>>>>>>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> >>>>>>>>>>>>>>> 3:
> >>>>>>>>
> >> https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> >>>>>>>>>>>>>>> 4:
> >>>> https://mariadb.com/kb/en/myrocks-transactional-isolation/
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> >>>>>>>>>>>>>>> <lb...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> since I last read it in April, the KIP has become much
> >>>>>> cleaner and
> >>>>>>>>>>>>>>>> easier to read. Great work!
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> It feels to me the last big open point is whether we
> >> can
> >>>>>> implement
> >>>>>>>>>>>>>>>> isolation level as a query parameter. I understand
> >> that
> >>>> there
> >>>>>> are
> >>>>>>>>>>>>>>>> implementation concerns, but as Colt says, it would
> >> be a
> >>>> great
> >>>>>>>>>>>>>>>> addition, and would also simplify the migration path
> >> for
> >>>> this
> >>>>>>>>>> change.
> >>>>>>>>>>>>>>>> Is the implementation problem you mentioned caused by
> >> the
> >>>>>>>> WriteBatch
> >>>>>>>>>>>>>>>> not having a notion of a snapshot, as the underlying
> >> DB
> >>>>>> iterator
> >>>>>>>>>>>> does?
> >>>>>>>>>>>>>>>> In that case, I am not sure a chain of WriteBatches
> >> as you
> >>>>>> propose
> >>>>>>>>>>>>>>>> would fully solve the problem, but maybe I didn't dig
> >>>> enough
> >>>>>> into
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>> details to fully understand it.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> If it's not possible to implement it now, would it be
> >> an
> >>>>>> option to
> >>>>>>>>>>>>>>>> make sure in this KIP that we do not fully close the
> >> door
> >>>> on
> >>>>>>>>>>>> per-query
> >>>>>>>>>>>>>>>> isolation levels in the interface, as it may be
> >> possible
> >>>> to
> >>>>>>>>>> implement
> >>>>>>>>>>>>>>>> the missing primitives in RocksDB or Speedb in the
> >> future.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Understanding:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to
> >> be
> >>>>>> persisted
> >>>>>>>>>>>>>>>> atomically with the records?
> >>>>>>>>>>>>>>>> * U2) Don't understand the default implementation of
> >>>>>>>>>>>> `isolationLevel`.
> >>>>>>>>>>>>>>>> The isolation level should be a property of the
> >> underlying
> >>>>>> store,
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>> not be defined by the default config? Existing stores
> >>>> probably
> >>>>>>>> don't
> >>>>>>>>>>>>>>>> guarantee READ_COMMITTED, so the default should be to
> >>>> return
> >>>>>>>>>>>>>>>> READ_UNCOMMITTED.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Nits:
> >>>>>>>>>>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong`
> >>>> return
> >>>>>> type,
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>> avoid the `null`?
> >>>>>>>>>>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
> >>>>>> `OptionalLong`
> >>>>>>>>>>>>>>>> return type, to avoid the `-1`?
> >>>>>>>>>>>>>>>> * N3) I don't understand why `managesOffsets` uses the
> >>>>>> 'manage'
> >>>>>>>>>> verb,
> >>>>>>>>>>>>>>>> whereas all other methods use the "commits" verb. I'd
> >>>> suggest
> >>>>>>>>>>>>>>>> `commitsOffsets`.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Either way, it feels this KIP is very close to the
> >> finish
> >>>>>> line,
> >>>>>>>> I'm
> >>>>>>>>>>>>>>>> looking forward to seeing this in production!
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>> Lucas
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> >>>>>> colt@littlehorse.io
> >>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Making IsolationLevel a query-time constraint,
> >> rather
> >>>> than
> >>>>>>>> linking
> >>>>>>>>>>>> it
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> the processing.guarantee.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> As I understand it, would this allow even a user of
> >> EOS
> >>>> to
> >>>>>>>> control
> >>>>>>>>>>>>>>>> whether
> >>>>>>>>>>>>>>>>> reading committed or uncommitted records? If so, I am
> >>>> highly
> >>>>>> in
> >>>>>>>>>>>> favor
> >>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I know that I was one of the early people to point
> >> out
> >>>> the
> >>>>>>>> current
> >>>>>>>>>>>>>>>>> shortcoming that IQ reads uncommitted records, but
> >> just
> >>>> this
> >>>>>>>>>>>> morning I
> >>>>>>>>>>>>>>>>> realized a pattern we use which means that (for
> >> certain
> >>>>>> queries)
> >>>>>>>>>> our
> >>>>>>>>>>>>>>>> system
> >>>>>>>>>>>>>>>>> needs to be able to read uncommitted records, which
> >> is
> >>>> the
> >>>>>>>> current
> >>>>>>>>>>>>>>>> behavior
> >>>>>>>>>>>>>>>>> of Kafka Streams in EOS.***
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> If IsolationLevel being a query-time decision allows
> >> for
> >>>>>> this,
> >>>>>>>> then
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>> would be amazing. I would also vote that the default
> >>>> behavior
> >>>>>>>>>> should
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>> reading uncommitted records, because it is totally
> >>>> possible
> >>>>>> for a
> >>>>>>>>>>>>> valid
> >>>>>>>>>>>>>>>>> application to depend on that behavior, and breaking
> >> it
> >>>> in a
> >>>>>>>> minor
> >>>>>>>>>>>>>>>> release
> >>>>>>>>>>>>>>>>> might be a bit strong.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> *** (Note, for the curious reader....) Our
> >> use-case/query
> >>>>>> pattern
> >>>>>>>>>>>> is a
> >>>>>>>>>>>>>>>> bit
> >>>>>>>>>>>>>>>>> complex, but reading "uncommitted" records is
> >> actually
> >>>> safe
> >>>>>> in
> >>>>>>>> our
> >>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>> 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 soon as the processor processes
> >> the
> >>>>>> result.
> >>>>>>>> If
> >>>>>>>>>>>>> they
> >>>>>>>>>>>>>>>>> can't read uncommitted records, they may receive a
> >> "201 -
> >>>>>>>> Created"
> >>>>>>>>>>>>>>>>> response, immediately followed by a "404 - 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 <
> >>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Addendum:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I think we would also face the same problem with the
> >>>>>> approach
> >>>>>>>> John
> >>>>>>>>>>>>>>>> outlined
> >>>>>>>>>>>>>>>>>> earlier (using the record cache as a transaction
> >> buffer
> >>>> and
> >>>>>>>>>>>> flushing
> >>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>> straight to SST files). This is because the record
> >> cache
> >>>>>> (the
> >>>>>>>>>>>>>>>> ThreadCache
> >>>>>>>>>>>>>>>>>> class) is not thread-safe, so every commit would
> >>>> invalidate
> >>>>>> open
> >>>>>>>>>> IQ
> >>>>>>>>>>>>>>>>>> Iterators in the same way that RocksDB WriteBatches
> >> do.
> >>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> >>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>>>> 2. Making IsolationLevel a query-time constraint,
> >>>> rather
> >>>>>> than
> >>>>>>>>>>>>>>>> linking it
> >>>>>>>>>>>>>>>>>>> to the processing.guarantee.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> There's a wrinkle that makes this a challenge:
> >>>> Interactive
> >>>>>>>>>> Queries
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> open an Iterator, when using transactions and
> >>>>>> READ_UNCOMMITTED.
> >>>>>>>>>>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries
> >>>> need
> >>>>>> to be
> >>>>>>>>>>>> able
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> read records from the currently uncommitted
> >> transaction
> >>>>>> buffer
> >>>>>>>>>>>>>>>>>>> (WriteBatch). This includes for Iterators, which
> >> should
> >>>>>> iterate
> >>>>>>>>>>>>>>>> both the
> >>>>>>>>>>>>>>>>>>> transaction buffer and underlying database (using
> >>>>>>>>>>>>>>>>>>> WriteBatch#iteratorWithBase()).
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> The issue is that when the StreamThread commits, it
> >>>> writes
> >>>>>> the
> >>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>> WriteBatch to RocksDB *and then clears the
> >> WriteBatch*.
> >>>>>>>> Clearing
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> WriteBatch while an Interactive Query holds an open
> >>>>>> Iterator on
> >>>>>>>>>> it
> >>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>> invalidate the Iterator. Worse, it turns out that
> >>>> Iterators
> >>>>>>>> over
> >>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> WriteBatch become invalidated not just when the
> >>>> WriteBatch
> >>>>>> is
> >>>>>>>>>>>>>>>> cleared,
> >>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>> also when the Iterators' current key receives a new
> >>>> write.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Now that I'm writing this, I remember that this is
> >> the
> >>>>>> major
> >>>>>>>>>>>> reason
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>> switched the original design from having a
> >> query-time
> >>>>>>>>>>>>>>>> IsolationLevel to
> >>>>>>>>>>>>>>>>>>> having the IsolationLevel linked to the
> >>>> transactionality
> >>>>>> of the
> >>>>>>>>>>>>>>>> stores
> >>>>>>>>>>>>>>>>>>> themselves.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> It *might* be possible to resolve this, by having a
> >>>>>> "chain" of
> >>>>>>>>>>>>>>>>>>> WriteBatches, with the StreamThread switching to a
> >> new
> >>>>>>>> WriteBatch
> >>>>>>>>>>>>>>>>>> whenever
> >>>>>>>>>>>>>>>>>>> a new Interactive Query attempts to read from the
> >>>>>> database, but
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>> cause some performance problems/memory pressure
> >> when
> >>>>>> subjected
> >>>>>>>> to
> >>>>>>>>>>>> a
> >>>>>>>>>>>>>>>> high
> >>>>>>>>>>>>>>>>>>> Interactive Query load. It would also reduce the
> >>>>>> efficiency of
> >>>>>>>>>>>>>>>>>> WriteBatches
> >>>>>>>>>>>>>>>>>>> on-commit, as we'd have to write N WriteBatches,
> >> where
> >>>> N
> >>>>>> is the
> >>>>>>>>>>>>>>>> number of
> >>>>>>>>>>>>>>>>>>> Interactive Queries since the last commit.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I realise this is getting into the weeds of the
> >>>>>> implementation,
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>> you'd
> >>>>>>>>>>>>>>>>>>> rather we focus on the API for now, but I think
> >> it's
> >>>>>> important
> >>>>>>>> to
> >>>>>>>>>>>>>>>>>> consider
> >>>>>>>>>>>>>>>>>>> how to implement the desired API, in case we come
> >> up
> >>>> with
> >>>>>> an
> >>>>>>>> API
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> cannot be implemented efficiently, or even at all!
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thoughts?
> >>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> >>>>>>>> cadonna@apache.org
> >>>>>>>>>>>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 6.
> >>>>>>>>>>>>>>>>>>>> Of course, you are right! My bad!
> >>>>>>>>>>>>>>>>>>>> Wiping out the state in the downgrading case is
> >> fine.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 3a.
> >>>>>>>>>>>>>>>>>>>> Focus on the public facing changes for the KIP. We
> >>>> will
> >>>>>> manage
> >>>>>>>>>> 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.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks for getting back to me!
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>>>>> The fact that implementations can always track
> >>>> estimated
> >>>>>>>> memory
> >>>>>>>>>>>>>>>> usage
> >>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as
> >> an
> >>>>>> option,
> >>>>>>>> and
> >>>>>>>>>>>>>>>> I'll
> >>>>>>>>>>>>>>>>>>>> clarify
> >>>>>>>>>>>>>>>>>>>>> the JavaDoc that 0 is not just for
> >> non-transactional
> >>>>>> stores,
> >>>>>>>>>>>>>>>> which is
> >>>>>>>>>>>>>>>>>>>>> currently misleading.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 6.
> >>>>>>>>>>>>>>>>>>>>> The problem with catching the exception in the
> >>>> downgrade
> >>>>>>>>>> process
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>> would require new code in the Kafka version being
> >>>>>> downgraded
> >>>>>>>>>> to.
> >>>>>>>>>>>>>>>> Since
> >>>>>>>>>>>>>>>>>>>>> users could conceivably downgrade to almost *any*
> >>>> older
> >>>>>>>> version
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
> >>>>>>>>>>>>>>>>>>>>> The only way I can think of doing it would be to
> >>>> provide
> >>>>>> a
> >>>>>>>>>>>>>>>> dedicated
> >>>>>>>>>>>>>>>>>>>>> downgrade tool, that goes through every local
> >> store
> >>>> and
> >>>>>>>> removes
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> offsets column families. But that seems like an
> >>>>>> unnecessary
> >>>>>>>>>>>>>>>> amount of
> >>>>>>>>>>>>>>>>>>>> extra
> >>>>>>>>>>>>>>>>>>>>> code to maintain just to handle a somewhat niche
> >>>>>> situation,
> >>>>>>>>>> when
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> alternative (automatically wipe and restore
> >> stores)
> >>>>>> should be
> >>>>>>>>>>>>>>>>>>>> acceptable.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've
> >>>> requested.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 3a.
> >>>>>>>>>>>>>>>>>>>>> I agree that IsolationLevel makes more sense at
> >>>>>> query-time,
> >>>>>>>> and
> >>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>> initially attempted to place the IsolationLevel
> >> at
> >>>>>>>> query-time,
> >>>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>> ran
> >>>>>>>>>>>>>>>>>>>>> into some problems:
> >>>>>>>>>>>>>>>>>>>>> - The key issue is that, under ALOS we're not
> >> staging
> >>>>>> writes
> >>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>> transactions, so can't perform writes at the
> >>>>>> READ_COMMITTED
> >>>>>>>>>>>>>>>> isolation
> >>>>>>>>>>>>>>>>>>>>> level. However, this may be addressed if we
> >> decide to
> >>>>>>>> *always*
> >>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>>>>> transactions as discussed under 3b.
> >>>>>>>>>>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> >>>> implementations. I
> >>>>>>>>>> remember
> >>>>>>>>>>>>>>>>>> having
> >>>>>>>>>>>>>>>>>>>>> some difficulty understanding the IQv1 internals,
> >>>> which
> >>>>>> made
> >>>>>>>> it
> >>>>>>>>>>>>>>>>>>>> difficult
> >>>>>>>>>>>>>>>>>>>>> to determine what needed to be changed. However,
> >> I
> >>>>>> *think*
> >>>>>>>> this
> >>>>>>>>>>>>>>>> can be
> >>>>>>>>>>>>>>>>>>>>> addressed for both implementations by wrapping
> >> the
> >>>>>>>> RocksDBStore
> >>>>>>>>>>>>>>>> in an
> >>>>>>>>>>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides
> >> read
> >>>>>> methods
> >>>>>>>>>>>>>>>> (get,
> >>>>>>>>>>>>>>>>>>>> etc.)
> >>>>>>>>>>>>>>>>>>>>> to either read directly from the database or
> >> from the
> >>>>>> ongoing
> >>>>>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>>>> But IQv1 might still be difficult.
> >>>>>>>>>>>>>>>>>>>>> - If IsolationLevel becomes a query constraint,
> >> then
> >>>> all
> >>>>>>>> other
> >>>>>>>>>>>>>>>>>>>> StateStores
> >>>>>>>>>>>>>>>>>>>>> will need to respect it, including the in-memory
> >>>> stores.
> >>>>>> This
> >>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>> require
> >>>>>>>>>>>>>>>>>>>>> us to adapt in-memory stores to stage their
> >> writes so
> >>>>>> they
> >>>>>>>> can
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>> isolated
> >>>>>>>>>>>>>>>>>>>>> from READ_COMMITTTED queries. It would also
> >> become an
> >>>>>>>> important
> >>>>>>>>>>>>>>>>>>>>> consideration for third-party stores on upgrade,
> >> as
> >>>>>> without
> >>>>>>>>>>>>>>>> changes,
> >>>>>>>>>>>>>>>>>>>> they
> >>>>>>>>>>>>>>>>>>>>> would not support READ_COMMITTED queries
> >> correctly.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Ultimately, I may need some help making the
> >> necessary
> >>>>>> change
> >>>>>>>> to
> >>>>>>>>>>>>>>>> IQv1
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> support this, but I don't think it's
> >> fundamentally
> >>>>>>>> impossible,
> >>>>>>>>>>>>>>>> if we
> >>>>>>>>>>>>>>>>>>>> want
> >>>>>>>>>>>>>>>>>>>>> to pursue this route.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 3b.
> >>>>>>>>>>>>>>>>>>>>> The main reason I chose to keep ALOS
> >> un-transactional
> >>>>>> was to
> >>>>>>>>>>>>>>>> minimize
> >>>>>>>>>>>>>>>>>>>>> behavioural change for most users (I believe most
> >>>> Streams
> >>>>>>>> users
> >>>>>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> default configuration, which is ALOS). That said,
> >>>> it's
> >>>>>> clear
> >>>>>>>>>>>>>>>> that if
> >>>>>>>>>>>>>>>>>>>> ALOS
> >>>>>>>>>>>>>>>>>>>>> also used transactional stores, the only change
> >> in
> >>>>>> behaviour
> >>>>>>>>>>>>>>>> would be
> >>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>> it would become *more correct*, which could be
> >>>>>> considered a
> >>>>>>>>>> "bug
> >>>>>>>>>>>>>>>> fix"
> >>>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>> users, rather than a change they need to handle.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I believe that performance using transactions
> >> (aka.
> >>>>>> RocksDB
> >>>>>>>>>>>>>>>>>>>> WriteBatches)
> >>>>>>>>>>>>>>>>>>>>> should actually be *better* than the un-batched
> >>>>>> write-path
> >>>>>>>> that
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> currently used[1]. The only "performance"
> >>>> consideration
> >>>>>> will
> >>>>>>>> be
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> increased memory usage that transactions require.
> >>>> Given
> >>>>>> the
> >>>>>>>>>>>>>>>>>> mitigations
> >>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>> this memory that we have in place, I would expect
> >>>> that
> >>>>>> this
> >>>>>>>> is
> >>>>>>>>>>>>>>>> not a
> >>>>>>>>>>>>>>>>>>>>> problem for most users.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> If we're happy to do so, we can make ALOS also
> >> use
> >>>>>>>>>> transactions.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Link 1:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>
> >>>>
> >> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> >>>>>>>>>>>> cadonna@apache.org
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks for the updates and sorry for the delay
> >> on my
> >>>>>> side!
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>>>>>> Making the default implementation for flush() a
> >>>> no-op
> >>>>>> sounds
> >>>>>>>>>>>>>>>> good to
> >>>>>>>>>>>>>>>>>>>> me.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>>>>>> I think what was bugging me here is that a
> >>>> third-party
> >>>>>> state
> >>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>> needs
> >>>>>>>>>>>>>>>>>>>>>> to implement the state store interface. That
> >> means
> >>>> they
> >>>>>> need
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> implement a wrapper around the actual state
> >> store
> >>>> as we
> >>>>>> do
> >>>>>>>> for
> >>>>>>>>>>>>>>>>>> RocksDB
> >>>>>>>>>>>>>>>>>>>>>> with RocksDBStore. So, a third-party state
> >> store can
> >>>>>> always
> >>>>>>>>>>>>>>>> estimate
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> uncommitted bytes, if it wants, because the
> >> wrapper
> >>>> can
> >>>>>>>> record
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> added
> >>>>>>>>>>>>>>>>>>>>>> bytes.
> >>>>>>>>>>>>>>>>>>>>>> One case I can think of where returning -1 makes
> >>>> sense
> >>>>>> is
> >>>>>>>> when
> >>>>>>>>>>>>>>>>>> Streams
> >>>>>>>>>>>>>>>>>>>>>> does not need to estimate the size of the write
> >>>> batch
> >>>>>> and
> >>>>>>>>>>>>>>>> trigger
> >>>>>>>>>>>>>>>>>>>>>> extraordinary commits, because the third-party
> >> state
> >>>>>> store
> >>>>>>>>>>>>>>>> takes care
> >>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> memory. But in that case the method could also
> >> just
> >>>>>> return
> >>>>>>>> 0.
> >>>>>>>>>>>>>>>> Even
> >>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> case would be better solved with a method that
> >>>> returns
> >>>>>>>> whether
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>> store manages itself the memory used for
> >> uncommitted
> >>>>>> bytes
> >>>>>>>> or
> >>>>>>>>>>>>>>>> not.
> >>>>>>>>>>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return
> >>>> value,
> >>>>>> I was
> >>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>>>> wondering when and if it will be used.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Regarding returning 0 for transactional state
> >> stores
> >>>>>> when
> >>>>>>>> the
> >>>>>>>>>>>>>>>> batch
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> empty, I was just wondering because you
> >> explicitly
> >>>>>> stated
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> "or {@code 0} if this StateStore does not
> >> support
> >>>>>>>>>>>> transactions."
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> So it seemed to me returning 0 could only
> >> happen for
> >>>>>>>>>>>>>>>>>> non-transactional
> >>>>>>>>>>>>>>>>>>>>>> state stores.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> a) What do you think if we move the isolation
> >> level
> >>>> to
> >>>>>> IQ
> >>>>>>>> (v1
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> v2)?
> >>>>>>>>>>>>>>>>>>>>>> In the end this is the only component that
> >> really
> >>>> needs
> >>>>>> to
> >>>>>>>>>>>>>>>> specify
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> isolation level. It is similar to the Kafka
> >> consumer
> >>>>>> that
> >>>>>>>> can
> >>>>>>>>>>>>>>>> choose
> >>>>>>>>>>>>>>>>>>>>>> with what isolation level to read the input
> >> topic.
> >>>>>>>>>>>>>>>>>>>>>> For IQv1 the isolation level should go into
> >>>>>>>>>>>>>>>> StoreQueryParameters. For
> >>>>>>>>>>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> b) Point a) raises the question what should
> >> happen
> >>>>>> during
> >>>>>>>>>>>>>>>>>> at-least-once
> >>>>>>>>>>>>>>>>>>>>>> processing when the state store does not use
> >>>>>> transactions?
> >>>>>>>>>> John
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> past proposed to also use transactions on state
> >>>> stores
> >>>>>> for
> >>>>>>>>>>>>>>>>>>>>>> at-least-once. I like that idea, because it
> >> avoids
> >>>>>>>> aggregating
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>> records over and over again in the case of a
> >>>> failure. We
> >>>>>>>> had a
> >>>>>>>>>>>>>>>> case
> >>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>> the past where a Streams applications in
> >>>> at-least-once
> >>>>>> mode
> >>>>>>>>>> was
> >>>>>>>>>>>>>>>>>> failing
> >>>>>>>>>>>>>>>>>>>>>> continuously for some reasons I do not remember
> >>>> before
> >>>>>>>>>>>>>>>> committing the
> >>>>>>>>>>>>>>>>>>>>>> offsets. After each failover, the app aggregated
> >>>> again
> >>>>>> and
> >>>>>>>>>>>>>>>> again the
> >>>>>>>>>>>>>>>>>>>>>> same records. Of course the aggregate increased
> >> to
> >>>> very
> >>>>>>>> wrong
> >>>>>>>>>>>>>>>> values
> >>>>>>>>>>>>>>>>>>>>>> just because of the failover. With transactions
> >> on
> >>>> the
> >>>>>> state
> >>>>>>>>>>>>>>>> stores
> >>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>> could have avoided this. The app would have
> >> output
> >>>> the
> >>>>>> same
> >>>>>>>>>>>>>>>> aggregate
> >>>>>>>>>>>>>>>>>>>>>> multiple times (i.e., after each failover) but
> >> at
> >>>> least
> >>>>>> the
> >>>>>>>>>>>>>>>> value of
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> aggregate would not depend on the number of
> >>>> failovers.
> >>>>>>>>>>>>>>>> Outputting the
> >>>>>>>>>>>>>>>>>>>>>> same aggregate multiple times would be incorrect
> >>>> under
> >>>>>>>>>>>>>>>> exactly-once
> >>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>> it is OK for at-least-once.
> >>>>>>>>>>>>>>>>>>>>>> If it makes sense to add a config to turn on
> >> and off
> >>>>>>>>>>>>>>>> transactions on
> >>>>>>>>>>>>>>>>>>>>>> state stores under at-least-once or just use
> >>>>>> transactions in
> >>>>>>>>>>>>>>>> any case
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> a question we should also discuss in this KIP.
> >> It
> >>>>>> depends a
> >>>>>>>>>> bit
> >>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would
> >>>> add a
> >>>>>>>> config.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 4.
> >>>>>>>>>>>>>>>>>>>>>> Your points are all valid. I tend to say to
> >> keep the
> >>>>>> metrics
> >>>>>>>>>>>>>>>> around
> >>>>>>>>>>>>>>>>>>>>>> flush() until we remove flush() completely from
> >> the
> >>>>>>>> interface.
> >>>>>>>>>>>>>>>> Calls
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> flush() might still exist since existing
> >> processors
> >>>>>> might
> >>>>>>>>>> still
> >>>>>>>>>>>>>>>> call
> >>>>>>>>>>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For
> >>>> sure, we
> >>>>>> need
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> document
> >>>>>>>>>>>>>>>>>>>>>> how the metrics change due to the transactions
> >> in
> >>>> the
> >>>>>>>> upgrade
> >>>>>>>>>>>>>>>> notes.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 5.
> >>>>>>>>>>>>>>>>>>>>>> I see. Then you should describe how the
> >> .position
> >>>> files
> >>>>>> are
> >>>>>>>>>>>>>>>> handled
> >>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>> a dedicated section of the KIP or incorporate
> >> the
> >>>>>>>> description
> >>>>>>>>>>>>>>>> in the
> >>>>>>>>>>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only
> >>>>>> mentioning it
> >>>>>>>>>> in
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> "Compatibility, 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:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> It is a bit brittle, but it works.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thanks for taking the time to review the KIP.
> >> I'm
> >>>> back
> >>>>>> from
> >>>>>>>>>>>>>>>> leave
> >>>>>>>>>>>>>>>>>> now
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>> intend to move this forwards as quickly as I
> >> can.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Addressing your points:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>>>>>>> Because flush() is part of the StateStore API,
> >> it's
> >>>>>> exposed
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> custom
> >>>>>>>>>>>>>>>>>>>>>>> Processors, which might be making calls to
> >> flush().
> >>>>>> This
> >>>>>>>> was
> >>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> case in a few integration tests.
> >>>>>>>>>>>>>>>>>>>>>>> To maintain as much compatibility as possible,
> >> I'd
> >>>>>> prefer
> >>>>>>>> not
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> an UnsupportedOperationException, as it will
> >> cause
> >>>>>>>> previously
> >>>>>>>>>>>>>>>>>> working
> >>>>>>>>>>>>>>>>>>>>>>> Processors to start throwing exceptions at
> >> runtime.
> >>>>>>>>>>>>>>>>>>>>>>> I agree that it doesn't make sense for it to
> >> proxy
> >>>>>>>> commit(),
> >>>>>>>>>>>>>>>> though,
> >>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>> that would cause it to violate the "StateStores
> >>>> commit
> >>>>>> only
> >>>>>>>>>>>>>>>> when the
> >>>>>>>>>>>>>>>>>>>> Task
> >>>>>>>>>>>>>>>>>>>>>>> commits" rule.
> >>>>>>>>>>>>>>>>>>>>>>> Instead, I think we should make this a no-op.
> >> That
> >>>> way,
> >>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>> user
> >>>>>>>>>>>>>>>>>>>>>>> Processors will continue to work as-before,
> >> without
> >>>>>>>> violation
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>> consistency that would be caused by premature
> >>>>>> flush/commit
> >>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> StateStore
> >>>>>>>>>>>>>>>>>>>>>>> data to disk.
> >>>>>>>>>>>>>>>>>>>>>>> What do you think?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
> >>>>>> implementation
> >>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> transactional, but is unable to estimate the
> >>>>>> uncommitted
> >>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>> usage,
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> method will return -1.
> >>>>>>>>>>>>>>>>>>>>>>> The intention here is to permit third-party
> >>>>>> implementations
> >>>>>>>>>>>>>>>> that may
> >>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>> able to estimate memory usage.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been
> >> written to
> >>>> the
> >>>>>>>> store
> >>>>>>>>>>>>>>>> yet. I
> >>>>>>>>>>>>>>>>>>>>>> thought
> >>>>>>>>>>>>>>>>>>>>>>> that was implied by "This method will return an
> >>>>>>>> approximation
> >>>>>>>>>>>>>>>> of the
> >>>>>>>>>>>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>>>>>>> would be freed by the next call to {@link
> >>>>>> #commit(Map)}"
> >>>>>>>> and
> >>>>>>>>>>>>>>>>>> "@return
> >>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>>> approximate size of all records awaiting {@link
> >>>>>>>>>>>> #commit(Map)}",
> >>>>>>>>>>>>>>>>>>>> however,
> >>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you
> >> think
> >>>> this
> >>>>>> is
> >>>>>>>>>>>>>>>> unclear?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>>> I realise this is probably the most contentious
> >>>> point
> >>>>>> in my
> >>>>>>>>>>>>>>>> design,
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>>>>>>>> open to changing it if I'm unable to convince
> >> you
> >>>> of
> >>>>>> the
> >>>>>>>>>>>>>>>> benefits.
> >>>>>>>>>>>>>>>>>>>>>>> Nevertheless, here's my argument:
> >>>>>>>>>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly
> >>>> provided
> >>>>>>>>>>>>>>>> StateStores
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> query, and it may be important for users to
> >>>>>>>> programmatically
> >>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>> mode the StateStore is operating under. If we
> >>>> simply
> >>>>>>>> provide
> >>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the
> >>>> internal
> >>>>>>>> streams
> >>>>>>>>>>>>>>>>>>>> engine), or
> >>>>>>>>>>>>>>>>>>>>>>> similar, then users will need to understand the
> >>>>>> operation
> >>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>> consequences
> >>>>>>>>>>>>>>>>>>>>>>> of each available processing mode and how it
> >>>> pertains
> >>>>>> to
> >>>>>>>>>> their
> >>>>>>>>>>>>>>>>>>>>>> StateStore.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Interactive Query users aren't the only people
> >> that
> >>>>>> care
> >>>>>>>>>> about
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
> >>>>>>>> implementers
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>> custom
> >>>>>>>>>>>>>>>>>>>>>>> StateStores also need to understand the
> >> behaviour
> >>>>>> expected
> >>>>>>>> of
> >>>>>>>>>>>>>>>> their
> >>>>>>>>>>>>>>>>>>>>>>> implementation. KIP-892 introduces some
> >> assumptions
> >>>>>> into
> >>>>>>>> the
> >>>>>>>>>>>>>>>> Streams
> >>>>>>>>>>>>>>>>>>>>>> Engine
> >>>>>>>>>>>>>>>>>>>>>>> about how StateStores operate under each
> >> processing
> >>>>>> mode,
> >>>>>>>> and
> >>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>> important that custom implementations adhere to
> >>>> those
> >>>>>>>>>>>>>>>> assumptions in
> >>>>>>>>>>>>>>>>>>>>>> order
> >>>>>>>>>>>>>>>>>>>>>>> to maintain the consistency guarantees.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> IsolationLevels provide a high-level contract
> >> on
> >>>> the
> >>>>>>>>>> behaviour
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> StateStore: a user knows that under
> >> READ_COMMITTED,
> >>>>>> they
> >>>>>>>> will
> >>>>>>>>>>>>>>>> see
> >>>>>>>>>>>>>>>>>>>> writes
> >>>>>>>>>>>>>>>>>>>>>>> only after the Task has committed, and under
> >>>>>>>> READ_UNCOMMITTED
> >>>>>>>>>>>>>>>> they
> >>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>> see
> >>>>>>>>>>>>>>>>>>>>>>> writes immediately. No understanding of the
> >>>> details of
> >>>>>> each
> >>>>>>>>>>>>>>>>>>>>>> processing.mode
> >>>>>>>>>>>>>>>>>>>>>>> is required, either for IQ users or StateStore
> >>>>>>>> implementers.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> An argument can be made that these contractual
> >>>>>> guarantees
> >>>>>>>> can
> >>>>>>>>>>>>>>>> simply
> >>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>> documented for the processing.mode (i.e. that
> >>>>>> exactly-once
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
> >>>>>>>> at-least-once
> >>>>>>>>>>>>>>>> behaves
> >>>>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small
> >>>> issues
> >>>>>> with
> >>>>>>>>>>>>>>>> this I'd
> >>>>>>>>>>>>>>>>>>>>>> prefer
> >>>>>>>>>>>>>>>>>>>>>>> to avoid:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>          - Where would we document these
> >> contracts,
> >>>> in
> >>>>>> a way
> >>>>>>>>>> that
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>> difficult
> >>>>>>>>>>>>>>>>>>>>>>>          for users/implementers to miss/ignore?
> >>>>>>>>>>>>>>>>>>>>>>>          - It's not clear to users that the
> >>>> processing
> >>>>>> mode
> >>>>>>>> is
> >>>>>>>>>>>>>>>>>>>> communicating
> >>>>>>>>>>>>>>>>>>>>>>>          an expectation of read isolation,
> >> unless
> >>>> they
> >>>>>> read
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> documentation. Users
> >>>>>>>>>>>>>>>>>>>>>>>          rarely consult documentation unless
> >> they
> >>>> feel
> >>>>>> they
> >>>>>>>>>> need
> >>>>>>>>>>>>>>>> to, so
> >>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>> likely
> >>>>>>>>>>>>>>>>>>>>>>>          this detail would get missed by many
> >> users.
> >>>>>>>>>>>>>>>>>>>>>>>          - It tightly couples processing modes
> >> to
> >>>> read
> >>>>>>>>>> isolation.
> >>>>>>>>>>>>>>>> Adding
> >>>>>>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>>>>>          processing modes, or changing the read
> >>>>>> isolation of
> >>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>>>>          modes would be difficult/impossible.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Ultimately, the cost of introducing
> >>>> IsolationLevels is
> >>>>>>>> just a
> >>>>>>>>>>>>>>>> single
> >>>>>>>>>>>>>>>>>>>>>>> method, since we re-use the existing
> >> IsolationLevel
> >>>>>> enum
> >>>>>>>> from
> >>>>>>>>>>>>>>>> Kafka.
> >>>>>>>>>>>>>>>>>>>> This
> >>>>>>>>>>>>>>>>>>>>>>> gives us a clear place to document the
> >> contractual
> >>>>>>>> guarantees
> >>>>>>>>>>>>>>>>>> expected
> >>>>>>>>>>>>>>>>>>>>>>> of/provided by StateStores, that is accessible
> >>>> both by
> >>>>>> the
> >>>>>>>>>>>>>>>>>> StateStore
> >>>>>>>>>>>>>>>>>>>>>>> itself, and by IQ users.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> (Writing this I've just realised that the
> >>>> StateStore
> >>>>>> and IQ
> >>>>>>>>>>>>>>>> APIs
> >>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>> don't provide access to StateStoreContext that
> >> IQ
> >>>> users
> >>>>>>>> would
> >>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>> direct
> >>>>>>>>>>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> >>>>>>>>>> isolationLevel()
> >>>>>>>>>>>>>>>>>> itself
> >>>>>>>>>>>>>>>>>>>>>> too?)
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 4.
> >>>>>>>>>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics
> >>>> in-place
> >>>>>>>>>>>>>>>> either, as
> >>>>>>>>>>>>>>>>>>>> it's a
> >>>>>>>>>>>>>>>>>>>>>>> backwards incompatible change. My concern is
> >> that,
> >>>> if
> >>>>>> we
> >>>>>>>>>> leave
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics in place, they will be
> >> confusing to
> >>>>>> users.
> >>>>>>>>>>>>>>>> Right
> >>>>>>>>>>>>>>>>>> now,
> >>>>>>>>>>>>>>>>>>>>>>> "flush" metrics record explicit flushes to
> >> disk,
> >>>> but
> >>>>>> under
> >>>>>>>>>>>>>>>> KIP-892,
> >>>>>>>>>>>>>>>>>>>> even
> >>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>> commit() will not explicitly flush data to
> >> disk -
> >>>>>> RocksDB
> >>>>>>>>>> will
> >>>>>>>>>>>>>>>>>> decide
> >>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>> when to flush memtables to disk itself.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd
> >> have
> >>>> two
> >>>>>>>>>>>> options,
> >>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>> both
> >>>>>>>>>>>>>>>>>>>>>>> seem pretty bad to me:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>          1. Have them record calls to commit(),
> >>>> which
> >>>>>> would
> >>>>>>>> be
> >>>>>>>>>>>>>>>>>>>> misleading, as
> >>>>>>>>>>>>>>>>>>>>>>>          data is no longer explicitly "flushed"
> >> to
> >>>> disk
> >>>>>> by
> >>>>>>>> this
> >>>>>>>>>>>>>>>> call.
> >>>>>>>>>>>>>>>>>>>>>>>          2. Have them record nothing at all,
> >> which
> >>>> is
> >>>>>>>>>> equivalent
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> removing
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>          metrics, except that users will see the
> >>>> metric
> >>>>>>>> still
> >>>>>>>>>>>>>>>> exists and
> >>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>> assume
> >>>>>>>>>>>>>>>>>>>>>>>          that the metric is correct, and that
> >>>> there's a
> >>>>>>>> problem
> >>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>> their
> >>>>>>>>>>>>>>>>>>>>>> system
> >>>>>>>>>>>>>>>>>>>>>>>          when there isn't.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I agree that removing them is also a bad
> >> solution,
> >>>> and
> >>>>>> I'd
> >>>>>>>>>>>>>>>> like some
> >>>>>>>>>>>>>>>>>>>>>>> guidance on the best path forward here.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 5.
> >>>>>>>>>>>>>>>>>>>>>>> Position files are updated on every write to a
> >>>>>> StateStore.
> >>>>>>>>>>>>>>>> Since our
> >>>>>>>>>>>>>>>>>>>>>> writes
> >>>>>>>>>>>>>>>>>>>>>>> are now buffered until commit(), we can't
> >> update
> >>>> the
> >>>>>>>> Position
> >>>>>>>>>>>>>>>> file
> >>>>>>>>>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>>>>>>>> commit() has been called, otherwise it would be
> >>>>>>>> inconsistent
> >>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> data
> >>>>>>>>>>>>>>>>>>>>>>> in the event of a rollback. Consequently, we
> >> need
> >>>> to
> >>>>>> manage
> >>>>>>>>>>>>>>>> these
> >>>>>>>>>>>>>>>>>>>> offsets
> >>>>>>>>>>>>>>>>>>>>>>> the same way we manage the checkpoint offsets,
> >> and
> >>>>>> ensure
> >>>>>>>>>>>>>>>> they're
> >>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>> written on commit().
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 6.
> >>>>>>>>>>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what
> >>>> tests to
> >>>>>>>>>> write.
> >>>>>>>>>>>>>>>> How
> >>>>>>>>>>>>>>>>>>>>>> explicit
> >>>>>>>>>>>>>>>>>>>>>>> do we need to be here in the KIP?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed
> >> to be
> >>>>>>>> seamless,
> >>>>>>>>>>>>>>>> and we
> >>>>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>>>> definitely add some tests around that.
> >> Downgrade,
> >>>> it
> >>>>>>>>>>>>>>>> transpires,
> >>>>>>>>>>>>>>>>>> isn't
> >>>>>>>>>>>>>>>>>>>>>>> currently possible, as the extra column family
> >> for
> >>>>>> offset
> >>>>>>>>>>>>>>>> storage is
> >>>>>>>>>>>>>>>>>>>>>>> incompatible with the pre-KIP-892
> >> implementation:
> >>>> when
> >>>>>> you
> >>>>>>>>>>>>>>>> open a
> >>>>>>>>>>>>>>>>>>>> RocksDB
> >>>>>>>>>>>>>>>>>>>>>>> database, you must open all available column
> >>>> families
> >>>>>> or
> >>>>>>>>>>>>>>>> receive an
> >>>>>>>>>>>>>>>>>>>>>> error.
> >>>>>>>>>>>>>>>>>>>>>>> What currently happens on downgrade is that it
> >>>>>> attempts to
> >>>>>>>>>>>>>>>> open the
> >>>>>>>>>>>>>>>>>>>>>> store,
> >>>>>>>>>>>>>>>>>>>>>>> throws an error about the offsets column
> >> family not
> >>>>>> being
> >>>>>>>>>>>>>>>> opened,
> >>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given
> >> that
> >>>>>>>>>> downgrades
> >>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>> uncommon, I think this is acceptable
> >> behaviour, as
> >>>> the
> >>>>>>>>>>>>>>>> end-state is
> >>>>>>>>>>>>>>>>>>>>>>> consistent, even if it results in an
> >> undesirable
> >>>> state
> >>>>>>>>>>>> restore.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Should I document the upgrade/downgrade
> >> behaviour
> >>>>>>>> explicitly
> >>>>>>>>>>>>>>>> in the
> >>>>>>>>>>>>>>>>>>>> KIP?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> >>>>>>>>>>>>>>>> cadonna@apache.org>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Hi Nick!
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates!
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> >>>>>>>>>>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> >>>>>>>>>>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore
> >>>> after
> >>>>>> this
> >>>>>>>> KIP
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>> released, I would rather throw an unsupported
> >>>>>> operation
> >>>>>>>>>>>>>>>> exception
> >>>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>>>>> default.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>>>>>>>> When would a state store return -1 from
> >>>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
> >> while
> >>>>>> being
> >>>>>>>>>>>>>>>>>>>> transactional?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Wouldn't
> >>>> StateStore#approximateNumUncommittedBytes()
> >>>>>> also
> >>>>>>>>>>>>>>>> return 0
> >>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>> the state store is transactional but nothing
> >> has
> >>>> been
> >>>>>>>>>> written
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> state store yet?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this
> >> KIP
> >>>> really
> >>>>>>>> need
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> introduce
> >>>>>>>>>>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> >>>> StateStoreContext
> >>>>>> has
> >>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>>>>>>> appConfigs() which basically exposes the same
> >>>>>> information,
> >>>>>>>>>>>>>>>> i.e., if
> >>>>>>>>>>>>>>>>>>>> EOS
> >>>>>>>>>>>>>>>>>>>>>>>> is enabled or not.
> >>>>>>>>>>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore
> >>>> interface
> >>>>>> as
> >>>>>>>>>>>>>>>> loosely
> >>>>>>>>>>>>>>>>>>>> coupled
> >>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> >>>>>> implementers
> >>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>> freedom,
> >>>>>>>>>>>>>>>>>>>>>>>> and reduce the amount of internal knowledge
> >>>> required."
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> While I understand the intent, I doubt that it
> >>>>>> decreases
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> coupling of
> >>>>>>>>>>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> >>>>>>>>>> READ_COMMITTED
> >>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>>> applies to IQ but not to reads by processors.
> >>>> Thus,
> >>>>>>>>>>>>>>>> implementers
> >>>>>>>>>>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>>>>>>>>>> understand how Streams accesses the state
> >> stores.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> I would like to hear what others think about
> >> this.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 4.
> >>>>>>>>>>>>>>>>>>>>>>>> Great exposing new metrics for transactional
> >> state
> >>>>>> stores!
> >>>>>>>>>>>>>>>>>> However, I
> >>>>>>>>>>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate
> >> (in
> >>>> the
> >>>>>>>> docs)
> >>>>>>>>>>>>>>>> the 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 also need to adapt/add
> >> integration
> >>>>>> tests
> >>>>>>>>>>>>>>>> besides
> >>>>>>>>>>>>>>>>>> unit
> >>>>>>>>>>>>>>>>>>>>>>>> tests. Additionally, we probably need
> >> integration
> >>>> or
> >>>>>>>> system
> >>>>>>>>>>>>>>>> tests
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> verify that upgrades and downgrades between
> >>>>>> transactional
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>> non-transactional state stores work as
> >> expected.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in
> >> the
> >>>> KIP,
> >>>>>>>> under
> >>>>>>>>>>>>>>>>>>>> "Rejected
> >>>>>>>>>>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea
> >> worth
> >>>>>> pursuing,
> >>>>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>>>> believe
> >>>>>>>>>>>>>>>>>>>>>>>>> that it's out of the scope of this KIP,
> >> because
> >>>> it
> >>>>>>>> solves a
> >>>>>>>>>>>>>>>>>>>> different
> >>>>>>>>>>>>>>>>>>>>>> set
> >>>>>>>>>>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of
> >> this
> >>>> one
> >>>>>> has
> >>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>> grown
> >>>>>>>>>>>>>>>>>>>>>>>> quite
> >>>>>>>>>>>>>>>>>>>>>>>>> large!
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP (
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>>>>>>>>>>>> )
> >>>>>>>>>>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing
> >> back
> >>>>>> "Atomic
> >>>>>>>>>>>>>>>>>>>> Checkpointing"
> >>>>>>>>>>>>>>>>>>>>>>>> (for
> >>>>>>>>>>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think
> >> the one
> >>>>>> thing
> >>>>>>>>>>>>>>>> missing is
> >>>>>>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>> changes to metrics (notably the store
> >> "flush"
> >>>>>> metrics
> >>>>>>>> will
> >>>>>>>>>>>>>>>> need
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>> renamed to "commit").
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> The reason I brought back Atomic
> >> Checkpointing
> >>>> was
> >>>>>> to
> >>>>>>>>>>>>>>>> decouple
> >>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>>> flush
> >>>>>>>>>>>>>>>>>>>>>>>>>> from store commit. This is important,
> >> because
> >>>> with
> >>>>>>>>>>>>>>>> Transactional
> >>>>>>>>>>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on
> >>>> *every*
> >>>>>> Task
> >>>>>>>>>>>>>>>> commit,
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>> just when the StateStore is closing,
> >> otherwise
> >>>> our
> >>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>> buffer
> >>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>> never be written and persisted, instead
> >> growing
> >>>>>>>> unbounded!
> >>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>> experimented
> >>>>>>>>>>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a
> >> store
> >>>>>> flush
> >>>>>>>>>>>>>>>> whenever
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its
> >>>>>> configured
> >>>>>>>>>>>>>>>> size, but
> >>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer
> >>>> from
> >>>>>> being
> >>>>>>>>>>>>>>>>>> configured
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>> unbounded, and it still would have required
> >>>> explicit
> >>>>>>>>>>>>>>>> flushes of
> >>>>>>>>>>>>>>>>>>>>>> RocksDB,
> >>>>>>>>>>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory
> >>>>>> utilization.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the
> >> "right"
> >>>> way
> >>>>>> to
> >>>>>>>>>>>>>>>> resolve
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog
> >> offsets
> >>>> that
> >>>>>>>>>>>>>>>> correspond
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> most
> >>>>>>>>>>>>>>>>>>>>>>>>>> recently written records are always
> >> atomically
> >>>>>> written
> >>>>>>>> to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> StateStore
> >>>>>>>>>>>>>>>>>>>>>>>>>> (by writing them to the same transaction
> >>>> buffer),
> >>>>>> we can
> >>>>>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>>>>> forcibly
> >>>>>>>>>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk,
> >> letting
> >>>>>> RocksDB
> >>>>>>>>>>>>>>>> flush
> >>>>>>>>>>>>>>>>>> them
> >>>>>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>>>>> when necessary, without losing any of our
> >>>>>> consistency
> >>>>>>>>>>>>>>>> guarantees.
> >>>>>>>>>>>>>>>>>>>> See
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>> updated KIP for more info.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> I have fully implemented these changes,
> >>>> although I'm
> >>>>>>>> still
> >>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>> entirely
> >>>>>>>>>>>>>>>>>>>>>>>>>> happy with the implementation for segmented
> >>>>>> StateStores,
> >>>>>>>>>> so
> >>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>> plan
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests
> >> pass. If
> >>>>>> you'd
> >>>>>>>> like
> >>>>>>>>>>>>>>>> to try
> >>>>>>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>>>>>>>>>> review this highly experimental and
> >> incomplete
> >>>>>> branch,
> >>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>> available
> >>>>>>>>>>>>>>>>>>>>>>>> here:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> >>>>>>>> .
> >>>>>>>>>>>>>>>> Note:
> >>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>> built
> >>>>>>>>>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable
> >> base
> >>>> to
> >>>>>> build
> >>>>>>>>>>>>>>>> and test
> >>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>> on,
> >>>>>>>>>>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples
> >> comparisons
> >>>> in a
> >>>>>>>> live
> >>>>>>>>>>>>>>>>>>>>>> environment. I
> >>>>>>>>>>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's
> >> nearer
> >>>>>>>>>> completion
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> has
> >>>>>>>>>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>>>>>>>>>> proven on our main application.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing
> >> and
> >>>>>> testing:
> >>>>>>>>>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window)
> >>>> stores
> >>>>>>>>>>>>>>>>>>>>>>>>>> - Global stores
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> As I do not currently use either of these,
> >> so my
> >>>>>> primary
> >>>>>>>>>>>>>>>> test
> >>>>>>>>>>>>>>>>>>>>>>>> environment
> >>>>>>>>>>>>>>>>>>>>>>>>>> doesn't test these areas.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next
> >> week
> >>>> for
> >>>>>> a few
> >>>>>>>>>>>>>>>> weeks,
> >>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>> not have time to move this forward until
> >> late
> >>>>>> August.
> >>>>>>>> That
> >>>>>>>>>>>>>>>> said,
> >>>>>>>>>>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just
> >>>> won't be
> >>>>>>>> able
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> respond
> >>>>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>>> quickly as usual.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact
> >> on IQ
> >>>> is
> >>>>>> not
> >>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>> had
> >>>>>>>>>>>>>>>>>>>>>>>>>>> considered.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the state
> >> store
> >>>>>> from the
> >>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> >> the
> >>>>>>>> checkpoint
> >>>>>>>>>>>>>>>> (thus,
> >>>>>>>>>>>>>>>>>>>>>>>> flushing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> >> data
> >>>>>> and/or
> >>>>>>>>>>>>>>>> number of
> >>>>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you
> >> suggesting
> >>>>>> that we
> >>>>>>>>>>>>>>>> add an
> >>>>>>>>>>>>>>>>>>>>>>>> additional
> >>>>>>>>>>>>>>>>>>>>>>>>>>> config for the max number of commit
> >> intervals
> >>>>>> between
> >>>>>>>>>>>>>>>>>> checkpoints?
> >>>>>>>>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the
> >>>>>> transaction
> >>>>>>>>>>>>>>>> buffers
> >>>>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>>>>>> nearly
> >>>>>>>>>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of
> >> commit
> >>>>>>>> intervals
> >>>>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>>>>>>>>> elapsed,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> whichever comes first?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although
> >> this
> >>>>>>>> re-ignites
> >>>>>>>>>>>>>>>> an
> >>>>>>>>>>>>>>>>>>>> earlier
> >>>>>>>>>>>>>>>>>>>>>>>>>>> debate about whether a config should be
> >>>> measured in
> >>>>>>>>>>>>>>>> "number of
> >>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute
> >> time.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the
> >> reason
> >>>> I
> >>>>>> was
> >>>>>>>>>>>>>>>> pursuing
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> Atomic
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable
> >> flush
> >>>> from
> >>>>>>>>>>>>>>>> checkpointing,
> >>>>>>>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every
> >> commit
> >>>>>> without
> >>>>>>>> any
> >>>>>>>>>>>>>>>>>>>> performance
> >>>>>>>>>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely
> >> the
> >>>>>> "best"
> >>>>>>>>>>>>>>>> solution,
> >>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>> I'm not
> >>>>>>>>>>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back
> >> into
> >>>> this
> >>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
> >>>>>> transactional
> >>>>>>>>>>>> logic
> >>>>>>>>>>>>>>>>>>>> directly
> >>>>>>>>>>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with
> >> the
> >>>>>>>>>>>>>>>>>>>>>> StateStore#newTransaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>> method, and reduces the number of new
> >> classes
> >>>>>>>> introduced,
> >>>>>>>>>>>>>>>>>>>>>> significantly
> >>>>>>>>>>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and
> >> the
> >>>>>>>> complexity
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> drastically
> >>>>>>>>>>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic
> >>>> Checkpoints
> >>>>>>>> into
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna
> >> <
> >>>>>>>>>>>>>>>> cadonna@apache.org>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> As far as I understand, you want to
> >> atomically
> >>>>>> update
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the
> >>>> memtable
> >>>>>> of a
> >>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit
> >> time
> >>>>>> elapsed
> >>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>> after
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that
> >> would
> >>>> lead
> >>>>>> to
> >>>>>>>>>>>>>>>> exceeding
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> >>>> before the
> >>>>>>>> next
> >>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>>> interval
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> ends.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would
> >> commit
> >>>>>> every
> >>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>> interval
> >>>>>>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store will only be atomically
> >>>> updated
> >>>>>>>> roughly
> >>>>>>>>>>>>>>>> every
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of
> >>>> data.
> >>>>>> Also
> >>>>>>>> IQ
> >>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>> then
> >>>>>>>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> see new data roughly every
> >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> After a failure the state store needs to
> >>>> restore
> >>>>>> up to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Is this correct?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the state
> >> store
> >>>>>> from
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> >> the
> >>>>>>>> checkpoint
> >>>>>>>>>>>>>>>> (thus,
> >>>>>>>>>>>>>>>>>>>>>>>> flushing
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> >> data
> >>>>>> and/or
> >>>>>>>>>>>>>>>> number of
> >>>>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have
> >> the
> >>>> same
> >>>>>> delay
> >>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because
> >> both
> >>>>>> would
> >>>>>>>>>>>>>>>> appear
> >>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However,
> >>>> after a
> >>>>>>>> failure
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> still needs to restore up to
> >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> it might restore data that is already in
> >> the
> >>>> state
> >>>>>>>> store
> >>>>>>>>>>>>>>>>>> because
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable
> >> offset
> >>>>>> (i.e.
> >>>>>>>> the
> >>>>>>>>>>>>>>>> last
> >>>>>>>>>>>>>>>>>>>>>> committed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring
> >>>> data
> >>>>>> that
> >>>>>>>> is
> >>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should
> >> not
> >>>>>> violated.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> This solution needs at least one new
> >> config to
> >>>>>> specify
> >>>>>>>>>>>>>>>> when a
> >>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> should be written.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail
> >>>> that
> >>>>>> does
> >>>>>>>> not
> >>>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>>>>>>>> anything
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit
> >>>> interval
> >>>>>> is 30
> >>>>>>>>>>>>>>>> seconds,
> >>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>> five
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> seconds.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch
> >> on
> >>>> our
> >>>>>>>> staging
> >>>>>>>>>>>>>>>>>>>>>> environment,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
> >>>> non-trivial
> >>>>>>>>>>>>>>>> application.
> >>>>>>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate
> >> that
> >>>> we
> >>>>>> saw
> >>>>>>>> the
> >>>>>>>>>>>>>>>> last
> >>>>>>>>>>>>>>>>>>>> time
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP,
> >> but
> >>>> this
> >>>>>>>> time,
> >>>>>>>>>> I
> >>>>>>>>>>>>>>>>>> think I
> >>>>>>>>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> why.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit,
> >> which is
> >>>>>> called
> >>>>>>>> at
> >>>>>>>>>>>>>>>> the end
> >>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Task
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit process, has the following
> >> behaviour:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under ALOS: checkpoint the
> >> state
> >>>>>> stores.
> >>>>>>>>>> This
> >>>>>>>>>>>>>>>>>> includes
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            flushing memtables in RocksDB.
> >>>> This is
> >>>>>>>>>>>> acceptable
> >>>>>>>>>>>>>>>>>>>> because the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> default
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit.interval.ms is 5
> >> seconds,
> >>>> so
> >>>>>>>> forcibly
> >>>>>>>>>>>>>>>> flushing
> >>>>>>>>>>>>>>>>>>>>>> memtables
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> every 5
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            seconds is acceptable for most
> >>>>>>>> applications.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under EOS: checkpointing is
> >> not
> >>>> done,
> >>>>>>>>>> *unless*
> >>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>> being
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> forced, due
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to e.g. the Task closing or
> >> being
> >>>>>> revoked.
> >>>>>>>>>> This
> >>>>>>>>>>>>>>>> means
> >>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> under
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> normal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing conditions, the
> >> state
> >>>> stores
> >>>>>>>> will
> >>>>>>>>>> not
> >>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>> checkpointed,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> and will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            not have memtables flushed at
> >> all ,
> >>>>>> unless
> >>>>>>>>>>>> RocksDB
> >>>>>>>>>>>>>>>>>>>> decides to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush them on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            its own. Checkpointing stores
> >> and
> >>>>>>>>>> force-flushing
> >>>>>>>>>>>>>>>> their
> >>>>>>>>>>>>>>>>>>>>>> memtables
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            done when a Task is being
> >> closed.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint
> >>>> stores on
> >>>>>> at
> >>>>>>>>>>>> least
> >>>>>>>>>>>>>>>>>> *some*
> >>>>>>>>>>>>>>>>>>>>>>>> normal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the
> >> RocksDB
> >>>>>>>> transaction
> >>>>>>>>>>>>>>>>>> buffers
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets
> >> are
> >>>>>> synced to
> >>>>>>>>>>>>>>>> disk to
> >>>>>>>>>>>>>>>>>>>>>> prevent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
> >>>> Consequently,
> >>>>>> my
> >>>>>>>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task
> >> commit,
> >>>>>> which
> >>>>>>>> is
> >>>>>>>>>>>>>>>> far too
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> frequent.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000
> >> records,
> >>>>>> which
> >>>>>>>> is
> >>>>>>>>>> a
> >>>>>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> flush
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
> >> performance
> >>>>>> problems
> >>>>>>>> for
> >>>>>>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> applications.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and
> >> I'm
> >>>>>>>> currently
> >>>>>>>>>>>>>>>> leaning
> >>>>>>>>>>>>>>>>>>>>>>>> towards
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> using the
> >>>> statestore.transaction.buffer.max.bytes
> >>>>>>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to
> >>>> exceed it.
> >>>>>>>> This
> >>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> complement the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
> >> functionality
> >>>> that
> >>>>>> this
> >>>>>>>>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Currently, we use
> >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> force an
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            early Task commit if processing
> >>>> more
> >>>>>>>> records
> >>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>> cause
> >>>>>>>>>>>>>>>>>>>> our
> >>>>>>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            transactions to exceed the
> >> memory
> >>>>>> assigned
> >>>>>>>> to
> >>>>>>>>>>>>>>>> them.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - New functionality: when a
> >> Task
> >>>> *does*
> >>>>>>>>>> commit,
> >>>>>>>>>>>>>>>> we will
> >>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the stores (and hence flush the
> >>>>>> transaction
> >>>>>>>>>>>>>>>> buffers)
> >>>>>>>>>>>>>>>>>>>> unless
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> expect to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            cross the
> >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> >>>>>>>>>>>>>>>>>>>> threshold
> >>>>>>>>>>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the next
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick
> >> Telford <
> >>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I
> >> meant in
> >>>>>> terms of
> >>>>>>>>>>>>>>>>>>>> understanding
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance profile under various
> >>>>>> circumstances. The
> >>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>> complex
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would
> >> be for
> >>>>>> users
> >>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance they see. For example,
> >> spilling
> >>>>>> records
> >>>>>>>> to
> >>>>>>>>>>>>>>>> disk
> >>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold
> >>>> would, I
> >>>>>>>>>> expect,
> >>>>>>>>>>>>>>>>>> reduce
> >>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
> >>>> throughput
> >>>>>> could
> >>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>> unexpected,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> >>>> diagnose/understand for
> >>>>>>>>>> users.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early
> >> commit"
> >>>>>> concept is
> >>>>>>>>>>>>>>>>>> relatively
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document,
> >> and
> >>>>>>>>>>>> conceptually
> >>>>>>>>>>>>>>>>>> fairly
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> obvious to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to
> >>>> make it
> >>>>>>>>>> easier
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it happens though.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an
> >>>> indirect
> >>>>>> way
> >>>>>>>> of
> >>>>>>>>>>>>>>>>>> telling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
> >>>>>> READ_COMMITTED
> >>>>>>>>>>>>>>>>>>>> essentially
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> requires
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that
> >> two
> >>>>>> threads
> >>>>>>>>>>>>>>>> calling
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes
> >>>> from
> >>>>>> the
> >>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> >>>>>> READ_UNCOMMITTED, all
> >>>>>>>>>>>>>>>> bets are
> >>>>>>>>>>>>>>>>>>>> off,
> >>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe
> >> written
> >>>>>> records
> >>>>>>>> at
> >>>>>>>>>>>>>>>> any
> >>>>>>>>>>>>>>>>>>>> time,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> which is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That
> >> said,
> >>>>>>>> StateStores
> >>>>>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>> free
> >>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> implement
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these guarantees however they can,
> >> which is
> >>>> a
> >>>>>> bit
> >>>>>>>> more
> >>>>>>>>>>>>>>>>>> relaxed
> >>>>>>>>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions".
> >> For
> >>>>>> example,
> >>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>> RocksDB
> >>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED ==
> >>>> WBWI-based
> >>>>>>>>>>>>>>>>>> "transactions",
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the
> >>>>>> database.
> >>>>>>>> But
> >>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> storage
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to
> >> *always*
> >>>> use
> >>>>>>>>>>>>>>>> transactions,
> >>>>>>>>>>>>>>>>>>>> even
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage
> >> engines
> >>>>>> that
> >>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>> provide
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation
> >> guarantees
> >>>> can
> >>>>>> be
> >>>>>>>> met
> >>>>>>>>>>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> different
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> technique.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the
> >> StateStore
> >>>>>> interface
> >>>>>>>> as
> >>>>>>>>>>>>>>>>>> loosely
> >>>>>>>>>>>>>>>>>>>>>>>> coupled
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to
> >> give
> >>>>>>>>>>>>>>>> implementers
> >>>>>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> freedom, and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge
> >>>>>> required.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> >>>> "IsolationLevel"
> >>>>>> might
> >>>>>>>>>> not
> >>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it
> >> much
> >>>> more
> >>>>>>>>>>>>>>>> explicit if
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> required, e.g.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7-8.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I can make these changes either later
> >> today
> >>>> or
> >>>>>>>>>>>> tomorrow.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Small update:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and
> >> fixed a
> >>>>>> bunch of
> >>>>>>>>>>>>>>>> issues
> >>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests
> >> pass,
> >>>>>> which is
> >>>>>>>>>>>>>>>>>> promising,
> >>>>>>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to undergo some performance
> >> testing. I
> >>>>>> haven't
> >>>>>>>>>>>>>>>> (yet)
> >>>>>>>>>>>>>>>>>>>> worked
> >>>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff,
> >> but I
> >>>>>> would
> >>>>>>>>>>>>>>>> expect
> >>>>>>>>>>>>>>>>>> that,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
> >>>> difference. The
> >>>>>>>>>> branch
> >>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> available
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> >>>>>>>>>> if
> >>>>>>>>>>>>>>>>>> anyone
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno
> >> Cadonna
> >>>> <
> >>>>>>>>>>>>>>>>>>>> cadonna@apache.org>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was
> >> actually
> >>>> also
> >>>>>> my
> >>>>>>>>>>>>>>>> point. I
> >>>>>>>>>>>>>>>>>>>>>>>> understood
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion
> >> path
> >>>> as
> >>>>>> a way
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> early
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the
> >>>> intent.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually
> >> it is
> >>>>>> public
> >>>>>>>>>>>>>>>> API. My
> >>>>>>>>>>>>>>>>>>>>>> question
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal
> >>>>>> processing.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the
> >> size
> >>>> of the
> >>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>>> buffer
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit
> >> affects
> >>>> the
> >>>>>> whole
> >>>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a
> >> state
> >>>> store
> >>>>>> is
> >>>>>>>> not
> >>>>>>>>>>>>>>>>>>>> confined to
> >>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and
> >>>>>> changes the
> >>>>>>>>>>>>>>>> behavior
> >>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you
> >> that
> >>>> it
> >>>>>> is a
> >>>>>>>>>>>>>>>> decent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> compromise. I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides
> >> and
> >>>> list
> >>>>>> the
> >>>>>>>>>>>>>>>> options to
> >>>>>>>>>>>>>>>>>>>>>>>> overcome
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all
> >>>> options
> >>>>>> seem
> >>>>>>>>>>>> quite
> >>>>>>>>>>>>>>>>>> heavy
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> compared
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand
> >> what you
> >>>>>> mean
> >>>>>>>> with
> >>>>>>>>>>>>>>>> "less
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> predictable
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for users", though.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
> >>>> alternatives
> >>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>>> interesting.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> But I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good
> >> and
> >>>> we
> >>>>>> should
> >>>>>>>>>>>>>>>> continue
> >>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>> it!
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my
> >> e-mail on
> >>>>>> June
> >>>>>>>>>> 20th:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning
> >> behind
> >>>>>> putting
> >>>>>>>>>>>>>>>> isolation
> >>>>>>>>>>>>>>>>>>>>>> level
> >>>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should
> >>>> that
> >>>>>> also
> >>>>>>>> be
> >>>>>>>>>> a
> >>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>> give
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to
> >> decide
> >>>>>> whether
> >>>>>>>> to
> >>>>>>>>>>>>>>>> turn on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned
> >> about
> >>>>>> how do
> >>>>>>>>>> you
> >>>>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>> if a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written
> >> under
> >>>> EOS,
> >>>>>> if
> >>>>>>>> you
> >>>>>>>>>>>>>>>> do not
> >>>>>>>>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
> >>>> transactional or
> >>>>>> not.
> >>>>>>>>>> If
> >>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can
> >> be
> >>>>>> written
> >>>>>>>>>>>>>>>> during
> >>>>>>>>>>>>>>>>>>>> normal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state
> >> store
> >>>> is not
> >>>>>>>>>>>>>>>>>> transactional,
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written
> >> under
> >>>> EOS.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My point was about not only
> >> considering the
> >>>>>> bytes
> >>>>>>>> in
> >>>>>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>> config
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but
> >> also
> >>>>>> bytes
> >>>>>>>> that
> >>>>>>>>>>>>>>>> might
> >>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> spilled
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering
> >>>> whether you
> >>>>>>>>>> should
> >>>>>>>>>>>>>>>>>> remove
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory
> >>>> bytes to
> >>>>>> be
> >>>>>>>>>> used
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store
> >> records." My
> >>>>>>>> thinking
> >>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> even
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to
> >>>> disk,
> >>>>>>>>>> limiting
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> overall
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> bytes
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it
> >> again
> >>>> and
> >>>>>>>>>>>>>>>> considering
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>> recent
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much
> >>>> sense
> >>>>>>>> anymore.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I like the name
> >>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes that
> >>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 8.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> >>>>>> implementation
> >>>>>>>>>>>>>>>> details) of
> >>>>>>>>>>>>>>>>>>>> how
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of
> >> changelog
> >>>>>>>>>>>>>>>> transactions,
> >>>>>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be
> >>>> great.
> >>>>>>>> Would
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> great
> >>>>>>>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the
> >>>>>> behavior in
> >>>>>>>>>>>>>>>> case of
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional
> >> state
> >>>>>> store
> >>>>>>>>>>>>>>>> recover
> >>>>>>>>>>>>>>>>>>>> after a
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
> >>>> transaction
> >>>>>>>> buffer,
> >>>>>>>>>>>>>>>> etc.
> >>>>>>>>>>>>>>>>>>>> (that
> >>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> >>>>>>>>>>>>>>>> WriteBatchWithIndex
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to
> >>>> exceed)
> >>>>>>>>>>>>>>>> configured
> >>>>>>>>>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> needs to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
> >>>> Ultimately,
> >>>>>> any
> >>>>>>>>>>>>>>>> approach
> >>>>>>>>>>>>>>>>>>>> based
> >>>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cracking
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in
> >>>> ways
> >>>>>> it's
> >>>>>>>> not
> >>>>>>>>>>>>>>>> really
> >>>>>>>>>>>>>>>>>>>>>>>> designed
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for is
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen
> >> performance
> >>>> or
> >>>>>>>>>>>>>>>> consistency
> >>>>>>>>>>>>>>>>>>>> issues.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing
> >> these
> >>>>>> early
> >>>>>>>>>>>>>>>> commits?
> >>>>>>>>>>>>>>>>>>>> While
> >>>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideal, I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to
> >>>> ensure
> >>>>>>>>>>>>>>>> consistency
> >>>>>>>>>>>>>>>>>>>> whilst
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maintaining
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem
> >> *very*
> >>>>>>>>>>>>>>>> complicated, and
> >>>>>>>>>>>>>>>>>>>> might
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for
> >> users
> >>>> as a
> >>>>>>>>>>>>>>>> consequence.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of
> >> this
> >>>>>> KIP is
> >>>>>>>>>>>>>>>> growing a
> >>>>>>>>>>>>>>>>>>>> bit
> >>>>>>>>>>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss
> >> ideas
> >>>> for
> >>>>>>>> future
> >>>>>>>>>>>>>>>>>>>>>>>> improvements, I
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
> >>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope
> >> down
> >>>> to a
> >>>>>>>> design
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> achieves
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> most
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
> >>>>>> restorations
> >>>>>>>>>>>>>>>> during
> >>>>>>>>>>>>>>>>>> dirty
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design
> >> that
> >>>>>> this KIP
> >>>>>>>>>>>>>>>> produces
> >>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ultimately
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially
> >> if
> >>>> the
> >>>>>> bulk
> >>>>>>>> of
> >>>>>>>>>>>>>>>> it is
> >>>>>>>>>>>>>>>>>>>>>> internal
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week
> >>>> trying
> >>>>>> to
> >>>>>>>>>>>>>>>> re-work
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove
> >> the
> >>>>>>>>>>>>>>>> newTransaction()
> >>>>>>>>>>>>>>>>>>>>>> method,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> such
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
> >>>>>>>> RocksDBStore.
> >>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>>> way, if
> >>>>>>>>>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the
> >> future,
> >>>>>> like
> >>>>>>>> the
> >>>>>>>>>>>>>>>> SST
> >>>>>>>>>>>>>>>>>> file
> >>>>>>>>>>>>>>>>>>>>>>>>>>>> management
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so
> >> with
> >>>>>> little/no
> >>>>>>>>>>>> API
> >>>>>>>>>>>>>>>>>>>> changes.
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>>
> >>>>
> >>
> >

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

Posted by Bruno Cadonna <ca...@apache.org>.
Hi Nick,

What you and Lucas wrote about the different configurations of ALOS/EOS 
and READ_COMMITTED/READ_UNCOMMITTED make sense to me. My earlier 
concerns about changelogs diverging from the content of the local state 
stores turned out to not apply. So I think, we can move on with those 
configurations.

Regarding the TaskCorruptedException and wiping out the state stores 
under EOS, couldn't we abort the transaction on the state store and 
close the task dirty? If the Kafka transaction was indeed committed, the 
store would restore the missing part from the changelog topic. If the 
Kafka transaction was not committed, changelog topic and state store are 
in-sync.

In any case, IMO those are implementation details that we do not need to 
discuss and solve in the KIP discussion. We can solve them on the PR. 
The important thing is that the processing guarantees hold.

Best,
Bruno

On 10/18/23 3:56 PM, Nick Telford wrote:
> Hi Lucas,
> 
> TaskCorruptedException is how Streams signals that the Task state needs to
> be wiped, so we can't retain that exception without also wiping state on
> timeouts.
> 
> Regards,
> Nick
> 
> On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <lb...@confluent.io.invalid>
> wrote:
> 
>> Hi Nick,
>>
>> I think indeed the better behavior would be to retry commitTransaction
>> until we risk running out of time to meet `max.poll.interval.ms`.
>>
>> However, if it's handled as a `TaskCorruptedException` at the moment,
>> I would do the same in this KIP, and leave exception handling
>> improvements to future work. 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 <ni...@gmail.com>
>> wrote:
>>>
>>> Hi Lucas,
>>>
>>> Yeah, this is pretty much the direction I'm thinking of going in now. You
>>> make an interesting point about committing on-error under
>>> ALOS/READ_COMMITTED, although I haven't had a chance to think through the
>>> implications yet.
>>>
>>> Something that I ran into earlier this week is an issue with the new
>>> handling of TimeoutException. Without TX stores, TimeoutException under
>> EOS
>>> throws a TaskCorruptedException, which wipes the stores. However, with TX
>>> stores, TimeoutException is now just bubbled up and dealt with as it is
>>> under ALOS. The problem arises when the Producer#commitTransaction call
>>> times out: Streams attempts to ignore the error and continue producing,
>>> which causes the next call to Producer#send to throw
>>> "IllegalStateException: Cannot attempt operation `send` because the
>>> previous call to `commitTransaction` timed out and must be retried".
>>>
>>> I'm not sure what we should do here: retrying the commitTransaction seems
>>> logical, but what if it times out again? Where do we draw the line and
>>> shutdown the instance?
>>>
>>> Regards,
>>> Nick
>>>
>>> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <lbrutschy@confluent.io
>> .invalid>
>>> wrote:
>>>
>>>> Hi all,
>>>>
>>>> I think I liked your suggestion of allowing EOS with READ_UNCOMMITTED,
>>>> but keep wiping the state on error, and I'd vote for this solution
>>>> when introducing `default.state.isolation.level`. This way, we'd have
>>>> the most low-risk roll-out of this feature (no behavior change without
>>>> reconfiguration), with the possibility of switching to the most sane /
>>>> battle-tested default settings in 4.0. Essentially, we'd have a
>>>> feature flag but call it `default.state.isolation.level` and don't
>>>> have to deprecate it later.
>>>>
>>>> So the possible configurations would then be this:
>>>>
>>>> 1. ALOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
>>>> reads from DB.
>>>> 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
>>>> WriteBatch/DB. Flush on error (see note below).
>>>> 3. EOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
>>>> reads from DB. Wipe state on error.
>>>> 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
>>>> WriteBatch/DB.
>>>>
>>>> I believe the feature is important enough that we will see good
>>>> adoption even without changing the default. In 4.0, when we have seen
>>>> this being adopted and is battle-tested, we make READ_COMMITTED the
>>>> default for EOS, or even READ_COMITTED always the default, depending
>>>> on our experiences. And we could add a clever implementation of
>>>> READ_UNCOMITTED with WriteBatches later.
>>>>
>>>> The only smell here is that `default.state.isolation.level` wouldn't
>>>> be purely an IQ setting, but it would also (slightly) change the
>>>> behavior of the processing, but that seems unavoidable as long as we
>>>> haven't solve READ_UNCOMITTED IQ with WriteBatches.
>>>>
>>>> Minor: As for Bruno's point 4, I think if we are concerned about this
>>>> 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).
>>>>
>>>> Cheers,
>>>> Lucas
>>>>
>>>> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <ni...@gmail.com>
>>>> wrote:
>>>>>
>>>>> Hi Guozhang,
>>>>>
>>>>> The KIP as it stands introduces a new configuration,
>>>>> default.state.isolation.level, which is independent of
>> processing.mode.
>>>>> It's intended that this new configuration be used to configure a
>> global
>>>> IQ
>>>>> isolation level in the short term, with a future KIP introducing the
>>>>> capability to change the isolation level on a per-query basis,
>> falling
>>>> back
>>>>> to the "default" defined by this config. That's why I called it
>>>> "default",
>>>>> for future-proofing.
>>>>>
>>>>> However, it currently includes the caveat that READ_UNCOMMITTED is
>> not
>>>>> available under EOS. I think this is the coupling you are alluding
>> to?
>>>>>
>>>>> This isn't intended to be a restriction of the API, but is currently
>> a
>>>>> technical limitation. However, after discussing with some users about
>>>>> use-cases that would require READ_UNCOMMITTED under EOS, I'm
>> inclined to
>>>>> remove that clause and put in the necessary work to make that
>> combination
>>>>> possible now.
>>>>>
>>>>> I currently see two possible approaches:
>>>>>
>>>>>     1. Disable TX StateStores internally when the IsolationLevel is
>>>>>     READ_UNCOMMITTED and the processing.mode is EOS. This is more
>>>> difficult
>>>>>     than it sounds, as there are many assumptions being made
>> throughout
>>>> the
>>>>>     internals about the guarantees StateStores provide. It would
>>>> definitely add
>>>>>     a lot of extra "if (read_uncommitted && eos)" branches,
>> complicating
>>>>>     maintenance and testing.
>>>>>     2. Invest the time *now* to make READ_UNCOMMITTED of EOS
>> StateStores
>>>>>     possible. I have some ideas on how this could be achieved, but
>> they
>>>> would
>>>>>     need testing and could introduce some additional issues. The
>> benefit
>>>> of
>>>>>     this approach is that it would make query-time IsolationLevels
>> much
>>>> simpler
>>>>>     to implement in the future.
>>>>>
>>>>> Unfortunately, both will require considerable work that will further
>>>> delay
>>>>> this KIP, which was the reason I placed the restriction in the KIP
>> in the
>>>>> first place.
>>>>>
>>>>> Regards,
>>>>> Nick
>>>>>
>>>>> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
>> guozhang.wang.us@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hello Nick,
>>>>>>
>>>>>> First of all, thanks a lot for the great effort you've put in
>> driving
>>>>>> this KIP! I really like it coming through finally, as many people
>> in
>>>>>> the community have raised this. At the same time I honestly feel a
>> bit
>>>>>> ashamed for not putting enough of my time supporting it and
>> pushing it
>>>>>> through the finish line (you raised this KIP almost a year ago).
>>>>>>
>>>>>> I briefly passed through the DISCUSS thread so far, not sure I've
>> 100
>>>>>> percent digested all the bullet points. But with the goal of
>> trying to
>>>>>> help take it through the finish line in mind, I'd want to throw
>>>>>> thoughts on top of my head only on the point #4 above which I felt
>> may
>>>>>> be the main hurdle for the current KIP to drive to a consensus now.
>>>>>>
>>>>>> The general question I asked myself is, whether we want to couple
>> "IQ
>>>>>> reading mode" with "processing mode". While technically I tend to
>>>>>> agree with you that, it's feels like a bug if some single user
>> chose
>>>>>> "EOS" for processing mode while choosing "read uncommitted" for IQ
>>>>>> reading mode, at the same time, I'm thinking if it's possible that
>>>>>> there could be two different persons (or even two teams) that
>> would be
>>>>>> using the stream API to build the app, and the IQ API to query the
>>>>>> running state of the app. I know this is less of a technical thing
>> but
>>>>>> rather a more design stuff, but if it could be ever the case, I'm
>>>>>> wondering if the personale using the IQ API knows about the risks
>> of
>>>>>> using read uncommitted but still chose so for the favor of
>>>>>> performance, no matter if the underlying stream processing mode
>>>>>> configured by another personale is EOS or not. In that regard, I'm
>>>>>> leaning towards a "leaving the door open, and close it later if we
>>>>>> found it's a bad idea" aspect with a configuration 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 can still enforce
>>>>>> coupling them.
>>>>>>
>>>>>> Again, just my 2c here. Thanks again for the great patience and
>>>>>> diligence on this KIP.
>>>>>>
>>>>>>
>>>>>> Guozhang
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
>> nick.telford@gmail.com>
>>>>>> wrote:
>>>>>>>
>>>>>>> Hi Bruno,
>>>>>>>
>>>>>>> 4.
>>>>>>> I'll hold off on making that change until we have a consensus as
>> to
>>>> what
>>>>>>> configuration to use to control all of this, as it'll be
>> affected by
>>>> the
>>>>>>> decision on EOS isolation levels.
>>>>>>>
>>>>>>> 5.
>>>>>>> Done. I've chosen "committedOffsets".
>>>>>>>
>>>>>>> Regards,
>>>>>>> Nick
>>>>>>>
>>>>>>> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <ca...@apache.org>
>>>> wrote:
>>>>>>>
>>>>>>>> Hi Nick,
>>>>>>>>
>>>>>>>> 1.
>>>>>>>> Yeah, you are probably right that it does not make too much
>> sense.
>>>>>>>> Thanks for the clarification!
>>>>>>>>
>>>>>>>>
>>>>>>>> 4.
>>>>>>>> Yes, sorry for the back and forth, but I think for the sake of
>> the
>>>> KIP
>>>>>>>> it is better to let the ALOS behavior as it is for now due to
>> the
>>>>>>>> possible issues you would run into. Maybe we can find a
>> solution
>>>> in the
>>>>>>>> future. Now the question returns to whether we really need
>>>>>>>> default.state.isolation.level. Maybe the config could be the
>>>> feature
>>>>>>>> flag Sophie requested.
>>>>>>>>
>>>>>>>>
>>>>>>>> 5.
>>>>>>>> There 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 committedOffsetFor().
>>>>>>>>
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Bruno
>>>>>>>>
>>>>>>>>
>>>>>>>> On 10/13/23 3:21 PM, Nick Telford wrote:
>>>>>>>>> Hi Bruno,
>>>>>>>>>
>>>>>>>>> Thanks for getting back to me.
>>>>>>>>>
>>>>>>>>> 1.
>>>>>>>>> I think this should be possible. Are you thinking of the
>>>> situation
>>>>>> where
>>>>>>>> a
>>>>>>>>> user may downgrade to a previous version of Kafka Streams? In
>>>> that
>>>>>> case,
>>>>>>>>> sadly, the RocksDBStore would get wiped by the older version
>> of
>>>> Kafka
>>>>>>>>> Streams anyway, because that version wouldn't understand the
>>>> extra
>>>>>> column
>>>>>>>>> family (that holds offsets), so the missing Position file
>> would
>>>>>>>>> automatically get rebuilt when the store is rebuilt from the
>>>>>> changelog.
>>>>>>>>> Are there other situations than downgrade where a
>> transactional
>>>> store
>>>>>>>> could
>>>>>>>>> be replaced by a non-transactional one? I can't think of any.
>>>>>>>>>
>>>>>>>>> 2.
>>>>>>>>> Ahh yes, the Test Plan - my Kryptonite! This section
>> definitely
>>>>>> needs to
>>>>>>>> be
>>>>>>>>> fleshed out. I'll work on that. How much detail do you need?
>>>>>>>>>
>>>>>>>>> 3.
>>>>>>>>> See my previous email discussing this.
>>>>>>>>>
>>>>>>>>> 4.
>>>>>>>>> Hmm, this is an interesting point. Are you suggesting that
>> under
>>>> ALOS
>>>>>>>>> READ_COMMITTED should not be supported?
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>> Nick
>>>>>>>>>
>>>>>>>>> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
>> cadonna@apache.org>
>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Nick,
>>>>>>>>>>
>>>>>>>>>> I think the KIP is converging!
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 1.
>>>>>>>>>> I am wondering whether it makes sense to write the position
>> file
>>>>>> during
>>>>>>>>>> close as we do for the checkpoint file, so that in case the
>>>> state
>>>>>> store
>>>>>>>>>> is replaced with a non-transactional state store the
>>>>>> non-transactional
>>>>>>>>>> state store finds the position file. I think, this is not
>>>> strictly
>>>>>>>>>> needed, but would be a nice behavior instead of just
>> deleting
>>>> the
>>>>>>>>>> position file.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 2.
>>>>>>>>>> The test plan does not mention integration tests. Do you not
>>>> need to
>>>>>>>>>> extend existing ones and add new ones. Also for upgrading
>> and
>>>>>>>>>> downgrading you might need integration and/or system tests.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 3.
>>>>>>>>>> I think Sophie made a point. Although, IQ reading from
>>>> uncommitted
>>>>>> data
>>>>>>>>>> under EOS might be considered a bug by some people. Thus,
>> your
>>>> KIP
>>>>>> would
>>>>>>>>>> fix a bug rather than changing the intended behavior.
>> However, I
>>>>>> also
>>>>>>>>>> see that a feature flag would help users that rely on this
>> buggy
>>>>>>>>>> behavior (at least until AK 4.0).
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 4.
>>>>>>>>>> This is related to the previous point. I assume that the
>>>> difference
>>>>>>>>>> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is
>> that in
>>>> the
>>>>>>>>>> former you enable transactions on the state store and in the
>>>> latter
>>>>>> you
>>>>>>>>>> disable them. If my assumption is correct, I think that is
>> an
>>>> issue.
>>>>>>>>>> Let's assume under ALOS Streams fails over a couple of times
>>>> more or
>>>>>>>>>> less at the same step in processing after value 3 is added
>> to an
>>>>>>>>>> aggregation but the offset of the corresponding input record
>>>> was not
>>>>>>>>>> committed. Without transactions disabled, the aggregation
>> value
>>>>>> would
>>>>>>>>>> increase by 3 for each failover. With transactions enabled,
>>>> value 3
>>>>>>>>>> would only be added to the aggregation once when the offset
>> of
>>>> the
>>>>>> input
>>>>>>>>>> record is committed and the transaction finally completes.
>> So
>>>> the
>>>>>>>>>> content of the state store would change depending on the
>>>>>> configuration
>>>>>>>>>> for IQ. IMO, the content of the state store should be
>>>> independent
>>>>>> from
>>>>>>>>>> IQ. Given this issue, I propose to not use transactions with
>>>> ALOS at
>>>>>>>>>> all. I was a big proponent of using transactions with ALOS,
>> but
>>>> I
>>>>>>>>>> realized that transactions with ALOS is not as easy as
>> enabling
>>>>>>>>>> transactions on state stores. Another aspect that is
>>>> problematic is
>>>>>> that
>>>>>>>>>> the changelog topic which actually replicates the state
>> store
>>>> is not
>>>>>>>>>> transactional under ALOS. Thus, it might happen that the
>> state
>>>>>> store and
>>>>>>>>>> the changelog differ in their content. All of this is maybe
>>>> solvable
>>>>>>>>>> somehow, but for the sake of this KIP, I would leave it for
>> the
>>>>>> future.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> Bruno
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
>>>>>>>>>>> Hey Nick! First of all thanks for taking up this awesome
>>>> feature,
>>>>>> I'm
>>>>>>>>>> sure
>>>>>>>>>>> every single
>>>>>>>>>>> Kafka Streams user and dev would agree that it is sorely
>>>> needed.
>>>>>>>>>>>
>>>>>>>>>>> I've just been catching up on the KIP and surrounding
>>>> discussion,
>>>>>> so
>>>>>>>>>> please
>>>>>>>>>>> forgive me
>>>>>>>>>>> for any misunderstandings or misinterpretations of the
>> current
>>>>>> plan and
>>>>>>>>>>> don't hesitate to
>>>>>>>>>>> correct me.
>>>>>>>>>>>
>>>>>>>>>>> Before I jump in, I just want to say that having seen this
>>>> drag on
>>>>>> for
>>>>>>>> so
>>>>>>>>>>> long, my singular
>>>>>>>>>>> goal in responding is to help this KIP past a perceived
>>>> impasse so
>>>>>> we
>>>>>>>> can
>>>>>>>>>>> finally move on
>>>>>>>>>>> to voting and implementing it. Long discussions are to be
>>>> expected
>>>>>> for
>>>>>>>>>>> major features like
>>>>>>>>>>> this but it's completely on us as the Streams devs to make
>> sure
>>>>>> there
>>>>>>>> is
>>>>>>>>>> an
>>>>>>>>>>> end in sight
>>>>>>>>>>> for any ongoing discussion.
>>>>>>>>>>>
>>>>>>>>>>> With that said, it's my understanding that the KIP as
>> currently
>>>>>>>> proposed
>>>>>>>>>> is
>>>>>>>>>>> just not tenable
>>>>>>>>>>> for Kafka Streams, and would prevent some EOS users from
>>>> upgrading
>>>>>> to
>>>>>>>> the
>>>>>>>>>>> version it
>>>>>>>>>>> first appears in. Given that we can't predict or guarantee
>>>> whether
>>>>>> any
>>>>>>>> of
>>>>>>>>>>> the followup KIPs
>>>>>>>>>>> would be completed in the same release cycle as this one,
>> we
>>>> need
>>>>>> to
>>>>>>>> make
>>>>>>>>>>> sure that the
>>>>>>>>>>> feature is either compatible with all current users or else
>>>>>>>>>> feature-flagged
>>>>>>>>>>> so that they may
>>>>>>>>>>> opt in/out.
>>>>>>>>>>>
>>>>>>>>>>> Therefore, IIUC we need to have either (or both) of these
>> as
>>>>>>>>>>> fully-implemented config options:
>>>>>>>>>>> 1. default.state.isolation.level
>>>>>>>>>>> 2. enable.transactional.state.stores
>>>>>>>>>>>
>>>>>>>>>>> This way EOS users for whom read_committed semantics are
>> not
>>>>>> viable can
>>>>>>>>>>> still upgrade,
>>>>>>>>>>> and either use the isolation.level config to leverage the
>> new
>>>> txn
>>>>>> state
>>>>>>>>>>> stores without sacrificing
>>>>>>>>>>> their application semantics, or else simply keep the
>>>> transactional
>>>>>>>> state
>>>>>>>>>>> stores disabled until we
>>>>>>>>>>> are able to fully implement the isolation level
>> configuration
>>>> at
>>>>>> either
>>>>>>>>>> an
>>>>>>>>>>> application or query level.
>>>>>>>>>>>
>>>>>>>>>>> Frankly you are the expert here and know much more about
>> the
>>>>>> tradeoffs
>>>>>>>> in
>>>>>>>>>>> both semantics and
>>>>>>>>>>> effort level of implementing one of these configs vs the
>>>> other. In
>>>>>> my
>>>>>>>>>>> opinion, either option would
>>>>>>>>>>> be fine and I would leave the decision of which one to
>> include
>>>> in
>>>>>> this
>>>>>>>>>> KIP
>>>>>>>>>>> completely up to you.
>>>>>>>>>>> I just don't see a way for the KIP to proceed without some
>>>>>> variation of
>>>>>>>>>> the
>>>>>>>>>>> above that would allow
>>>>>>>>>>> EOS users to opt-out of read_committed.
>>>>>>>>>>>
>>>>>>>>>>> (If it's all the same to you, I would recommend 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 that gets released.
>>>>>>>>>>> So personally, my choice would be to add the feature flag
>> and
>>>>>> leave it
>>>>>>>>>> off
>>>>>>>>>>> by default. If all 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
>>>>>>>>>>>
>>>>>>>>>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
>>>>>> nick.telford@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>
>>>>>>>>>>>> Agreed, I can live with that for now.
>>>>>>>>>>>>
>>>>>>>>>>>> In an effort to keep the scope of this KIP from
>> expanding, I'm
>>>>>> leaning
>>>>>>>>>>>> towards just providing a configurable
>>>>>> default.state.isolation.level
>>>>>>>> and
>>>>>>>>>>>> removing IsolationLevel from the StateStoreContext. This
>>>> would be
>>>>>>>>>>>> compatible with adding support for query-time
>> IsolationLevels
>>>> in
>>>>>> the
>>>>>>>>>>>> future, whilst providing a way for users to select an
>>>> isolation
>>>>>> level
>>>>>>>>>> now.
>>>>>>>>>>>>
>>>>>>>>>>>> The big problem with this, however, is that if a user
>> selects
>>>>>>>>>>>> processing.mode
>>>>>>>>>>>> = "exactly-once(-v2|-beta)", and
>>>> default.state.isolation.level =
>>>>>>>>>>>> "READ_UNCOMMITTED", we need to guarantee that the data
>> isn't
>>>>>> written
>>>>>>>> to
>>>>>>>>>>>> disk until commit() is called, but we also need to permit
>> IQ
>>>>>> threads
>>>>>>>> to
>>>>>>>>>>>> read from the ongoing transaction.
>>>>>>>>>>>>
>>>>>>>>>>>> A simple solution would be to (temporarily) forbid this
>>>>>> combination of
>>>>>>>>>>>> configuration, and have default.state.isolation.level
>>>>>> automatically
>>>>>>>>>> switch
>>>>>>>>>>>> to READ_COMMITTED when processing.mode is anything other
>> than
>>>>>>>>>>>> at-least-once. Do you think this would be acceptable?
>>>>>>>>>>>>
>>>>>>>>>>>> In a later KIP, we can add support for query-time
>> isolation
>>>>>> levels and
>>>>>>>>>>>> solve this particular problem there, which would relax
>> this
>>>>>>>> restriction.
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>> Nick
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
>>>> cadonna@apache.org>
>>>>>>>>>> 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,
>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
>>>>>>>>>>>>>> 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 <
>>>>>> nick.telford@gmail.com
>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I agree that having IsolationLevel be determined at
>>>> query-time
>>>>>> is
>>>>>>>> the
>>>>>>>>>>>>>>> ideal design, but there are a few sticking points:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>> There needs to be some way to communicate the
>>>> IsolationLevel
>>>>>> down
>>>>>>>> to
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> RocksDBStore itself, so that the query can respect it.
>>>> Since
>>>>>> stores
>>>>>>>>>>>> are
>>>>>>>>>>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
>>>>>> MeteredStore,
>>>>>>>>>>>>> etc.),
>>>>>>>>>>>>>>> we need some way to deliver that information to the
>> bottom
>>>>>> layer.
>>>>>>>> For
>>>>>>>>>>>>> IQv2,
>>>>>>>>>>>>>>> we can use the existing State#query() method, but IQv1
>> has
>>>> no
>>>>>> way
>>>>>>>> to
>>>>>>>>>>>> do
>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> A simple approach, which would potentially open up
>> other
>>>>>> options,
>>>>>>>>>>>> would
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
>>>>>>>>>>>>>>> readOnlyView(IsolationLevel isolationLevel) to
>>>>>>>> ReadOnlyKeyValueStore
>>>>>>>>>>>>> (and
>>>>>>>>>>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore,
>>>> etc.).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>> As mentioned above, RocksDB WriteBatches are not
>>>> thread-safe,
>>>>>> which
>>>>>>>>>>>>> causes
>>>>>>>>>>>>>>> a problem if we want to provide READ_UNCOMMITTED
>>>> Iterators. I
>>>>>> also
>>>>>>>>>>>> had a
>>>>>>>>>>>>>>> look at RocksDB Transactions[1], but they solve a very
>>>>>> different
>>>>>>>>>>>>> problem,
>>>>>>>>>>>>>>> and have the same thread-safety issue.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> One possible approach that I mentioned is chaining
>>>>>> WriteBatches:
>>>>>>>>>> every
>>>>>>>>>>>>>>> time a new Interactive Query is received (i.e.
>>>> readOnlyView,
>>>>>> see
>>>>>>>>>>>> above,
>>>>>>>>>>>>>>> is called) we "freeze" the existing WriteBatch, and
>> start a
>>>>>> new one
>>>>>>>>>>>> for
>>>>>>>>>>>>> new
>>>>>>>>>>>>>>> writes. The Interactive Query queries the "chain" of
>>>> previous
>>>>>>>>>>>>> WriteBatches
>>>>>>>>>>>>>>> + the underlying database; while the StreamThread
>> starts
>>>>>> writing to
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> *new* WriteBatch. On-commit, the StreamThread would
>> write
>>>> *all*
>>>>>>>>>>>>>>> WriteBatches in the chain to the database (that have
>> not
>>>> yet
>>>>>> been
>>>>>>>>>>>>> written).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> WriteBatches would be closed/freed only when they have
>> been
>>>>>> both
>>>>>>>>>>>>>>> committed, and all open Interactive Queries on them
>> have
>>>> been
>>>>>>>> closed.
>>>>>>>>>>>>> This
>>>>>>>>>>>>>>> would require some reference counting.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Obviously a drawback of this approach is the potential
>> for
>>>>>>>> increased
>>>>>>>>>>>>>>> memory usage: if an Interactive Query is long-lived,
>> for
>>>>>> example by
>>>>>>>>>>>>> doing a
>>>>>>>>>>>>>>> full scan over a large database, or even just pausing
>> in
>>>> the
>>>>>> middle
>>>>>>>>>> of
>>>>>>>>>>>>> an
>>>>>>>>>>>>>>> iteration, then the existing chain of WriteBatches
>> could be
>>>>>> kept
>>>>>>>>>>>> around
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>> a long time, potentially forever.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> A.
>>>>>>>>>>>>>>> Going off on a tangent, it looks like in addition to
>>>> supporting
>>>>>>>>>>>>>>> READ_COMMITTED queries, we could go further and support
>>>>>>>>>>>> REPEATABLE_READ
>>>>>>>>>>>>>>> queries (i.e. where subsequent reads to the same key
>> in the
>>>>>> same
>>>>>>>>>>>>>>> Interactive Query are guaranteed to yield the same
>> value)
>>>> by
>>>>>> making
>>>>>>>>>>>> use
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so
>> the
>>>>>>>>>> performance
>>>>>>>>>>>>>>> impact is likely to be negligible, but they do require
>>>> that the
>>>>>>>>>>>>> Interactive
>>>>>>>>>>>>>>> Query session can be explicitly closed.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This could be achieved if we made the above
>> readOnlyView
>>>>>> interface
>>>>>>>>>>>> look
>>>>>>>>>>>>>>> more like:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> interface ReadOnlyKeyValueView<K, V> implements
>>>>>>>>>>>> ReadOnlyKeyValueStore<K,
>>>>>>>>>>>>>>> V>, AutoCloseable {}
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
>>>>>>>>>>>>>>>         ...
>>>>>>>>>>>>>>>         ReadOnlyKeyValueView<K, V>
>>>> readOnlyView(IsolationLevel
>>>>>>>>>>>>> isolationLevel);
>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> But this would be a breaking change, as existing IQv1
>>>> queries
>>>>>> are
>>>>>>>>>>>>>>> guaranteed to never call store.close(), and therefore
>> these
>>>>>> would
>>>>>>>>>> leak
>>>>>>>>>>>>>>> memory under REPEATABLE_READ.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> B.
>>>>>>>>>>>>>>> One thing that's notable: MyRocks states that they
>> support
>>>>>>>>>>>>> READ_COMMITTED
>>>>>>>>>>>>>>> and REPEATABLE_READ, but they make no mention of
>>>>>>>>>>>> READ_UNCOMMITTED[3][4].
>>>>>>>>>>>>>>> This could be because doing so is technically
>>>>>> difficult/impossible
>>>>>>>>>>>> using
>>>>>>>>>>>>>>> the primitives available in RocksDB.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Lucas, to address your points:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> U1.
>>>>>>>>>>>>>>> It's only "SHOULD" to permit alternative (i.e.
>> non-RocksDB)
>>>>>>>>>>>>>>> implementations of StateStore that do not support
>> atomic
>>>>>> writes.
>>>>>>>>>>>>> Obviously
>>>>>>>>>>>>>>> in those cases, the guarantees Kafka Streams
>>>> provides/expects
>>>>>> would
>>>>>>>>>> be
>>>>>>>>>>>>>>> relaxed. Do you think we should require all
>>>> implementations to
>>>>>>>>>> support
>>>>>>>>>>>>>>> atomic writes?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> U2.
>>>>>>>>>>>>>>> Stores can support multiple IsolationLevels. As we've
>>>> discussed
>>>>>>>>>> above,
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> ideal scenario would be to specify the IsolationLevel
>> at
>>>>>>>> query-time.
>>>>>>>>>>>>>>> Failing that, I think the second-best approach is to
>>>> define the
>>>>>>>>>>>>>>> IsolationLevel for *all* queries based on the
>>>> processing.mode,
>>>>>>>> which
>>>>>>>>>>>> is
>>>>>>>>>>>>>>> what the default StateStoreContext#isolationLevel()
>>>> achieves.
>>>>>> Would
>>>>>>>>>>>> you
>>>>>>>>>>>>>>> prefer an alternative?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> While the existing implementation is equivalent to
>>>>>>>> READ_UNCOMMITTED,
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>> can yield unexpected results/errors under EOS, if a
>>>>>> transaction is
>>>>>>>>>>>>> rolled
>>>>>>>>>>>>>>> back. While this would be a change in behaviour for
>> users,
>>>> it
>>>>>> would
>>>>>>>>>>>> look
>>>>>>>>>>>>>>> more like a bug fix than a breaking change. That said,
>> we
>>>>>> *could*
>>>>>>>>>> make
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>> configurable, and default to the existing behaviour
>>>>>>>>>> (READ_UNCOMMITTED)
>>>>>>>>>>>>>>> instead of inferring it from the processing.mode?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> N1, N2.
>>>>>>>>>>>>>>> These were only primitives to avoid boxing costs, but
>> since
>>>>>> this is
>>>>>>>>>>>> not
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>> performance sensitive area, it should be fine to
>> change if
>>>>>> that's
>>>>>>>>>>>>> desirable.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> N3.
>>>>>>>>>>>>>>> It's because the store "manages its own offsets", which
>>>>>> includes
>>>>>>>> both
>>>>>>>>>>>>>>> committing the offset, *and providing it* via
>>>>>> getCommittedOffset().
>>>>>>>>>>>>>>> Personally, I think "managesOffsets" conveys this best,
>>>> but I
>>>>>> don't
>>>>>>>>>>>> mind
>>>>>>>>>>>>>>> changing it if the nomenclature is unclear.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Sorry for the massive emails/essays!
>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 1:
>> https://github.com/facebook/rocksdb/wiki/Transactions
>>>>>>>>>>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
>>>>>>>>>>>>>>> 3:
>>>>>>>>
>> https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
>>>>>>>>>>>>>>> 4:
>>>> https://mariadb.com/kb/en/myrocks-transactional-isolation/
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
>>>>>>>>>>>>>>> <lb...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> since I last read it in April, the KIP has become much
>>>>>> cleaner and
>>>>>>>>>>>>>>>> easier to read. Great work!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> It feels to me the last big open point is whether we
>> can
>>>>>> implement
>>>>>>>>>>>>>>>> isolation level as a query parameter. I understand
>> that
>>>> there
>>>>>> are
>>>>>>>>>>>>>>>> implementation concerns, but as Colt says, it would
>> be a
>>>> great
>>>>>>>>>>>>>>>> addition, and would also simplify the migration path
>> for
>>>> this
>>>>>>>>>> change.
>>>>>>>>>>>>>>>> Is the implementation problem you mentioned caused by
>> the
>>>>>>>> WriteBatch
>>>>>>>>>>>>>>>> not having a notion of a snapshot, as the underlying
>> DB
>>>>>> iterator
>>>>>>>>>>>> does?
>>>>>>>>>>>>>>>> In that case, I am not sure a chain of WriteBatches
>> as you
>>>>>> propose
>>>>>>>>>>>>>>>> would fully solve the problem, but maybe I didn't dig
>>>> enough
>>>>>> into
>>>>>>>>>> the
>>>>>>>>>>>>>>>> details to fully understand it.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If it's not possible to implement it now, would it be
>> an
>>>>>> option to
>>>>>>>>>>>>>>>> make sure in this KIP that we do not fully close the
>> door
>>>> on
>>>>>>>>>>>> per-query
>>>>>>>>>>>>>>>> isolation levels in the interface, as it may be
>> possible
>>>> to
>>>>>>>>>> implement
>>>>>>>>>>>>>>>> the missing primitives in RocksDB or Speedb in the
>> future.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Understanding:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to
>> be
>>>>>> persisted
>>>>>>>>>>>>>>>> atomically with the records?
>>>>>>>>>>>>>>>> * U2) Don't understand the default implementation of
>>>>>>>>>>>> `isolationLevel`.
>>>>>>>>>>>>>>>> The isolation level should be a property of the
>> underlying
>>>>>> store,
>>>>>>>>>> and
>>>>>>>>>>>>>>>> not be defined by the default config? Existing stores
>>>> probably
>>>>>>>> don't
>>>>>>>>>>>>>>>> guarantee READ_COMMITTED, so the default should be to
>>>> return
>>>>>>>>>>>>>>>> READ_UNCOMMITTED.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Nits:
>>>>>>>>>>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong`
>>>> return
>>>>>> type,
>>>>>>>>>> to
>>>>>>>>>>>>>>>> avoid the `null`?
>>>>>>>>>>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
>>>>>> `OptionalLong`
>>>>>>>>>>>>>>>> return type, to avoid the `-1`?
>>>>>>>>>>>>>>>> * N3) I don't understand why `managesOffsets` uses the
>>>>>> 'manage'
>>>>>>>>>> verb,
>>>>>>>>>>>>>>>> whereas all other methods use the "commits" verb. I'd
>>>> suggest
>>>>>>>>>>>>>>>> `commitsOffsets`.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Either way, it feels this KIP is very close to the
>> finish
>>>>>> line,
>>>>>>>> I'm
>>>>>>>>>>>>>>>> looking forward to seeing this in production!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>> Lucas
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
>>>>>> colt@littlehorse.io
>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Making IsolationLevel a query-time constraint,
>> rather
>>>> than
>>>>>>>> linking
>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> the processing.guarantee.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> As I understand it, would this allow even a user of
>> EOS
>>>> to
>>>>>>>> control
>>>>>>>>>>>>>>>> whether
>>>>>>>>>>>>>>>>> reading committed or uncommitted records? If so, I am
>>>> highly
>>>>>> in
>>>>>>>>>>>> favor
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I know that I was one of the early people to point
>> out
>>>> the
>>>>>>>> current
>>>>>>>>>>>>>>>>> shortcoming that IQ reads uncommitted records, but
>> just
>>>> this
>>>>>>>>>>>> morning I
>>>>>>>>>>>>>>>>> realized a pattern we use which means that (for
>> certain
>>>>>> queries)
>>>>>>>>>> our
>>>>>>>>>>>>>>>> system
>>>>>>>>>>>>>>>>> needs to be able to read uncommitted records, which
>> is
>>>> the
>>>>>>>> current
>>>>>>>>>>>>>>>> behavior
>>>>>>>>>>>>>>>>> of Kafka Streams in EOS.***
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> If IsolationLevel being a query-time decision allows
>> for
>>>>>> this,
>>>>>>>> then
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> would be amazing. I would also vote that the default
>>>> behavior
>>>>>>>>>> should
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>> reading uncommitted records, because it is totally
>>>> possible
>>>>>> for a
>>>>>>>>>>>>> valid
>>>>>>>>>>>>>>>>> application to depend on that behavior, and breaking
>> it
>>>> in a
>>>>>>>> minor
>>>>>>>>>>>>>>>> release
>>>>>>>>>>>>>>>>> might be a bit strong.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> *** (Note, for the curious reader....) Our
>> use-case/query
>>>>>> pattern
>>>>>>>>>>>> is a
>>>>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>>> complex, but reading "uncommitted" records is
>> actually
>>>> safe
>>>>>> in
>>>>>>>> our
>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>> 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 soon as the processor processes
>> the
>>>>>> result.
>>>>>>>> If
>>>>>>>>>>>>> they
>>>>>>>>>>>>>>>>> can't read uncommitted records, they may receive a
>> "201 -
>>>>>>>> Created"
>>>>>>>>>>>>>>>>> response, immediately followed by a "404 - 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 <
>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Addendum:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I think we would also face the same problem with the
>>>>>> approach
>>>>>>>> John
>>>>>>>>>>>>>>>> outlined
>>>>>>>>>>>>>>>>>> earlier (using the record cache as a transaction
>> buffer
>>>> and
>>>>>>>>>>>> flushing
>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>> straight to SST files). This is because the record
>> cache
>>>>>> (the
>>>>>>>>>>>>>>>> ThreadCache
>>>>>>>>>>>>>>>>>> class) is not thread-safe, so every commit would
>>>> invalidate
>>>>>> open
>>>>>>>>>> IQ
>>>>>>>>>>>>>>>>>> Iterators in the same way that RocksDB WriteBatches
>> do.
>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>> 2. Making IsolationLevel a query-time constraint,
>>>> rather
>>>>>> than
>>>>>>>>>>>>>>>> linking it
>>>>>>>>>>>>>>>>>>> to the processing.guarantee.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> There's a wrinkle that makes this a challenge:
>>>> Interactive
>>>>>>>>>> Queries
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> open an Iterator, when using transactions and
>>>>>> READ_UNCOMMITTED.
>>>>>>>>>>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries
>>>> need
>>>>>> to be
>>>>>>>>>>>> able
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> read records from the currently uncommitted
>> transaction
>>>>>> buffer
>>>>>>>>>>>>>>>>>>> (WriteBatch). This includes for Iterators, which
>> should
>>>>>> iterate
>>>>>>>>>>>>>>>> both the
>>>>>>>>>>>>>>>>>>> transaction buffer and underlying database (using
>>>>>>>>>>>>>>>>>>> WriteBatch#iteratorWithBase()).
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> The issue is that when the StreamThread commits, it
>>>> writes
>>>>>> the
>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>> WriteBatch to RocksDB *and then clears the
>> WriteBatch*.
>>>>>>>> Clearing
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> WriteBatch while an Interactive Query holds an open
>>>>>> Iterator on
>>>>>>>>>> it
>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>> invalidate the Iterator. Worse, it turns out that
>>>> Iterators
>>>>>>>> over
>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> WriteBatch become invalidated not just when the
>>>> WriteBatch
>>>>>> is
>>>>>>>>>>>>>>>> cleared,
>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>> also when the Iterators' current key receives a new
>>>> write.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Now that I'm writing this, I remember that this is
>> the
>>>>>> major
>>>>>>>>>>>> reason
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>> switched the original design from having a
>> query-time
>>>>>>>>>>>>>>>> IsolationLevel to
>>>>>>>>>>>>>>>>>>> having the IsolationLevel linked to the
>>>> transactionality
>>>>>> of the
>>>>>>>>>>>>>>>> stores
>>>>>>>>>>>>>>>>>>> themselves.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> It *might* be possible to resolve this, by having a
>>>>>> "chain" of
>>>>>>>>>>>>>>>>>>> WriteBatches, with the StreamThread switching to a
>> new
>>>>>>>> WriteBatch
>>>>>>>>>>>>>>>>>> whenever
>>>>>>>>>>>>>>>>>>> a new Interactive Query attempts to read from the
>>>>>> database, but
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>> cause some performance problems/memory pressure
>> when
>>>>>> subjected
>>>>>>>> to
>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> high
>>>>>>>>>>>>>>>>>>> Interactive Query load. It would also reduce the
>>>>>> efficiency of
>>>>>>>>>>>>>>>>>> WriteBatches
>>>>>>>>>>>>>>>>>>> on-commit, as we'd have to write N WriteBatches,
>> where
>>>> N
>>>>>> is the
>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>> Interactive Queries since the last commit.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I realise this is getting into the weeds of the
>>>>>> implementation,
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> you'd
>>>>>>>>>>>>>>>>>>> rather we focus on the API for now, but I think
>> it's
>>>>>> important
>>>>>>>> to
>>>>>>>>>>>>>>>>>> consider
>>>>>>>>>>>>>>>>>>> how to implement the desired API, in case we come
>> up
>>>> with
>>>>>> an
>>>>>>>> API
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> cannot be implemented efficiently, or even at all!
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
>>>>>>>> cadonna@apache.org
>>>>>>>>>>>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 6.
>>>>>>>>>>>>>>>>>>>> Of course, you are right! My bad!
>>>>>>>>>>>>>>>>>>>> Wiping out the state in the downgrading case is
>> fine.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 3a.
>>>>>>>>>>>>>>>>>>>> Focus on the public facing changes for the KIP. We
>>>> will
>>>>>> manage
>>>>>>>>>> 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.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks for getting back to me!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>>>> The fact that implementations can always track
>>>> estimated
>>>>>>>> memory
>>>>>>>>>>>>>>>> usage
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as
>> an
>>>>>> option,
>>>>>>>> and
>>>>>>>>>>>>>>>> I'll
>>>>>>>>>>>>>>>>>>>> clarify
>>>>>>>>>>>>>>>>>>>>> the JavaDoc that 0 is not just for
>> non-transactional
>>>>>> stores,
>>>>>>>>>>>>>>>> which is
>>>>>>>>>>>>>>>>>>>>> currently misleading.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 6.
>>>>>>>>>>>>>>>>>>>>> The problem with catching the exception in the
>>>> downgrade
>>>>>>>>>> process
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>> would require new code in the Kafka version being
>>>>>> downgraded
>>>>>>>>>> to.
>>>>>>>>>>>>>>>> Since
>>>>>>>>>>>>>>>>>>>>> users could conceivably downgrade to almost *any*
>>>> older
>>>>>>>> version
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
>>>>>>>>>>>>>>>>>>>>> The only way I can think of doing it would be to
>>>> provide
>>>>>> a
>>>>>>>>>>>>>>>> dedicated
>>>>>>>>>>>>>>>>>>>>> downgrade tool, that goes through every local
>> store
>>>> and
>>>>>>>> removes
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> offsets column families. But that seems like an
>>>>>> unnecessary
>>>>>>>>>>>>>>>> amount of
>>>>>>>>>>>>>>>>>>>> extra
>>>>>>>>>>>>>>>>>>>>> code to maintain just to handle a somewhat niche
>>>>>> situation,
>>>>>>>>>> when
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> alternative (automatically wipe and restore
>> stores)
>>>>>> should be
>>>>>>>>>>>>>>>>>>>> acceptable.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've
>>>> requested.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 3a.
>>>>>>>>>>>>>>>>>>>>> I agree that IsolationLevel makes more sense at
>>>>>> query-time,
>>>>>>>> and
>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>> initially attempted to place the IsolationLevel
>> at
>>>>>>>> query-time,
>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>> ran
>>>>>>>>>>>>>>>>>>>>> into some problems:
>>>>>>>>>>>>>>>>>>>>> - The key issue is that, under ALOS we're not
>> staging
>>>>>> writes
>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> transactions, so can't perform writes at the
>>>>>> READ_COMMITTED
>>>>>>>>>>>>>>>> isolation
>>>>>>>>>>>>>>>>>>>>> level. However, this may be addressed if we
>> decide to
>>>>>>>> *always*
>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>>> transactions as discussed under 3b.
>>>>>>>>>>>>>>>>>>>>> - IQv1 and IQv2 have quite different
>>>> implementations. I
>>>>>>>>>> remember
>>>>>>>>>>>>>>>>>> having
>>>>>>>>>>>>>>>>>>>>> some difficulty understanding the IQv1 internals,
>>>> which
>>>>>> made
>>>>>>>> it
>>>>>>>>>>>>>>>>>>>> difficult
>>>>>>>>>>>>>>>>>>>>> to determine what needed to be changed. However,
>> I
>>>>>> *think*
>>>>>>>> this
>>>>>>>>>>>>>>>> can be
>>>>>>>>>>>>>>>>>>>>> addressed for both implementations by wrapping
>> the
>>>>>>>> RocksDBStore
>>>>>>>>>>>>>>>> in an
>>>>>>>>>>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides
>> read
>>>>>> methods
>>>>>>>>>>>>>>>> (get,
>>>>>>>>>>>>>>>>>>>> etc.)
>>>>>>>>>>>>>>>>>>>>> to either read directly from the database or
>> from the
>>>>>> ongoing
>>>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>>> But IQv1 might still be difficult.
>>>>>>>>>>>>>>>>>>>>> - If IsolationLevel becomes a query constraint,
>> then
>>>> all
>>>>>>>> other
>>>>>>>>>>>>>>>>>>>> StateStores
>>>>>>>>>>>>>>>>>>>>> will need to respect it, including the in-memory
>>>> stores.
>>>>>> This
>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> require
>>>>>>>>>>>>>>>>>>>>> us to adapt in-memory stores to stage their
>> writes so
>>>>>> they
>>>>>>>> can
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> isolated
>>>>>>>>>>>>>>>>>>>>> from READ_COMMITTTED queries. It would also
>> become an
>>>>>>>> important
>>>>>>>>>>>>>>>>>>>>> consideration for third-party stores on upgrade,
>> as
>>>>>> without
>>>>>>>>>>>>>>>> changes,
>>>>>>>>>>>>>>>>>>>> they
>>>>>>>>>>>>>>>>>>>>> would not support READ_COMMITTED queries
>> correctly.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Ultimately, I may need some help making the
>> necessary
>>>>>> change
>>>>>>>> to
>>>>>>>>>>>>>>>> IQv1
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> support this, but I don't think it's
>> fundamentally
>>>>>>>> impossible,
>>>>>>>>>>>>>>>> if we
>>>>>>>>>>>>>>>>>>>> want
>>>>>>>>>>>>>>>>>>>>> to pursue this route.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 3b.
>>>>>>>>>>>>>>>>>>>>> The main reason I chose to keep ALOS
>> un-transactional
>>>>>> was to
>>>>>>>>>>>>>>>> minimize
>>>>>>>>>>>>>>>>>>>>> behavioural change for most users (I believe most
>>>> Streams
>>>>>>>> users
>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> default configuration, which is ALOS). That said,
>>>> it's
>>>>>> clear
>>>>>>>>>>>>>>>> that if
>>>>>>>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>>>>>> also used transactional stores, the only change
>> in
>>>>>> behaviour
>>>>>>>>>>>>>>>> would be
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>> it would become *more correct*, which could be
>>>>>> considered a
>>>>>>>>>> "bug
>>>>>>>>>>>>>>>> fix"
>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>> users, rather than a change they need to handle.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I believe that performance using transactions
>> (aka.
>>>>>> RocksDB
>>>>>>>>>>>>>>>>>>>> WriteBatches)
>>>>>>>>>>>>>>>>>>>>> should actually be *better* than the un-batched
>>>>>> write-path
>>>>>>>> that
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> currently used[1]. The only "performance"
>>>> consideration
>>>>>> will
>>>>>>>> be
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> increased memory usage that transactions require.
>>>> Given
>>>>>> the
>>>>>>>>>>>>>>>>>> mitigations
>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>> this memory that we have in place, I would expect
>>>> that
>>>>>> this
>>>>>>>> is
>>>>>>>>>>>>>>>> not a
>>>>>>>>>>>>>>>>>>>>> problem for most users.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> If we're happy to do so, we can make ALOS also
>> use
>>>>>>>>>> transactions.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Link 1:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>
>>>>
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
>>>>>>>>>>>> cadonna@apache.org
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates and sorry for the delay
>> on my
>>>>>> side!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>>>>>> Making the default implementation for flush() a
>>>> no-op
>>>>>> sounds
>>>>>>>>>>>>>>>> good to
>>>>>>>>>>>>>>>>>>>> me.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>>>>> I think what was bugging me here is that a
>>>> third-party
>>>>>> state
>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>> needs
>>>>>>>>>>>>>>>>>>>>>> to implement the state store interface. That
>> means
>>>> they
>>>>>> need
>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> implement a wrapper around the actual state
>> store
>>>> as we
>>>>>> do
>>>>>>>> for
>>>>>>>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>>>>>>>>> with RocksDBStore. So, a third-party state
>> store can
>>>>>> always
>>>>>>>>>>>>>>>> estimate
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> uncommitted bytes, if it wants, because the
>> wrapper
>>>> can
>>>>>>>> record
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>>>>>>>> bytes.
>>>>>>>>>>>>>>>>>>>>>> One case I can think of where returning -1 makes
>>>> sense
>>>>>> is
>>>>>>>> when
>>>>>>>>>>>>>>>>>> Streams
>>>>>>>>>>>>>>>>>>>>>> does not need to estimate the size of the write
>>>> batch
>>>>>> and
>>>>>>>>>>>>>>>> trigger
>>>>>>>>>>>>>>>>>>>>>> extraordinary commits, because the third-party
>> state
>>>>>> store
>>>>>>>>>>>>>>>> takes care
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> memory. But in that case the method could also
>> just
>>>>>> return
>>>>>>>> 0.
>>>>>>>>>>>>>>>> Even
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> case would be better solved with a method that
>>>> returns
>>>>>>>> whether
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>> store manages itself the memory used for
>> uncommitted
>>>>>> bytes
>>>>>>>> or
>>>>>>>>>>>>>>>> not.
>>>>>>>>>>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return
>>>> value,
>>>>>> I was
>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>> wondering when and if it will be used.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regarding returning 0 for transactional state
>> stores
>>>>>> when
>>>>>>>> the
>>>>>>>>>>>>>>>> batch
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> empty, I was just wondering because you
>> explicitly
>>>>>> stated
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> "or {@code 0} if this StateStore does not
>> support
>>>>>>>>>>>> transactions."
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> So it seemed to me returning 0 could only
>> happen for
>>>>>>>>>>>>>>>>>> non-transactional
>>>>>>>>>>>>>>>>>>>>>> state stores.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> a) What do you think if we move the isolation
>> level
>>>> to
>>>>>> IQ
>>>>>>>> (v1
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> v2)?
>>>>>>>>>>>>>>>>>>>>>> In the end this is the only component that
>> really
>>>> needs
>>>>>> to
>>>>>>>>>>>>>>>> specify
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> isolation level. It is similar to the Kafka
>> consumer
>>>>>> that
>>>>>>>> can
>>>>>>>>>>>>>>>> choose
>>>>>>>>>>>>>>>>>>>>>> with what isolation level to read the input
>> topic.
>>>>>>>>>>>>>>>>>>>>>> For IQv1 the isolation level should go into
>>>>>>>>>>>>>>>> StoreQueryParameters. For
>>>>>>>>>>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> b) Point a) raises the question what should
>> happen
>>>>>> during
>>>>>>>>>>>>>>>>>> at-least-once
>>>>>>>>>>>>>>>>>>>>>> processing when the state store does not use
>>>>>> transactions?
>>>>>>>>>> John
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> past proposed to also use transactions on state
>>>> stores
>>>>>> for
>>>>>>>>>>>>>>>>>>>>>> at-least-once. I like that idea, because it
>> avoids
>>>>>>>> aggregating
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>> records over and over again in the case of a
>>>> failure. We
>>>>>>>> had a
>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>> the past where a Streams applications in
>>>> at-least-once
>>>>>> mode
>>>>>>>>>> was
>>>>>>>>>>>>>>>>>> failing
>>>>>>>>>>>>>>>>>>>>>> continuously for some reasons I do not remember
>>>> before
>>>>>>>>>>>>>>>> committing the
>>>>>>>>>>>>>>>>>>>>>> offsets. After each failover, the app aggregated
>>>> again
>>>>>> and
>>>>>>>>>>>>>>>> again the
>>>>>>>>>>>>>>>>>>>>>> same records. Of course the aggregate increased
>> to
>>>> very
>>>>>>>> wrong
>>>>>>>>>>>>>>>> values
>>>>>>>>>>>>>>>>>>>>>> just because of the failover. With transactions
>> on
>>>> the
>>>>>> state
>>>>>>>>>>>>>>>> stores
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>> could have avoided this. The app would have
>> output
>>>> the
>>>>>> same
>>>>>>>>>>>>>>>> aggregate
>>>>>>>>>>>>>>>>>>>>>> multiple times (i.e., after each failover) but
>> at
>>>> least
>>>>>> the
>>>>>>>>>>>>>>>> value of
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> aggregate would not depend on the number of
>>>> failovers.
>>>>>>>>>>>>>>>> Outputting the
>>>>>>>>>>>>>>>>>>>>>> same aggregate multiple times would be incorrect
>>>> under
>>>>>>>>>>>>>>>> exactly-once
>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>> it is OK for at-least-once.
>>>>>>>>>>>>>>>>>>>>>> If it makes sense to add a config to turn on
>> and off
>>>>>>>>>>>>>>>> transactions on
>>>>>>>>>>>>>>>>>>>>>> state stores under at-least-once or just use
>>>>>> transactions in
>>>>>>>>>>>>>>>> any case
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> a question we should also discuss in this KIP.
>> It
>>>>>> depends a
>>>>>>>>>> bit
>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would
>>>> add a
>>>>>>>> config.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 4.
>>>>>>>>>>>>>>>>>>>>>> Your points are all valid. I tend to say to
>> keep the
>>>>>> metrics
>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>> flush() until we remove flush() completely from
>> the
>>>>>>>> interface.
>>>>>>>>>>>>>>>> Calls
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> flush() might still exist since existing
>> processors
>>>>>> might
>>>>>>>>>> still
>>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For
>>>> sure, we
>>>>>> need
>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> document
>>>>>>>>>>>>>>>>>>>>>> how the metrics change due to the transactions
>> in
>>>> the
>>>>>>>> upgrade
>>>>>>>>>>>>>>>> notes.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 5.
>>>>>>>>>>>>>>>>>>>>>> I see. Then you should describe how the
>> .position
>>>> files
>>>>>> are
>>>>>>>>>>>>>>>> handled
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>> a dedicated section of the KIP or incorporate
>> the
>>>>>>>> description
>>>>>>>>>>>>>>>> in the
>>>>>>>>>>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only
>>>>>> mentioning it
>>>>>>>>>> in
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> "Compatibility, 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:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>
>> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> It is a bit brittle, but it works.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks for taking the time to review the KIP.
>> I'm
>>>> back
>>>>>> from
>>>>>>>>>>>>>>>> leave
>>>>>>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> intend to move this forwards as quickly as I
>> can.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Addressing your points:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>>>>>>> Because flush() is part of the StateStore API,
>> it's
>>>>>> exposed
>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> custom
>>>>>>>>>>>>>>>>>>>>>>> Processors, which might be making calls to
>> flush().
>>>>>> This
>>>>>>>> was
>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> case in a few integration tests.
>>>>>>>>>>>>>>>>>>>>>>> To maintain as much compatibility as possible,
>> I'd
>>>>>> prefer
>>>>>>>> not
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> an UnsupportedOperationException, as it will
>> cause
>>>>>>>> previously
>>>>>>>>>>>>>>>>>> working
>>>>>>>>>>>>>>>>>>>>>>> Processors to start throwing exceptions at
>> runtime.
>>>>>>>>>>>>>>>>>>>>>>> I agree that it doesn't make sense for it to
>> proxy
>>>>>>>> commit(),
>>>>>>>>>>>>>>>> though,
>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>> that would cause it to violate the "StateStores
>>>> commit
>>>>>> only
>>>>>>>>>>>>>>>> when the
>>>>>>>>>>>>>>>>>>>> Task
>>>>>>>>>>>>>>>>>>>>>>> commits" rule.
>>>>>>>>>>>>>>>>>>>>>>> Instead, I think we should make this a no-op.
>> That
>>>> way,
>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>> user
>>>>>>>>>>>>>>>>>>>>>>> Processors will continue to work as-before,
>> without
>>>>>>>> violation
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>> consistency that would be caused by premature
>>>>>> flush/commit
>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>>>>>>>>>> data to disk.
>>>>>>>>>>>>>>>>>>>>>>> What do you think?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
>>>>>> implementation
>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> transactional, but is unable to estimate the
>>>>>> uncommitted
>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>> usage,
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> method will return -1.
>>>>>>>>>>>>>>>>>>>>>>> The intention here is to permit third-party
>>>>>> implementations
>>>>>>>>>>>>>>>> that may
>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> able to estimate memory usage.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been
>> written to
>>>> the
>>>>>>>> store
>>>>>>>>>>>>>>>> yet. I
>>>>>>>>>>>>>>>>>>>>>> thought
>>>>>>>>>>>>>>>>>>>>>>> that was implied by "This method will return an
>>>>>>>> approximation
>>>>>>>>>>>>>>>> of the
>>>>>>>>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>>>>>>> would be freed by the next call to {@link
>>>>>> #commit(Map)}"
>>>>>>>> and
>>>>>>>>>>>>>>>>>> "@return
>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>> approximate size of all records awaiting {@link
>>>>>>>>>>>> #commit(Map)}",
>>>>>>>>>>>>>>>>>>>> however,
>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you
>> think
>>>> this
>>>>>> is
>>>>>>>>>>>>>>>> unclear?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>>> I realise this is probably the most contentious
>>>> point
>>>>>> in my
>>>>>>>>>>>>>>>> design,
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>>>>>> open to changing it if I'm unable to convince
>> you
>>>> of
>>>>>> the
>>>>>>>>>>>>>>>> benefits.
>>>>>>>>>>>>>>>>>>>>>>> Nevertheless, here's my argument:
>>>>>>>>>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly
>>>> provided
>>>>>>>>>>>>>>>> StateStores
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> query, and it may be important for users to
>>>>>>>> programmatically
>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>> mode the StateStore is operating under. If we
>>>> simply
>>>>>>>> provide
>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the
>>>> internal
>>>>>>>> streams
>>>>>>>>>>>>>>>>>>>> engine), or
>>>>>>>>>>>>>>>>>>>>>>> similar, then users will need to understand the
>>>>>> operation
>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> consequences
>>>>>>>>>>>>>>>>>>>>>>> of each available processing mode and how it
>>>> pertains
>>>>>> to
>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>>>> StateStore.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Interactive Query users aren't the only people
>> that
>>>>>> care
>>>>>>>>>> about
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
>>>>>>>> implementers
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> custom
>>>>>>>>>>>>>>>>>>>>>>> StateStores also need to understand the
>> behaviour
>>>>>> expected
>>>>>>>> of
>>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>>>>> implementation. KIP-892 introduces some
>> assumptions
>>>>>> into
>>>>>>>> the
>>>>>>>>>>>>>>>> Streams
>>>>>>>>>>>>>>>>>>>>>> Engine
>>>>>>>>>>>>>>>>>>>>>>> about how StateStores operate under each
>> processing
>>>>>> mode,
>>>>>>>> and
>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>> important that custom implementations adhere to
>>>> those
>>>>>>>>>>>>>>>> assumptions in
>>>>>>>>>>>>>>>>>>>>>> order
>>>>>>>>>>>>>>>>>>>>>>> to maintain the consistency guarantees.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> IsolationLevels provide a high-level contract
>> on
>>>> the
>>>>>>>>>> behaviour
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> StateStore: a user knows that under
>> READ_COMMITTED,
>>>>>> they
>>>>>>>> will
>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>> writes
>>>>>>>>>>>>>>>>>>>>>>> only after the Task has committed, and under
>>>>>>>> READ_UNCOMMITTED
>>>>>>>>>>>>>>>> they
>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>> writes immediately. No understanding of the
>>>> details of
>>>>>> each
>>>>>>>>>>>>>>>>>>>>>> processing.mode
>>>>>>>>>>>>>>>>>>>>>>> is required, either for IQ users or StateStore
>>>>>>>> implementers.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> An argument can be made that these contractual
>>>>>> guarantees
>>>>>>>> can
>>>>>>>>>>>>>>>> simply
>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> documented for the processing.mode (i.e. that
>>>>>> exactly-once
>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
>>>>>>>> at-least-once
>>>>>>>>>>>>>>>> behaves
>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small
>>>> issues
>>>>>> with
>>>>>>>>>>>>>>>> this I'd
>>>>>>>>>>>>>>>>>>>>>> prefer
>>>>>>>>>>>>>>>>>>>>>>> to avoid:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>          - Where would we document these
>> contracts,
>>>> in
>>>>>> a way
>>>>>>>>>> that
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> difficult
>>>>>>>>>>>>>>>>>>>>>>>          for users/implementers to miss/ignore?
>>>>>>>>>>>>>>>>>>>>>>>          - It's not clear to users that the
>>>> processing
>>>>>> mode
>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> communicating
>>>>>>>>>>>>>>>>>>>>>>>          an expectation of read isolation,
>> unless
>>>> they
>>>>>> read
>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> documentation. Users
>>>>>>>>>>>>>>>>>>>>>>>          rarely consult documentation unless
>> they
>>>> feel
>>>>>> they
>>>>>>>>>> need
>>>>>>>>>>>>>>>> to, so
>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>> likely
>>>>>>>>>>>>>>>>>>>>>>>          this detail would get missed by many
>> users.
>>>>>>>>>>>>>>>>>>>>>>>          - It tightly couples processing modes
>> to
>>>> read
>>>>>>>>>> isolation.
>>>>>>>>>>>>>>>> Adding
>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>          processing modes, or changing the read
>>>>>> isolation of
>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>>          modes would be difficult/impossible.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Ultimately, the cost of introducing
>>>> IsolationLevels is
>>>>>>>> just a
>>>>>>>>>>>>>>>> single
>>>>>>>>>>>>>>>>>>>>>>> method, since we re-use the existing
>> IsolationLevel
>>>>>> enum
>>>>>>>> from
>>>>>>>>>>>>>>>> Kafka.
>>>>>>>>>>>>>>>>>>>> This
>>>>>>>>>>>>>>>>>>>>>>> gives us a clear place to document the
>> contractual
>>>>>>>> guarantees
>>>>>>>>>>>>>>>>>> expected
>>>>>>>>>>>>>>>>>>>>>>> of/provided by StateStores, that is accessible
>>>> both by
>>>>>> the
>>>>>>>>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>>>>>>>>>> itself, and by IQ users.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> (Writing this I've just realised that the
>>>> StateStore
>>>>>> and IQ
>>>>>>>>>>>>>>>> APIs
>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>> don't provide access to StateStoreContext that
>> IQ
>>>> users
>>>>>>>> would
>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>> direct
>>>>>>>>>>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
>>>>>>>>>> isolationLevel()
>>>>>>>>>>>>>>>>>> itself
>>>>>>>>>>>>>>>>>>>>>> too?)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 4.
>>>>>>>>>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics
>>>> in-place
>>>>>>>>>>>>>>>> either, as
>>>>>>>>>>>>>>>>>>>> it's a
>>>>>>>>>>>>>>>>>>>>>>> backwards incompatible change. My concern is
>> that,
>>>> if
>>>>>> we
>>>>>>>>>> leave
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>> "flush" metrics in place, they will be
>> confusing to
>>>>>> users.
>>>>>>>>>>>>>>>> Right
>>>>>>>>>>>>>>>>>> now,
>>>>>>>>>>>>>>>>>>>>>>> "flush" metrics record explicit flushes to
>> disk,
>>>> but
>>>>>> under
>>>>>>>>>>>>>>>> KIP-892,
>>>>>>>>>>>>>>>>>>>> even
>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>> commit() will not explicitly flush data to
>> disk -
>>>>>> RocksDB
>>>>>>>>>> will
>>>>>>>>>>>>>>>>>> decide
>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>> when to flush memtables to disk itself.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd
>> have
>>>> two
>>>>>>>>>>>> options,
>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>> both
>>>>>>>>>>>>>>>>>>>>>>> seem pretty bad to me:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>          1. Have them record calls to commit(),
>>>> which
>>>>>> would
>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> misleading, as
>>>>>>>>>>>>>>>>>>>>>>>          data is no longer explicitly "flushed"
>> to
>>>> disk
>>>>>> by
>>>>>>>> this
>>>>>>>>>>>>>>>> call.
>>>>>>>>>>>>>>>>>>>>>>>          2. Have them record nothing at all,
>> which
>>>> is
>>>>>>>>>> equivalent
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> removing
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>          metrics, except that users will see the
>>>> metric
>>>>>>>> still
>>>>>>>>>>>>>>>> exists and
>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>> assume
>>>>>>>>>>>>>>>>>>>>>>>          that the metric is correct, and that
>>>> there's a
>>>>>>>> problem
>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>>>> system
>>>>>>>>>>>>>>>>>>>>>>>          when there isn't.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I agree that removing them is also a bad
>> solution,
>>>> and
>>>>>> I'd
>>>>>>>>>>>>>>>> like some
>>>>>>>>>>>>>>>>>>>>>>> guidance on the best path forward here.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 5.
>>>>>>>>>>>>>>>>>>>>>>> Position files are updated on every write to a
>>>>>> StateStore.
>>>>>>>>>>>>>>>> Since our
>>>>>>>>>>>>>>>>>>>>>> writes
>>>>>>>>>>>>>>>>>>>>>>> are now buffered until commit(), we can't
>> update
>>>> the
>>>>>>>> Position
>>>>>>>>>>>>>>>> file
>>>>>>>>>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>> commit() has been called, otherwise it would be
>>>>>>>> inconsistent
>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> data
>>>>>>>>>>>>>>>>>>>>>>> in the event of a rollback. Consequently, we
>> need
>>>> to
>>>>>> manage
>>>>>>>>>>>>>>>> these
>>>>>>>>>>>>>>>>>>>> offsets
>>>>>>>>>>>>>>>>>>>>>>> the same way we manage the checkpoint offsets,
>> and
>>>>>> ensure
>>>>>>>>>>>>>>>> they're
>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>> written on commit().
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 6.
>>>>>>>>>>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what
>>>> tests to
>>>>>>>>>> write.
>>>>>>>>>>>>>>>> How
>>>>>>>>>>>>>>>>>>>>>> explicit
>>>>>>>>>>>>>>>>>>>>>>> do we need to be here in the KIP?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed
>> to be
>>>>>>>> seamless,
>>>>>>>>>>>>>>>> and we
>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>> definitely add some tests around that.
>> Downgrade,
>>>> it
>>>>>>>>>>>>>>>> transpires,
>>>>>>>>>>>>>>>>>> isn't
>>>>>>>>>>>>>>>>>>>>>>> currently possible, as the extra column family
>> for
>>>>>> offset
>>>>>>>>>>>>>>>> storage is
>>>>>>>>>>>>>>>>>>>>>>> incompatible with the pre-KIP-892
>> implementation:
>>>> when
>>>>>> you
>>>>>>>>>>>>>>>> open a
>>>>>>>>>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>>>>>>>>>> database, you must open all available column
>>>> families
>>>>>> or
>>>>>>>>>>>>>>>> receive an
>>>>>>>>>>>>>>>>>>>>>> error.
>>>>>>>>>>>>>>>>>>>>>>> What currently happens on downgrade is that it
>>>>>> attempts to
>>>>>>>>>>>>>>>> open the
>>>>>>>>>>>>>>>>>>>>>> store,
>>>>>>>>>>>>>>>>>>>>>>> throws an error about the offsets column
>> family not
>>>>>> being
>>>>>>>>>>>>>>>> opened,
>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given
>> that
>>>>>>>>>> downgrades
>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> uncommon, I think this is acceptable
>> behaviour, as
>>>> the
>>>>>>>>>>>>>>>> end-state is
>>>>>>>>>>>>>>>>>>>>>>> consistent, even if it results in an
>> undesirable
>>>> state
>>>>>>>>>>>> restore.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Should I document the upgrade/downgrade
>> behaviour
>>>>>>>> explicitly
>>>>>>>>>>>>>>>> in the
>>>>>>>>>>>>>>>>>>>> KIP?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
>>>>>>>>>>>>>>>> cadonna@apache.org>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick!
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the updates!
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>>>>>>>> Why does StateStore#flush() default to
>>>>>>>>>>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
>>>>>>>>>>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore
>>>> after
>>>>>> this
>>>>>>>> KIP
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>> released, I would rather throw an unsupported
>>>>>> operation
>>>>>>>>>>>>>>>> exception
>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>> default.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>>>>>>> When would a state store return -1 from
>>>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
>> while
>>>>>> being
>>>>>>>>>>>>>>>>>>>> transactional?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Wouldn't
>>>> StateStore#approximateNumUncommittedBytes()
>>>>>> also
>>>>>>>>>>>>>>>> return 0
>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>> the state store is transactional but nothing
>> has
>>>> been
>>>>>>>>>> written
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> state store yet?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this
>> KIP
>>>> really
>>>>>>>> need
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
>>>> StateStoreContext
>>>>>> has
>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>>>>> appConfigs() which basically exposes the same
>>>>>> information,
>>>>>>>>>>>>>>>> i.e., if
>>>>>>>>>>>>>>>>>>>> EOS
>>>>>>>>>>>>>>>>>>>>>>>> is enabled or not.
>>>>>>>>>>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore
>>>> interface
>>>>>> as
>>>>>>>>>>>>>>>> loosely
>>>>>>>>>>>>>>>>>>>> coupled
>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
>>>>>> implementers
>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>> freedom,
>>>>>>>>>>>>>>>>>>>>>>>> and reduce the amount of internal knowledge
>>>> required."
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> While I understand the intent, I doubt that it
>>>>>> decreases
>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> coupling of
>>>>>>>>>>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
>>>>>>>>>> READ_COMMITTED
>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>> applies to IQ but not to reads by processors.
>>>> Thus,
>>>>>>>>>>>>>>>> implementers
>>>>>>>>>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>>>>>>>>>> understand how Streams accesses the state
>> stores.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I would like to hear what others think about
>> this.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 4.
>>>>>>>>>>>>>>>>>>>>>>>> Great exposing new metrics for transactional
>> state
>>>>>> stores!
>>>>>>>>>>>>>>>>>> However, I
>>>>>>>>>>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate
>> (in
>>>> the
>>>>>>>> docs)
>>>>>>>>>>>>>>>> the 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 also need to adapt/add
>> integration
>>>>>> tests
>>>>>>>>>>>>>>>> besides
>>>>>>>>>>>>>>>>>> unit
>>>>>>>>>>>>>>>>>>>>>>>> tests. Additionally, we probably need
>> integration
>>>> or
>>>>>>>> system
>>>>>>>>>>>>>>>> tests
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> verify that upgrades and downgrades between
>>>>>> transactional
>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>> non-transactional state stores work as
>> expected.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in
>> the
>>>> KIP,
>>>>>>>> under
>>>>>>>>>>>>>>>>>>>> "Rejected
>>>>>>>>>>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea
>> worth
>>>>>> pursuing,
>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>> believe
>>>>>>>>>>>>>>>>>>>>>>>>> that it's out of the scope of this KIP,
>> because
>>>> it
>>>>>>>> solves a
>>>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>>>> set
>>>>>>>>>>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of
>> this
>>>> one
>>>>>> has
>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>> grown
>>>>>>>>>>>>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>>>>>>>>>> large!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
>>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP (
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>>>>>>>>>> )
>>>>>>>>>>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing
>> back
>>>>>> "Atomic
>>>>>>>>>>>>>>>>>>>> Checkpointing"
>>>>>>>>>>>>>>>>>>>>>>>> (for
>>>>>>>>>>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think
>> the one
>>>>>> thing
>>>>>>>>>>>>>>>> missing is
>>>>>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>> changes to metrics (notably the store
>> "flush"
>>>>>> metrics
>>>>>>>> will
>>>>>>>>>>>>>>>> need
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>> renamed to "commit").
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> The reason I brought back Atomic
>> Checkpointing
>>>> was
>>>>>> to
>>>>>>>>>>>>>>>> decouple
>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>> flush
>>>>>>>>>>>>>>>>>>>>>>>>>> from store commit. This is important,
>> because
>>>> with
>>>>>>>>>>>>>>>> Transactional
>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on
>>>> *every*
>>>>>> Task
>>>>>>>>>>>>>>>> commit,
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>> just when the StateStore is closing,
>> otherwise
>>>> our
>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>> buffer
>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>> never be written and persisted, instead
>> growing
>>>>>>>> unbounded!
>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>> experimented
>>>>>>>>>>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a
>> store
>>>>>> flush
>>>>>>>>>>>>>>>> whenever
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its
>>>>>> configured
>>>>>>>>>>>>>>>> size, but
>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer
>>>> from
>>>>>> being
>>>>>>>>>>>>>>>>>> configured
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>> unbounded, and it still would have required
>>>> explicit
>>>>>>>>>>>>>>>> flushes of
>>>>>>>>>>>>>>>>>>>>>> RocksDB,
>>>>>>>>>>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory
>>>>>> utilization.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the
>> "right"
>>>> way
>>>>>> to
>>>>>>>>>>>>>>>> resolve
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog
>> offsets
>>>> that
>>>>>>>>>>>>>>>> correspond
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> most
>>>>>>>>>>>>>>>>>>>>>>>>>> recently written records are always
>> atomically
>>>>>> written
>>>>>>>> to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>>>>>>>>>>>>> (by writing them to the same transaction
>>>> buffer),
>>>>>> we can
>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>>> forcibly
>>>>>>>>>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk,
>> letting
>>>>>> RocksDB
>>>>>>>>>>>>>>>> flush
>>>>>>>>>>>>>>>>>> them
>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>>> when necessary, without losing any of our
>>>>>> consistency
>>>>>>>>>>>>>>>> guarantees.
>>>>>>>>>>>>>>>>>>>> See
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> updated KIP for more info.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I have fully implemented these changes,
>>>> although I'm
>>>>>>>> still
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>> entirely
>>>>>>>>>>>>>>>>>>>>>>>>>> happy with the implementation for segmented
>>>>>> StateStores,
>>>>>>>>>> so
>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>> plan
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests
>> pass. If
>>>>>> you'd
>>>>>>>> like
>>>>>>>>>>>>>>>> to try
>>>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>>>>>>>>> review this highly experimental and
>> incomplete
>>>>>> branch,
>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>>>>>>>> here:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
>>>>>>>> .
>>>>>>>>>>>>>>>> Note:
>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>> built
>>>>>>>>>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable
>> base
>>>> to
>>>>>> build
>>>>>>>>>>>>>>>> and test
>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>> on,
>>>>>>>>>>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples
>> comparisons
>>>> in a
>>>>>>>> live
>>>>>>>>>>>>>>>>>>>>>> environment. I
>>>>>>>>>>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's
>> nearer
>>>>>>>>>> completion
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>>>>>>>> proven on our main application.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing
>> and
>>>>>> testing:
>>>>>>>>>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window)
>>>> stores
>>>>>>>>>>>>>>>>>>>>>>>>>> - Global stores
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> As I do not currently use either of these,
>> so my
>>>>>> primary
>>>>>>>>>>>>>>>> test
>>>>>>>>>>>>>>>>>>>>>>>> environment
>>>>>>>>>>>>>>>>>>>>>>>>>> doesn't test these areas.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next
>> week
>>>> for
>>>>>> a few
>>>>>>>>>>>>>>>> weeks,
>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>> not have time to move this forward until
>> late
>>>>>> August.
>>>>>>>> That
>>>>>>>>>>>>>>>> said,
>>>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just
>>>> won't be
>>>>>>>> able
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> respond
>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>> quickly as usual.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
>>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact
>> on IQ
>>>> is
>>>>>> not
>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>> had
>>>>>>>>>>>>>>>>>>>>>>>>>>> considered.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the state
>> store
>>>>>> from the
>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
>> the
>>>>>>>> checkpoint
>>>>>>>>>>>>>>>> (thus,
>>>>>>>>>>>>>>>>>>>>>>>> flushing
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
>> data
>>>>>> and/or
>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you
>> suggesting
>>>>>> that we
>>>>>>>>>>>>>>>> add an
>>>>>>>>>>>>>>>>>>>>>>>> additional
>>>>>>>>>>>>>>>>>>>>>>>>>>> config for the max number of commit
>> intervals
>>>>>> between
>>>>>>>>>>>>>>>>>> checkpoints?
>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the
>>>>>> transaction
>>>>>>>>>>>>>>>> buffers
>>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>>> nearly
>>>>>>>>>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of
>> commit
>>>>>>>> intervals
>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>>>>>>> elapsed,
>>>>>>>>>>>>>>>>>>>>>>>>>>> whichever comes first?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although
>> this
>>>>>>>> re-ignites
>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>> earlier
>>>>>>>>>>>>>>>>>>>>>>>>>>> debate about whether a config should be
>>>> measured in
>>>>>>>>>>>>>>>> "number of
>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute
>> time.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the
>> reason
>>>> I
>>>>>> was
>>>>>>>>>>>>>>>> pursuing
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> Atomic
>>>>>>>>>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable
>> flush
>>>> from
>>>>>>>>>>>>>>>> checkpointing,
>>>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every
>> commit
>>>>>> without
>>>>>>>> any
>>>>>>>>>>>>>>>>>>>> performance
>>>>>>>>>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely
>> the
>>>>>> "best"
>>>>>>>>>>>>>>>> solution,
>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>> I'm not
>>>>>>>>>>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back
>> into
>>>> this
>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
>>>>>> transactional
>>>>>>>>>>>> logic
>>>>>>>>>>>>>>>>>>>> directly
>>>>>>>>>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with
>> the
>>>>>>>>>>>>>>>>>>>>>> StateStore#newTransaction
>>>>>>>>>>>>>>>>>>>>>>>>>>> method, and reduces the number of new
>> classes
>>>>>>>> introduced,
>>>>>>>>>>>>>>>>>>>>>> significantly
>>>>>>>>>>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and
>> the
>>>>>>>> complexity
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> drastically
>>>>>>>>>>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic
>>>> Checkpoints
>>>>>>>> into
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna
>> <
>>>>>>>>>>>>>>>> cadonna@apache.org>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> As far as I understand, you want to
>> atomically
>>>>>> update
>>>>>>>>>> the
>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the
>>>> memtable
>>>>>> of a
>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit
>> time
>>>>>> elapsed
>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>> after
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that
>> would
>>>> lead
>>>>>> to
>>>>>>>>>>>>>>>> exceeding
>>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
>>>> before the
>>>>>>>> next
>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>> interval
>>>>>>>>>>>>>>>>>>>>>>>>>>>> ends.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would
>> commit
>>>>>> every
>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>> interval
>>>>>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store will only be atomically
>>>> updated
>>>>>>>> roughly
>>>>>>>>>>>>>>>> every
>>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of
>>>> data.
>>>>>> Also
>>>>>>>> IQ
>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> then
>>>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>>>>> see new data roughly every
>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> After a failure the state store needs to
>>>> restore
>>>>>> up to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Is this correct?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> What about atomically updating the state
>> store
>>>>>> from
>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
>> the
>>>>>>>> checkpoint
>>>>>>>>>>>>>>>> (thus,
>>>>>>>>>>>>>>>>>>>>>>>> flushing
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
>> data
>>>>>> and/or
>>>>>>>>>>>>>>>> number of
>>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have
>> the
>>>> same
>>>>>> delay
>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because
>> both
>>>>>> would
>>>>>>>>>>>>>>>> appear
>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However,
>>>> after a
>>>>>>>> failure
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>>>>> still needs to restore up to
>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>> it might restore data that is already in
>> the
>>>> state
>>>>>>>> store
>>>>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable
>> offset
>>>>>> (i.e.
>>>>>>>> the
>>>>>>>>>>>>>>>> last
>>>>>>>>>>>>>>>>>>>>>> committed
>>>>>>>>>>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring
>>>> data
>>>>>> that
>>>>>>>> is
>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should
>> not
>>>>>> violated.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> This solution needs at least one new
>> config to
>>>>>> specify
>>>>>>>>>>>>>>>> when a
>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>>>>>>>>>>>>>>> should be written.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail
>>>> that
>>>>>> does
>>>>>>>> not
>>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>>>>>>>> anything
>>>>>>>>>>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit
>>>> interval
>>>>>> is 30
>>>>>>>>>>>>>>>> seconds,
>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>> five
>>>>>>>>>>>>>>>>>>>>>>>>>>>> seconds.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch
>> on
>>>> our
>>>>>>>> staging
>>>>>>>>>>>>>>>>>>>>>> environment,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
>>>> non-trivial
>>>>>>>>>>>>>>>> application.
>>>>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate
>> that
>>>> we
>>>>>> saw
>>>>>>>> the
>>>>>>>>>>>>>>>> last
>>>>>>>>>>>>>>>>>>>> time
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP,
>> but
>>>> this
>>>>>>>> time,
>>>>>>>>>> I
>>>>>>>>>>>>>>>>>> think I
>>>>>>>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>>>>>>>>>> why.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit,
>> which is
>>>>>> called
>>>>>>>> at
>>>>>>>>>>>>>>>> the end
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commit process, has the following
>> behaviour:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under ALOS: checkpoint the
>> state
>>>>>> stores.
>>>>>>>>>> This
>>>>>>>>>>>>>>>>>> includes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            flushing memtables in RocksDB.
>>>> This is
>>>>>>>>>>>> acceptable
>>>>>>>>>>>>>>>>>>>> because the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit.interval.ms is 5
>> seconds,
>>>> so
>>>>>>>> forcibly
>>>>>>>>>>>>>>>> flushing
>>>>>>>>>>>>>>>>>>>>>> memtables
>>>>>>>>>>>>>>>>>>>>>>>>>>>> every 5
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            seconds is acceptable for most
>>>>>>>> applications.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Under EOS: checkpointing is
>> not
>>>> done,
>>>>>>>>>> *unless*
>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>> being
>>>>>>>>>>>>>>>>>>>>>>>>>>>> forced, due
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            to e.g. the Task closing or
>> being
>>>>>> revoked.
>>>>>>>>>> This
>>>>>>>>>>>>>>>> means
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> under
>>>>>>>>>>>>>>>>>>>>>>>>>>>> normal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            processing conditions, the
>> state
>>>> stores
>>>>>>>> will
>>>>>>>>>> not
>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>> checkpointed,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> and will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            not have memtables flushed at
>> all ,
>>>>>> unless
>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>>>>>>> decides to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> flush them on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            its own. Checkpointing stores
>> and
>>>>>>>>>> force-flushing
>>>>>>>>>>>>>>>> their
>>>>>>>>>>>>>>>>>>>>>> memtables
>>>>>>>>>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            done when a Task is being
>> closed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint
>>>> stores on
>>>>>> at
>>>>>>>>>>>> least
>>>>>>>>>>>>>>>>>> *some*
>>>>>>>>>>>>>>>>>>>>>>>> normal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the
>> RocksDB
>>>>>>>> transaction
>>>>>>>>>>>>>>>>>> buffers
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets
>> are
>>>>>> synced to
>>>>>>>>>>>>>>>> disk to
>>>>>>>>>>>>>>>>>>>>>> prevent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
>>>> Consequently,
>>>>>> my
>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task
>> commit,
>>>>>> which
>>>>>>>> is
>>>>>>>>>>>>>>>> far too
>>>>>>>>>>>>>>>>>>>>>>>>>>>> frequent.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000
>> records,
>>>>>> which
>>>>>>>> is
>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>> flush
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
>> performance
>>>>>> problems
>>>>>>>> for
>>>>>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>>>>>>>>> applications.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and
>> I'm
>>>>>>>> currently
>>>>>>>>>>>>>>>> leaning
>>>>>>>>>>>>>>>>>>>>>>>> towards
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> using the
>>>> statestore.transaction.buffer.max.bytes
>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to
>>>> exceed it.
>>>>>>>> This
>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>> complement the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
>> functionality
>>>> that
>>>>>> this
>>>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> provides, in the following way:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - Currently, we use
>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> force an
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            early Task commit if processing
>>>> more
>>>>>>>> records
>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>> cause
>>>>>>>>>>>>>>>>>>>> our
>>>>>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            transactions to exceed the
>> memory
>>>>>> assigned
>>>>>>>> to
>>>>>>>>>>>>>>>> them.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            - New functionality: when a
>> Task
>>>> *does*
>>>>>>>>>> commit,
>>>>>>>>>>>>>>>> we will
>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            the stores (and hence flush the
>>>>>> transaction
>>>>>>>>>>>>>>>> buffers)
>>>>>>>>>>>>>>>>>>>> unless
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> expect to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            cross the
>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
>>>>>>>>>>>>>>>>>>>> threshold
>>>>>>>>>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the next
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>            commit
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick
>> Telford <
>>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I
>> meant in
>>>>>> terms of
>>>>>>>>>>>>>>>>>>>> understanding
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance profile under various
>>>>>> circumstances. The
>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>> complex
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would
>> be for
>>>>>> users
>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> performance they see. For example,
>> spilling
>>>>>> records
>>>>>>>> to
>>>>>>>>>>>>>>>> disk
>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold
>>>> would, I
>>>>>>>>>> expect,
>>>>>>>>>>>>>>>>>> reduce
>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
>>>> throughput
>>>>>> could
>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>> unexpected,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> potentially difficult to
>>>> diagnose/understand for
>>>>>>>>>> users.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early
>> commit"
>>>>>> concept is
>>>>>>>>>>>>>>>>>> relatively
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document,
>> and
>>>>>>>>>>>> conceptually
>>>>>>>>>>>>>>>>>> fairly
>>>>>>>>>>>>>>>>>>>>>>>>>>>> obvious to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to
>>>> make it
>>>>>>>>>> easier
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it happens though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an
>>>> indirect
>>>>>> way
>>>>>>>> of
>>>>>>>>>>>>>>>>>> telling
>>>>>>>>>>>>>>>>>>>>>>>>>>>> StateStores
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
>>>>>> READ_COMMITTED
>>>>>>>>>>>>>>>>>>>> essentially
>>>>>>>>>>>>>>>>>>>>>>>>>>>> requires
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that
>> two
>>>>>> threads
>>>>>>>>>>>>>>>> calling
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes
>>>> from
>>>>>> the
>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> they have been committed. With
>>>>>> READ_UNCOMMITTED, all
>>>>>>>>>>>>>>>> bets are
>>>>>>>>>>>>>>>>>>>> off,
>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe
>> written
>>>>>> records
>>>>>>>> at
>>>>>>>>>>>>>>>> any
>>>>>>>>>>>>>>>>>>>> time,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> which is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That
>> said,
>>>>>>>> StateStores
>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>> free
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> these guarantees however they can,
>> which is
>>>> a
>>>>>> bit
>>>>>>>> more
>>>>>>>>>>>>>>>>>> relaxed
>>>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions".
>> For
>>>>>> example,
>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED ==
>>>> WBWI-based
>>>>>>>>>>>>>>>>>> "transactions",
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the
>>>>>> database.
>>>>>>>> But
>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>> storage
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to
>> *always*
>>>> use
>>>>>>>>>>>>>>>> transactions,
>>>>>>>>>>>>>>>>>>>> even
>>>>>>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage
>> engines
>>>>>> that
>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation
>> guarantees
>>>> can
>>>>>> be
>>>>>>>> met
>>>>>>>>>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> technique.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the
>> StateStore
>>>>>> interface
>>>>>>>> as
>>>>>>>>>>>>>>>>>> loosely
>>>>>>>>>>>>>>>>>>>>>>>> coupled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to
>> give
>>>>>>>>>>>>>>>> implementers
>>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>>>>>>>> freedom, and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge
>>>>>> required.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> That said, I understand that
>>>> "IsolationLevel"
>>>>>> might
>>>>>>>>>> not
>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it
>> much
>>>> more
>>>>>>>>>>>>>>>> explicit if
>>>>>>>>>>>>>>>>>>>>>>>>>>>> required, e.g.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7-8.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I can make these changes either later
>> today
>>>> or
>>>>>>>>>>>> tomorrow.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Small update:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and
>> fixed a
>>>>>> bunch of
>>>>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>> needed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests
>> pass,
>>>>>> which is
>>>>>>>>>>>>>>>>>> promising,
>>>>>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> need to undergo some performance
>> testing. I
>>>>>> haven't
>>>>>>>>>>>>>>>> (yet)
>>>>>>>>>>>>>>>>>>>> worked
>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff,
>> but I
>>>>>> would
>>>>>>>>>>>>>>>> expect
>>>>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
>>>> difference. The
>>>>>>>>>> branch
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
>>>>>>>>>> if
>>>>>>>>>>>>>>>>>> anyone
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno
>> Cadonna
>>>> <
>>>>>>>>>>>>>>>>>>>> cadonna@apache.org>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was
>> actually
>>>> also
>>>>>> my
>>>>>>>>>>>>>>>> point. I
>>>>>>>>>>>>>>>>>>>>>>>> understood
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion
>> path
>>>> as
>>>>>> a way
>>>>>>>>>> to
>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> early
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the
>>>> intent.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually
>> it is
>>>>>> public
>>>>>>>>>>>>>>>> API. My
>>>>>>>>>>>>>>>>>>>>>> question
>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal
>>>>>> processing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the
>> size
>>>> of the
>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>> buffer
>>>>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit
>> affects
>>>> the
>>>>>> whole
>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a
>> state
>>>> store
>>>>>> is
>>>>>>>> not
>>>>>>>>>>>>>>>>>>>> confined to
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and
>>>>>> changes the
>>>>>>>>>>>>>>>> behavior
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you
>> that
>>>> it
>>>>>> is a
>>>>>>>>>>>>>>>> decent
>>>>>>>>>>>>>>>>>>>>>>>>>>>> compromise. I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides
>> and
>>>> list
>>>>>> the
>>>>>>>>>>>>>>>> options to
>>>>>>>>>>>>>>>>>>>>>>>> overcome
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all
>>>> options
>>>>>> seem
>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>>> heavy
>>>>>>>>>>>>>>>>>>>>>>>>>>>> compared
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand
>> what you
>>>>>> mean
>>>>>>>> with
>>>>>>>>>>>>>>>> "less
>>>>>>>>>>>>>>>>>>>>>>>>>>>> predictable
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for users", though.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
>>>> alternatives
>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>> interesting.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> But I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good
>> and
>>>> we
>>>>>> should
>>>>>>>>>>>>>>>> continue
>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>> it!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my
>> e-mail on
>>>>>> June
>>>>>>>>>> 20th:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning
>> behind
>>>>>> putting
>>>>>>>>>>>>>>>> isolation
>>>>>>>>>>>>>>>>>>>>>> level
>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should
>>>> that
>>>>>> also
>>>>>>>> be
>>>>>>>>>> a
>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> give
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to
>> decide
>>>>>> whether
>>>>>>>> to
>>>>>>>>>>>>>>>> turn on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned
>> about
>>>>>> how do
>>>>>>>>>> you
>>>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written
>> under
>>>> EOS,
>>>>>> if
>>>>>>>> you
>>>>>>>>>>>>>>>> do not
>>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
>>>> transactional or
>>>>>> not.
>>>>>>>>>> If
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can
>> be
>>>>>> written
>>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>>>>> normal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state
>> store
>>>> is not
>>>>>>>>>>>>>>>>>> transactional,
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written
>> under
>>>> EOS.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 7.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My point was about not only
>> considering the
>>>>>> bytes
>>>>>>>> in
>>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but
>> also
>>>>>> bytes
>>>>>>>> that
>>>>>>>>>>>>>>>> might
>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>>> spilled
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering
>>>> whether you
>>>>>>>>>> should
>>>>>>>>>>>>>>>>>> remove
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory
>>>> bytes to
>>>>>> be
>>>>>>>>>> used
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store
>> records." My
>>>>>>>> thinking
>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> even
>>>>>>>>>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to
>>>> disk,
>>>>>>>>>> limiting
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>>>>>>>>>> bytes
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it
>> again
>>>> and
>>>>>>>>>>>>>>>> considering
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> recent
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much
>>>> sense
>>>>>>>> anymore.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I like the name
>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes that
>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>> proposed.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 8.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
>>>>>> implementation
>>>>>>>>>>>>>>>> details) of
>>>>>>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of
>> changelog
>>>>>>>>>>>>>>>> transactions,
>>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be
>>>> great.
>>>>>>>> Would
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> great
>>>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the
>>>>>> behavior in
>>>>>>>>>>>>>>>> case of
>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional
>> state
>>>>>> store
>>>>>>>>>>>>>>>> recover
>>>>>>>>>>>>>>>>>>>> after a
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
>>>> transaction
>>>>>>>> buffer,
>>>>>>>>>>>>>>>> etc.
>>>>>>>>>>>>>>>>>>>> (that
>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
>>>>>>>>>>>>>>>> WriteBatchWithIndex
>>>>>>>>>>>>>>>>>>>>>>>>>>>> transactions
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to
>>>> exceed)
>>>>>>>>>>>>>>>> configured
>>>>>>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>>>>>>>>>>>> needs to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
>>>> Ultimately,
>>>>>> any
>>>>>>>>>>>>>>>> approach
>>>>>>>>>>>>>>>>>>>> based
>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> cracking
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in
>>>> ways
>>>>>> it's
>>>>>>>> not
>>>>>>>>>>>>>>>> really
>>>>>>>>>>>>>>>>>>>>>>>> designed
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> for is
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen
>> performance
>>>> or
>>>>>>>>>>>>>>>> consistency
>>>>>>>>>>>>>>>>>>>> issues.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing
>> these
>>>>>> early
>>>>>>>>>>>>>>>> commits?
>>>>>>>>>>>>>>>>>>>> While
>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ideal, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to
>>>> ensure
>>>>>>>>>>>>>>>> consistency
>>>>>>>>>>>>>>>>>>>> whilst
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> maintaining
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem
>> *very*
>>>>>>>>>>>>>>>> complicated, and
>>>>>>>>>>>>>>>>>>>> might
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for
>> users
>>>> as a
>>>>>>>>>>>>>>>> consequence.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of
>> this
>>>>>> KIP is
>>>>>>>>>>>>>>>> growing a
>>>>>>>>>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss
>> ideas
>>>> for
>>>>>>>> future
>>>>>>>>>>>>>>>>>>>>>>>> improvements, I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> think
>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope
>> down
>>>> to a
>>>>>>>> design
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> achieves
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> most
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
>>>>>> restorations
>>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>>> dirty
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design
>> that
>>>>>> this KIP
>>>>>>>>>>>>>>>> produces
>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ultimately
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially
>> if
>>>> the
>>>>>> bulk
>>>>>>>> of
>>>>>>>>>>>>>>>> it is
>>>>>>>>>>>>>>>>>>>>>> internal
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week
>>>> trying
>>>>>> to
>>>>>>>>>>>>>>>> re-work
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove
>> the
>>>>>>>>>>>>>>>> newTransaction()
>>>>>>>>>>>>>>>>>>>>>> method,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> such
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
>>>>>>>> RocksDBStore.
>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>> way, if
>>>>>>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> want to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the
>> future,
>>>>>> like
>>>>>>>> the
>>>>>>>>>>>>>>>> SST
>>>>>>>>>>>>>>>>>> file
>>>>>>>>>>>>>>>>>>>>>>>>>>>> management
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so
>> with
>>>>>> little/no
>>>>>>>>>>>> API
>>>>>>>>>>>>>>>>>>>> changes.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>>
>>>>
>>
> 

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

Posted by Lucas Brutschy <lb...@confluent.io.INVALID>.
HI Nick,

what I meant was, why don't you leave the behavior of Kafka Streams in
this case as is (wipe the state, abort the transaction), since the
contribution of the KIP is to allow transactional state stores, not to
eliminate all cases of state wiping in Kafka Streams. But either way,
that's something that could be discussed in the PR, not the KIP.

Cheers,
Lucas

On Wed, Oct 18, 2023 at 3:58 PM Nick Telford <ni...@gmail.com> wrote:
>
> Hi Lucas,
>
> TaskCorruptedException is how Streams signals that the Task state needs to
> be wiped, so we can't retain that exception without also wiping state on
> timeouts.
>
> Regards,
> Nick
>
> On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <lb...@confluent.io.invalid>
> wrote:
>
> > Hi Nick,
> >
> > I think indeed the better behavior would be to retry commitTransaction
> > until we risk running out of time to meet `max.poll.interval.ms`.
> >
> > However, if it's handled as a `TaskCorruptedException` at the moment,
> > I would do the same in this KIP, and leave exception handling
> > improvements to future work. 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 <ni...@gmail.com>
> > wrote:
> > >
> > > Hi Lucas,
> > >
> > > Yeah, this is pretty much the direction I'm thinking of going in now. You
> > > make an interesting point about committing on-error under
> > > ALOS/READ_COMMITTED, although I haven't had a chance to think through the
> > > implications yet.
> > >
> > > Something that I ran into earlier this week is an issue with the new
> > > handling of TimeoutException. Without TX stores, TimeoutException under
> > EOS
> > > throws a TaskCorruptedException, which wipes the stores. However, with TX
> > > stores, TimeoutException is now just bubbled up and dealt with as it is
> > > under ALOS. The problem arises when the Producer#commitTransaction call
> > > times out: Streams attempts to ignore the error and continue producing,
> > > which causes the next call to Producer#send to throw
> > > "IllegalStateException: Cannot attempt operation `send` because the
> > > previous call to `commitTransaction` timed out and must be retried".
> > >
> > > I'm not sure what we should do here: retrying the commitTransaction seems
> > > logical, but what if it times out again? Where do we draw the line and
> > > shutdown the instance?
> > >
> > > Regards,
> > > Nick
> > >
> > > On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <lbrutschy@confluent.io
> > .invalid>
> > > wrote:
> > >
> > > > Hi all,
> > > >
> > > > I think I liked your suggestion of allowing EOS with READ_UNCOMMITTED,
> > > > but keep wiping the state on error, and I'd vote for this solution
> > > > when introducing `default.state.isolation.level`. This way, we'd have
> > > > the most low-risk roll-out of this feature (no behavior change without
> > > > reconfiguration), with the possibility of switching to the most sane /
> > > > battle-tested default settings in 4.0. Essentially, we'd have a
> > > > feature flag but call it `default.state.isolation.level` and don't
> > > > have to deprecate it later.
> > > >
> > > > So the possible configurations would then be this:
> > > >
> > > > 1. ALOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
> > > > reads from DB.
> > > > 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> > > > WriteBatch/DB. Flush on error (see note below).
> > > > 3. EOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
> > > > reads from DB. Wipe state on error.
> > > > 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> > > > WriteBatch/DB.
> > > >
> > > > I believe the feature is important enough that we will see good
> > > > adoption even without changing the default. In 4.0, when we have seen
> > > > this being adopted and is battle-tested, we make READ_COMMITTED the
> > > > default for EOS, or even READ_COMITTED always the default, depending
> > > > on our experiences. And we could add a clever implementation of
> > > > READ_UNCOMITTED with WriteBatches later.
> > > >
> > > > The only smell here is that `default.state.isolation.level` wouldn't
> > > > be purely an IQ setting, but it would also (slightly) change the
> > > > behavior of the processing, but that seems unavoidable as long as we
> > > > haven't solve READ_UNCOMITTED IQ with WriteBatches.
> > > >
> > > > Minor: As for Bruno's point 4, I think if we are concerned about this
> > > > 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).
> > > >
> > > > Cheers,
> > > > Lucas
> > > >
> > > > On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <ni...@gmail.com>
> > > > wrote:
> > > > >
> > > > > Hi Guozhang,
> > > > >
> > > > > The KIP as it stands introduces a new configuration,
> > > > > default.state.isolation.level, which is independent of
> > processing.mode.
> > > > > It's intended that this new configuration be used to configure a
> > global
> > > > IQ
> > > > > isolation level in the short term, with a future KIP introducing the
> > > > > capability to change the isolation level on a per-query basis,
> > falling
> > > > back
> > > > > to the "default" defined by this config. That's why I called it
> > > > "default",
> > > > > for future-proofing.
> > > > >
> > > > > However, it currently includes the caveat that READ_UNCOMMITTED is
> > not
> > > > > available under EOS. I think this is the coupling you are alluding
> > to?
> > > > >
> > > > > This isn't intended to be a restriction of the API, but is currently
> > a
> > > > > technical limitation. However, after discussing with some users about
> > > > > use-cases that would require READ_UNCOMMITTED under EOS, I'm
> > inclined to
> > > > > remove that clause and put in the necessary work to make that
> > combination
> > > > > possible now.
> > > > >
> > > > > I currently see two possible approaches:
> > > > >
> > > > >    1. Disable TX StateStores internally when the IsolationLevel is
> > > > >    READ_UNCOMMITTED and the processing.mode is EOS. This is more
> > > > difficult
> > > > >    than it sounds, as there are many assumptions being made
> > throughout
> > > > the
> > > > >    internals about the guarantees StateStores provide. It would
> > > > definitely add
> > > > >    a lot of extra "if (read_uncommitted && eos)" branches,
> > complicating
> > > > >    maintenance and testing.
> > > > >    2. Invest the time *now* to make READ_UNCOMMITTED of EOS
> > StateStores
> > > > >    possible. I have some ideas on how this could be achieved, but
> > they
> > > > would
> > > > >    need testing and could introduce some additional issues. The
> > benefit
> > > > of
> > > > >    this approach is that it would make query-time IsolationLevels
> > much
> > > > simpler
> > > > >    to implement in the future.
> > > > >
> > > > > Unfortunately, both will require considerable work that will further
> > > > delay
> > > > > this KIP, which was the reason I placed the restriction in the KIP
> > in the
> > > > > first place.
> > > > >
> > > > > Regards,
> > > > > Nick
> > > > >
> > > > > On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
> > guozhang.wang.us@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hello Nick,
> > > > > >
> > > > > > First of all, thanks a lot for the great effort you've put in
> > driving
> > > > > > this KIP! I really like it coming through finally, as many people
> > in
> > > > > > the community have raised this. At the same time I honestly feel a
> > bit
> > > > > > ashamed for not putting enough of my time supporting it and
> > pushing it
> > > > > > through the finish line (you raised this KIP almost a year ago).
> > > > > >
> > > > > > I briefly passed through the DISCUSS thread so far, not sure I've
> > 100
> > > > > > percent digested all the bullet points. But with the goal of
> > trying to
> > > > > > help take it through the finish line in mind, I'd want to throw
> > > > > > thoughts on top of my head only on the point #4 above which I felt
> > may
> > > > > > be the main hurdle for the current KIP to drive to a consensus now.
> > > > > >
> > > > > > The general question I asked myself is, whether we want to couple
> > "IQ
> > > > > > reading mode" with "processing mode". While technically I tend to
> > > > > > agree with you that, it's feels like a bug if some single user
> > chose
> > > > > > "EOS" for processing mode while choosing "read uncommitted" for IQ
> > > > > > reading mode, at the same time, I'm thinking if it's possible that
> > > > > > there could be two different persons (or even two teams) that
> > would be
> > > > > > using the stream API to build the app, and the IQ API to query the
> > > > > > running state of the app. I know this is less of a technical thing
> > but
> > > > > > rather a more design stuff, but if it could be ever the case, I'm
> > > > > > wondering if the personale using the IQ API knows about the risks
> > of
> > > > > > using read uncommitted but still chose so for the favor of
> > > > > > performance, no matter if the underlying stream processing mode
> > > > > > configured by another personale is EOS or not. In that regard, I'm
> > > > > > leaning towards a "leaving the door open, and close it later if we
> > > > > > found it's a bad idea" aspect with a configuration 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 can still enforce
> > > > > > coupling them.
> > > > > >
> > > > > > Again, just my 2c here. Thanks again for the great patience and
> > > > > > diligence on this KIP.
> > > > > >
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
> > nick.telford@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > Hi Bruno,
> > > > > > >
> > > > > > > 4.
> > > > > > > I'll hold off on making that change until we have a consensus as
> > to
> > > > what
> > > > > > > configuration to use to control all of this, as it'll be
> > affected by
> > > > the
> > > > > > > decision on EOS isolation levels.
> > > > > > >
> > > > > > > 5.
> > > > > > > Done. I've chosen "committedOffsets".
> > > > > > >
> > > > > > > Regards,
> > > > > > > Nick
> > > > > > >
> > > > > > > On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <ca...@apache.org>
> > > > wrote:
> > > > > > >
> > > > > > > > Hi Nick,
> > > > > > > >
> > > > > > > > 1.
> > > > > > > > Yeah, you are probably right that it does not make too much
> > sense.
> > > > > > > > Thanks for the clarification!
> > > > > > > >
> > > > > > > >
> > > > > > > > 4.
> > > > > > > > Yes, sorry for the back and forth, but I think for the sake of
> > the
> > > > KIP
> > > > > > > > it is better to let the ALOS behavior as it is for now due to
> > the
> > > > > > > > possible issues you would run into. Maybe we can find a
> > solution
> > > > in the
> > > > > > > > future. Now the question returns to whether we really need
> > > > > > > > default.state.isolation.level. Maybe the config could be the
> > > > feature
> > > > > > > > flag Sophie requested.
> > > > > > > >
> > > > > > > >
> > > > > > > > 5.
> > > > > > > > There 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 committedOffsetFor().
> > > > > > > >
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Bruno
> > > > > > > >
> > > > > > > >
> > > > > > > > On 10/13/23 3:21 PM, Nick Telford wrote:
> > > > > > > > > Hi Bruno,
> > > > > > > > >
> > > > > > > > > Thanks for getting back to me.
> > > > > > > > >
> > > > > > > > > 1.
> > > > > > > > > I think this should be possible. Are you thinking of the
> > > > situation
> > > > > > where
> > > > > > > > a
> > > > > > > > > user may downgrade to a previous version of Kafka Streams? In
> > > > that
> > > > > > case,
> > > > > > > > > sadly, the RocksDBStore would get wiped by the older version
> > of
> > > > Kafka
> > > > > > > > > Streams anyway, because that version wouldn't understand the
> > > > extra
> > > > > > column
> > > > > > > > > family (that holds offsets), so the missing Position file
> > would
> > > > > > > > > automatically get rebuilt when the store is rebuilt from the
> > > > > > changelog.
> > > > > > > > > Are there other situations than downgrade where a
> > transactional
> > > > store
> > > > > > > > could
> > > > > > > > > be replaced by a non-transactional one? I can't think of any.
> > > > > > > > >
> > > > > > > > > 2.
> > > > > > > > > Ahh yes, the Test Plan - my Kryptonite! This section
> > definitely
> > > > > > needs to
> > > > > > > > be
> > > > > > > > > fleshed out. I'll work on that. How much detail do you need?
> > > > > > > > >
> > > > > > > > > 3.
> > > > > > > > > See my previous email discussing this.
> > > > > > > > >
> > > > > > > > > 4.
> > > > > > > > > Hmm, this is an interesting point. Are you suggesting that
> > under
> > > > ALOS
> > > > > > > > > READ_COMMITTED should not be supported?
> > > > > > > > >
> > > > > > > > > Regards,
> > > > > > > > > Nick
> > > > > > > > >
> > > > > > > > > On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
> > cadonna@apache.org>
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > >> Hi Nick,
> > > > > > > > >>
> > > > > > > > >> I think the KIP is converging!
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> 1.
> > > > > > > > >> I am wondering whether it makes sense to write the position
> > file
> > > > > > during
> > > > > > > > >> close as we do for the checkpoint file, so that in case the
> > > > state
> > > > > > store
> > > > > > > > >> is replaced with a non-transactional state store the
> > > > > > non-transactional
> > > > > > > > >> state store finds the position file. I think, this is not
> > > > strictly
> > > > > > > > >> needed, but would be a nice behavior instead of just
> > deleting
> > > > the
> > > > > > > > >> position file.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> 2.
> > > > > > > > >> The test plan does not mention integration tests. Do you not
> > > > need to
> > > > > > > > >> extend existing ones and add new ones. Also for upgrading
> > and
> > > > > > > > >> downgrading you might need integration and/or system tests.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> 3.
> > > > > > > > >> I think Sophie made a point. Although, IQ reading from
> > > > uncommitted
> > > > > > data
> > > > > > > > >> under EOS might be considered a bug by some people. Thus,
> > your
> > > > KIP
> > > > > > would
> > > > > > > > >> fix a bug rather than changing the intended behavior.
> > However, I
> > > > > > also
> > > > > > > > >> see that a feature flag would help users that rely on this
> > buggy
> > > > > > > > >> behavior (at least until AK 4.0).
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> 4.
> > > > > > > > >> This is related to the previous point. I assume that the
> > > > difference
> > > > > > > > >> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is
> > that in
> > > > the
> > > > > > > > >> former you enable transactions on the state store and in the
> > > > latter
> > > > > > you
> > > > > > > > >> disable them. If my assumption is correct, I think that is
> > an
> > > > issue.
> > > > > > > > >> Let's assume under ALOS Streams fails over a couple of times
> > > > more or
> > > > > > > > >> less at the same step in processing after value 3 is added
> > to an
> > > > > > > > >> aggregation but the offset of the corresponding input record
> > > > was not
> > > > > > > > >> committed. Without transactions disabled, the aggregation
> > value
> > > > > > would
> > > > > > > > >> increase by 3 for each failover. With transactions enabled,
> > > > value 3
> > > > > > > > >> would only be added to the aggregation once when the offset
> > of
> > > > the
> > > > > > input
> > > > > > > > >> record is committed and the transaction finally completes.
> > So
> > > > the
> > > > > > > > >> content of the state store would change depending on the
> > > > > > configuration
> > > > > > > > >> for IQ. IMO, the content of the state store should be
> > > > independent
> > > > > > from
> > > > > > > > >> IQ. Given this issue, I propose to not use transactions with
> > > > ALOS at
> > > > > > > > >> all. I was a big proponent of using transactions with ALOS,
> > but
> > > > I
> > > > > > > > >> realized that transactions with ALOS is not as easy as
> > enabling
> > > > > > > > >> transactions on state stores. Another aspect that is
> > > > problematic is
> > > > > > that
> > > > > > > > >> the changelog topic which actually replicates the state
> > store
> > > > is not
> > > > > > > > >> transactional under ALOS. Thus, it might happen that the
> > state
> > > > > > store and
> > > > > > > > >> the changelog differ in their content. All of this is maybe
> > > > solvable
> > > > > > > > >> somehow, but for the sake of this KIP, I would leave it for
> > the
> > > > > > future.
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> Best,
> > > > > > > > >> Bruno
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >>
> > > > > > > > >> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > > > > > > > >>> Hey Nick! First of all thanks for taking up this awesome
> > > > feature,
> > > > > > I'm
> > > > > > > > >> sure
> > > > > > > > >>> every single
> > > > > > > > >>> Kafka Streams user and dev would agree that it is sorely
> > > > needed.
> > > > > > > > >>>
> > > > > > > > >>> I've just been catching up on the KIP and surrounding
> > > > discussion,
> > > > > > so
> > > > > > > > >> please
> > > > > > > > >>> forgive me
> > > > > > > > >>> for any misunderstandings or misinterpretations of the
> > current
> > > > > > plan and
> > > > > > > > >>> don't hesitate to
> > > > > > > > >>> correct me.
> > > > > > > > >>>
> > > > > > > > >>> Before I jump in, I just want to say that having seen this
> > > > drag on
> > > > > > for
> > > > > > > > so
> > > > > > > > >>> long, my singular
> > > > > > > > >>> goal in responding is to help this KIP past a perceived
> > > > impasse so
> > > > > > we
> > > > > > > > can
> > > > > > > > >>> finally move on
> > > > > > > > >>> to voting and implementing it. Long discussions are to be
> > > > expected
> > > > > > for
> > > > > > > > >>> major features like
> > > > > > > > >>> this but it's completely on us as the Streams devs to make
> > sure
> > > > > > there
> > > > > > > > is
> > > > > > > > >> an
> > > > > > > > >>> end in sight
> > > > > > > > >>> for any ongoing discussion.
> > > > > > > > >>>
> > > > > > > > >>> With that said, it's my understanding that the KIP as
> > currently
> > > > > > > > proposed
> > > > > > > > >> is
> > > > > > > > >>> just not tenable
> > > > > > > > >>> for Kafka Streams, and would prevent some EOS users from
> > > > upgrading
> > > > > > to
> > > > > > > > the
> > > > > > > > >>> version it
> > > > > > > > >>> first appears in. Given that we can't predict or guarantee
> > > > whether
> > > > > > any
> > > > > > > > of
> > > > > > > > >>> the followup KIPs
> > > > > > > > >>> would be completed in the same release cycle as this one,
> > we
> > > > need
> > > > > > to
> > > > > > > > make
> > > > > > > > >>> sure that the
> > > > > > > > >>> feature is either compatible with all current users or else
> > > > > > > > >> feature-flagged
> > > > > > > > >>> so that they may
> > > > > > > > >>> opt in/out.
> > > > > > > > >>>
> > > > > > > > >>> Therefore, IIUC we need to have either (or both) of these
> > as
> > > > > > > > >>> fully-implemented config options:
> > > > > > > > >>> 1. default.state.isolation.level
> > > > > > > > >>> 2. enable.transactional.state.stores
> > > > > > > > >>>
> > > > > > > > >>> This way EOS users for whom read_committed semantics are
> > not
> > > > > > viable can
> > > > > > > > >>> still upgrade,
> > > > > > > > >>> and either use the isolation.level config to leverage the
> > new
> > > > txn
> > > > > > state
> > > > > > > > >>> stores without sacrificing
> > > > > > > > >>> their application semantics, or else simply keep the
> > > > transactional
> > > > > > > > state
> > > > > > > > >>> stores disabled until we
> > > > > > > > >>> are able to fully implement the isolation level
> > configuration
> > > > at
> > > > > > either
> > > > > > > > >> an
> > > > > > > > >>> application or query level.
> > > > > > > > >>>
> > > > > > > > >>> Frankly you are the expert here and know much more about
> > the
> > > > > > tradeoffs
> > > > > > > > in
> > > > > > > > >>> both semantics and
> > > > > > > > >>> effort level of implementing one of these configs vs the
> > > > other. In
> > > > > > my
> > > > > > > > >>> opinion, either option would
> > > > > > > > >>> be fine and I would leave the decision of which one to
> > include
> > > > in
> > > > > > this
> > > > > > > > >> KIP
> > > > > > > > >>> completely up to you.
> > > > > > > > >>> I just don't see a way for the KIP to proceed without some
> > > > > > variation of
> > > > > > > > >> the
> > > > > > > > >>> above that would allow
> > > > > > > > >>> EOS users to opt-out of read_committed.
> > > > > > > > >>>
> > > > > > > > >>> (If it's all the same to you, I would recommend 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 that gets released.
> > > > > > > > >>> So personally, my choice would be to add the feature flag
> > and
> > > > > > leave it
> > > > > > > > >> off
> > > > > > > > >>> by default. If all 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
> > > > > > > > >>>
> > > > > > > > >>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > > > > > nick.telford@gmail.com>
> > > > > > > > >> wrote:
> > > > > > > > >>>
> > > > > > > > >>>> Hi Bruno,
> > > > > > > > >>>>
> > > > > > > > >>>> Agreed, I can live with that for now.
> > > > > > > > >>>>
> > > > > > > > >>>> In an effort to keep the scope of this KIP from
> > expanding, I'm
> > > > > > leaning
> > > > > > > > >>>> towards just providing a configurable
> > > > > > default.state.isolation.level
> > > > > > > > and
> > > > > > > > >>>> removing IsolationLevel from the StateStoreContext. This
> > > > would be
> > > > > > > > >>>> compatible with adding support for query-time
> > IsolationLevels
> > > > in
> > > > > > the
> > > > > > > > >>>> future, whilst providing a way for users to select an
> > > > isolation
> > > > > > level
> > > > > > > > >> now.
> > > > > > > > >>>>
> > > > > > > > >>>> The big problem with this, however, is that if a user
> > selects
> > > > > > > > >>>> processing.mode
> > > > > > > > >>>> = "exactly-once(-v2|-beta)", and
> > > > default.state.isolation.level =
> > > > > > > > >>>> "READ_UNCOMMITTED", we need to guarantee that the data
> > isn't
> > > > > > written
> > > > > > > > to
> > > > > > > > >>>> disk until commit() is called, but we also need to permit
> > IQ
> > > > > > threads
> > > > > > > > to
> > > > > > > > >>>> read from the ongoing transaction.
> > > > > > > > >>>>
> > > > > > > > >>>> A simple solution would be to (temporarily) forbid this
> > > > > > combination of
> > > > > > > > >>>> configuration, and have default.state.isolation.level
> > > > > > automatically
> > > > > > > > >> switch
> > > > > > > > >>>> to READ_COMMITTED when processing.mode is anything other
> > than
> > > > > > > > >>>> at-least-once. Do you think this would be acceptable?
> > > > > > > > >>>>
> > > > > > > > >>>> In a later KIP, we can add support for query-time
> > isolation
> > > > > > levels and
> > > > > > > > >>>> solve this particular problem there, which would relax
> > this
> > > > > > > > restriction.
> > > > > > > > >>>>
> > > > > > > > >>>> Regards,
> > > > > > > > >>>> Nick
> > > > > > > > >>>>
> > > > > > > > >>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> > > > cadonna@apache.org>
> > > > > > > > >> 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,
> > > > > > > > >>>>> Bruno
> > > > > > > > >>>>>
> > > > > > > > >>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > > > > > > > >>>>>> 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 <
> > > > > > nick.telford@gmail.com
> > > > > > > > >
> > > > > > > > >>>>> wrote:
> > > > > > > > >>>>>>
> > > > > > > > >>>>>>> Hi everyone,
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> I agree that having IsolationLevel be determined at
> > > > query-time
> > > > > > is
> > > > > > > > the
> > > > > > > > >>>>>>> ideal design, but there are a few sticking points:
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> 1.
> > > > > > > > >>>>>>> There needs to be some way to communicate the
> > > > IsolationLevel
> > > > > > down
> > > > > > > > to
> > > > > > > > >>>> the
> > > > > > > > >>>>>>> RocksDBStore itself, so that the query can respect it.
> > > > Since
> > > > > > stores
> > > > > > > > >>>> are
> > > > > > > > >>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
> > > > > > MeteredStore,
> > > > > > > > >>>>> etc.),
> > > > > > > > >>>>>>> we need some way to deliver that information to the
> > bottom
> > > > > > layer.
> > > > > > > > For
> > > > > > > > >>>>> IQv2,
> > > > > > > > >>>>>>> we can use the existing State#query() method, but IQv1
> > has
> > > > no
> > > > > > way
> > > > > > > > to
> > > > > > > > >>>> do
> > > > > > > > >>>>>>> this.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> A simple approach, which would potentially open up
> > other
> > > > > > options,
> > > > > > > > >>>> would
> > > > > > > > >>>>> be
> > > > > > > > >>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> > > > > > > > >>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > > > > > > > ReadOnlyKeyValueStore
> > > > > > > > >>>>> (and
> > > > > > > > >>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore,
> > > > etc.).
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> 2.
> > > > > > > > >>>>>>> As mentioned above, RocksDB WriteBatches are not
> > > > thread-safe,
> > > > > > which
> > > > > > > > >>>>> causes
> > > > > > > > >>>>>>> a problem if we want to provide READ_UNCOMMITTED
> > > > Iterators. I
> > > > > > also
> > > > > > > > >>>> had a
> > > > > > > > >>>>>>> look at RocksDB Transactions[1], but they solve a very
> > > > > > different
> > > > > > > > >>>>> problem,
> > > > > > > > >>>>>>> and have the same thread-safety issue.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> One possible approach that I mentioned is chaining
> > > > > > WriteBatches:
> > > > > > > > >> every
> > > > > > > > >>>>>>> time a new Interactive Query is received (i.e.
> > > > readOnlyView,
> > > > > > see
> > > > > > > > >>>> above,
> > > > > > > > >>>>>>> is called) we "freeze" the existing WriteBatch, and
> > start a
> > > > > > new one
> > > > > > > > >>>> for
> > > > > > > > >>>>> new
> > > > > > > > >>>>>>> writes. The Interactive Query queries the "chain" of
> > > > previous
> > > > > > > > >>>>> WriteBatches
> > > > > > > > >>>>>>> + the underlying database; while the StreamThread
> > starts
> > > > > > writing to
> > > > > > > > >>>> the
> > > > > > > > >>>>>>> *new* WriteBatch. On-commit, the StreamThread would
> > write
> > > > *all*
> > > > > > > > >>>>>>> WriteBatches in the chain to the database (that have
> > not
> > > > yet
> > > > > > been
> > > > > > > > >>>>> written).
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> WriteBatches would be closed/freed only when they have
> > been
> > > > > > both
> > > > > > > > >>>>>>> committed, and all open Interactive Queries on them
> > have
> > > > been
> > > > > > > > closed.
> > > > > > > > >>>>> This
> > > > > > > > >>>>>>> would require some reference counting.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Obviously a drawback of this approach is the potential
> > for
> > > > > > > > increased
> > > > > > > > >>>>>>> memory usage: if an Interactive Query is long-lived,
> > for
> > > > > > example by
> > > > > > > > >>>>> doing a
> > > > > > > > >>>>>>> full scan over a large database, or even just pausing
> > in
> > > > the
> > > > > > middle
> > > > > > > > >> of
> > > > > > > > >>>>> an
> > > > > > > > >>>>>>> iteration, then the existing chain of WriteBatches
> > could be
> > > > > > kept
> > > > > > > > >>>> around
> > > > > > > > >>>>> for
> > > > > > > > >>>>>>> a long time, potentially forever.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> --
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> A.
> > > > > > > > >>>>>>> Going off on a tangent, it looks like in addition to
> > > > supporting
> > > > > > > > >>>>>>> READ_COMMITTED queries, we could go further and support
> > > > > > > > >>>> REPEATABLE_READ
> > > > > > > > >>>>>>> queries (i.e. where subsequent reads to the same key
> > in the
> > > > > > same
> > > > > > > > >>>>>>> Interactive Query are guaranteed to yield the same
> > value)
> > > > by
> > > > > > making
> > > > > > > > >>>> use
> > > > > > > > >>>>> of
> > > > > > > > >>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so
> > the
> > > > > > > > >> performance
> > > > > > > > >>>>>>> impact is likely to be negligible, but they do require
> > > > that the
> > > > > > > > >>>>> Interactive
> > > > > > > > >>>>>>> Query session can be explicitly closed.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> This could be achieved if we made the above
> > readOnlyView
> > > > > > interface
> > > > > > > > >>>> look
> > > > > > > > >>>>>>> more like:
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> > > > > > > > >>>> ReadOnlyKeyValueStore<K,
> > > > > > > > >>>>>>> V>, AutoCloseable {}
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > > > > > > > >>>>>>>        ...
> > > > > > > > >>>>>>>        ReadOnlyKeyValueView<K, V>
> > > > readOnlyView(IsolationLevel
> > > > > > > > >>>>> isolationLevel);
> > > > > > > > >>>>>>> }
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> But this would be a breaking change, as existing IQv1
> > > > queries
> > > > > > are
> > > > > > > > >>>>>>> guaranteed to never call store.close(), and therefore
> > these
> > > > > > would
> > > > > > > > >> leak
> > > > > > > > >>>>>>> memory under REPEATABLE_READ.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> B.
> > > > > > > > >>>>>>> One thing that's notable: MyRocks states that they
> > support
> > > > > > > > >>>>> READ_COMMITTED
> > > > > > > > >>>>>>> and REPEATABLE_READ, but they make no mention of
> > > > > > > > >>>> READ_UNCOMMITTED[3][4].
> > > > > > > > >>>>>>> This could be because doing so is technically
> > > > > > difficult/impossible
> > > > > > > > >>>> using
> > > > > > > > >>>>>>> the primitives available in RocksDB.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> --
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Lucas, to address your points:
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> U1.
> > > > > > > > >>>>>>> It's only "SHOULD" to permit alternative (i.e.
> > non-RocksDB)
> > > > > > > > >>>>>>> implementations of StateStore that do not support
> > atomic
> > > > > > writes.
> > > > > > > > >>>>> Obviously
> > > > > > > > >>>>>>> in those cases, the guarantees Kafka Streams
> > > > provides/expects
> > > > > > would
> > > > > > > > >> be
> > > > > > > > >>>>>>> relaxed. Do you think we should require all
> > > > implementations to
> > > > > > > > >> support
> > > > > > > > >>>>>>> atomic writes?
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> U2.
> > > > > > > > >>>>>>> Stores can support multiple IsolationLevels. As we've
> > > > discussed
> > > > > > > > >> above,
> > > > > > > > >>>>> the
> > > > > > > > >>>>>>> ideal scenario would be to specify the IsolationLevel
> > at
> > > > > > > > query-time.
> > > > > > > > >>>>>>> Failing that, I think the second-best approach is to
> > > > define the
> > > > > > > > >>>>>>> IsolationLevel for *all* queries based on the
> > > > processing.mode,
> > > > > > > > which
> > > > > > > > >>>> is
> > > > > > > > >>>>>>> what the default StateStoreContext#isolationLevel()
> > > > achieves.
> > > > > > Would
> > > > > > > > >>>> you
> > > > > > > > >>>>>>> prefer an alternative?
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> While the existing implementation is equivalent to
> > > > > > > > READ_UNCOMMITTED,
> > > > > > > > >>>>> this
> > > > > > > > >>>>>>> can yield unexpected results/errors under EOS, if a
> > > > > > transaction is
> > > > > > > > >>>>> rolled
> > > > > > > > >>>>>>> back. While this would be a change in behaviour for
> > users,
> > > > it
> > > > > > would
> > > > > > > > >>>> look
> > > > > > > > >>>>>>> more like a bug fix than a breaking change. That said,
> > we
> > > > > > *could*
> > > > > > > > >> make
> > > > > > > > >>>>> it
> > > > > > > > >>>>>>> configurable, and default to the existing behaviour
> > > > > > > > >> (READ_UNCOMMITTED)
> > > > > > > > >>>>>>> instead of inferring it from the processing.mode?
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> N1, N2.
> > > > > > > > >>>>>>> These were only primitives to avoid boxing costs, but
> > since
> > > > > > this is
> > > > > > > > >>>> not
> > > > > > > > >>>>> a
> > > > > > > > >>>>>>> performance sensitive area, it should be fine to
> > change if
> > > > > > that's
> > > > > > > > >>>>> desirable.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> N3.
> > > > > > > > >>>>>>> It's because the store "manages its own offsets", which
> > > > > > includes
> > > > > > > > both
> > > > > > > > >>>>>>> committing the offset, *and providing it* via
> > > > > > getCommittedOffset().
> > > > > > > > >>>>>>> Personally, I think "managesOffsets" conveys this best,
> > > > but I
> > > > > > don't
> > > > > > > > >>>> mind
> > > > > > > > >>>>>>> changing it if the nomenclature is unclear.
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> Sorry for the massive emails/essays!
> > > > > > > > >>>>>>> --
> > > > > > > > >>>>>>> Nick
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> 1:
> > https://github.com/facebook/rocksdb/wiki/Transactions
> > > > > > > > >>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > > > > > > > >>>>>>> 3:
> > > > > > > >
> > https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > > > > > > >>>>>>> 4:
> > > > https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > > > > > > > >>>>>>> <lb...@confluent.io.invalid> wrote:
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>>> Hi Nick,
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> since I last read it in April, the KIP has become much
> > > > > > cleaner and
> > > > > > > > >>>>>>>> easier to read. Great work!
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> It feels to me the last big open point is whether we
> > can
> > > > > > implement
> > > > > > > > >>>>>>>> isolation level as a query parameter. I understand
> > that
> > > > there
> > > > > > are
> > > > > > > > >>>>>>>> implementation concerns, but as Colt says, it would
> > be a
> > > > great
> > > > > > > > >>>>>>>> addition, and would also simplify the migration path
> > for
> > > > this
> > > > > > > > >> change.
> > > > > > > > >>>>>>>> Is the implementation problem you mentioned caused by
> > the
> > > > > > > > WriteBatch
> > > > > > > > >>>>>>>> not having a notion of a snapshot, as the underlying
> > DB
> > > > > > iterator
> > > > > > > > >>>> does?
> > > > > > > > >>>>>>>> In that case, I am not sure a chain of WriteBatches
> > as you
> > > > > > propose
> > > > > > > > >>>>>>>> would fully solve the problem, but maybe I didn't dig
> > > > enough
> > > > > > into
> > > > > > > > >> the
> > > > > > > > >>>>>>>> details to fully understand it.
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> If it's not possible to implement it now, would it be
> > an
> > > > > > option to
> > > > > > > > >>>>>>>> make sure in this KIP that we do not fully close the
> > door
> > > > on
> > > > > > > > >>>> per-query
> > > > > > > > >>>>>>>> isolation levels in the interface, as it may be
> > possible
> > > > to
> > > > > > > > >> implement
> > > > > > > > >>>>>>>> the missing primitives in RocksDB or Speedb in the
> > future.
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> Understanding:
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to
> > be
> > > > > > persisted
> > > > > > > > >>>>>>>> atomically with the records?
> > > > > > > > >>>>>>>> * U2) Don't understand the default implementation of
> > > > > > > > >>>> `isolationLevel`.
> > > > > > > > >>>>>>>> The isolation level should be a property of the
> > underlying
> > > > > > store,
> > > > > > > > >> and
> > > > > > > > >>>>>>>> not be defined by the default config? Existing stores
> > > > probably
> > > > > > > > don't
> > > > > > > > >>>>>>>> guarantee READ_COMMITTED, so the default should be to
> > > > return
> > > > > > > > >>>>>>>> READ_UNCOMMITTED.
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> Nits:
> > > > > > > > >>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong`
> > > > return
> > > > > > type,
> > > > > > > > >> to
> > > > > > > > >>>>>>>> avoid the `null`?
> > > > > > > > >>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
> > > > > > `OptionalLong`
> > > > > > > > >>>>>>>> return type, to avoid the `-1`?
> > > > > > > > >>>>>>>> * N3) I don't understand why `managesOffsets` uses the
> > > > > > 'manage'
> > > > > > > > >> verb,
> > > > > > > > >>>>>>>> whereas all other methods use the "commits" verb. I'd
> > > > suggest
> > > > > > > > >>>>>>>> `commitsOffsets`.
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> Either way, it feels this KIP is very close to the
> > finish
> > > > > > line,
> > > > > > > > I'm
> > > > > > > > >>>>>>>> looking forward to seeing this in production!
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> Cheers,
> > > > > > > > >>>>>>>> Lucas
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> > > > > > colt@littlehorse.io
> > > > > > > > >
> > > > > > > > >>>>> wrote:
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>> Making IsolationLevel a query-time constraint,
> > rather
> > > > than
> > > > > > > > linking
> > > > > > > > >>>> it
> > > > > > > > >>>>>>>> to
> > > > > > > > >>>>>>>>> the processing.guarantee.
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> As I understand it, would this allow even a user of
> > EOS
> > > > to
> > > > > > > > control
> > > > > > > > >>>>>>>> whether
> > > > > > > > >>>>>>>>> reading committed or uncommitted records? If so, I am
> > > > highly
> > > > > > in
> > > > > > > > >>>> favor
> > > > > > > > >>>>> of
> > > > > > > > >>>>>>>>> this.
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> I know that I was one of the early people to point
> > out
> > > > the
> > > > > > > > current
> > > > > > > > >>>>>>>>> shortcoming that IQ reads uncommitted records, but
> > just
> > > > this
> > > > > > > > >>>> morning I
> > > > > > > > >>>>>>>>> realized a pattern we use which means that (for
> > certain
> > > > > > queries)
> > > > > > > > >> our
> > > > > > > > >>>>>>>> system
> > > > > > > > >>>>>>>>> needs to be able to read uncommitted records, which
> > is
> > > > the
> > > > > > > > current
> > > > > > > > >>>>>>>> behavior
> > > > > > > > >>>>>>>>> of Kafka Streams in EOS.***
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> If IsolationLevel being a query-time decision allows
> > for
> > > > > > this,
> > > > > > > > then
> > > > > > > > >>>>> that
> > > > > > > > >>>>>>>>> would be amazing. I would also vote that the default
> > > > behavior
> > > > > > > > >> should
> > > > > > > > >>>>> be
> > > > > > > > >>>>>>>> for
> > > > > > > > >>>>>>>>> reading uncommitted records, because it is totally
> > > > possible
> > > > > > for a
> > > > > > > > >>>>> valid
> > > > > > > > >>>>>>>>> application to depend on that behavior, and breaking
> > it
> > > > in a
> > > > > > > > minor
> > > > > > > > >>>>>>>> release
> > > > > > > > >>>>>>>>> might be a bit strong.
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>> *** (Note, for the curious reader....) Our
> > use-case/query
> > > > > > pattern
> > > > > > > > >>>> is a
> > > > > > > > >>>>>>>> bit
> > > > > > > > >>>>>>>>> complex, but reading "uncommitted" records is
> > actually
> > > > safe
> > > > > > in
> > > > > > > > our
> > > > > > > > >>>>> case
> > > > > > > > >>>>>>>>> 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 soon as the processor processes
> > the
> > > > > > result.
> > > > > > > > If
> > > > > > > > >>>>> they
> > > > > > > > >>>>>>>>> can't read uncommitted records, they may receive a
> > "201 -
> > > > > > > > Created"
> > > > > > > > >>>>>>>>> response, immediately followed by a "404 - 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 <
> > > > > > > > >>>> nick.telford@gmail.com>
> > > > > > > > >>>>>>>> wrote:
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>> Addendum:
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>> I think we would also face the same problem with the
> > > > > > approach
> > > > > > > > John
> > > > > > > > >>>>>>>> outlined
> > > > > > > > >>>>>>>>>> earlier (using the record cache as a transaction
> > buffer
> > > > and
> > > > > > > > >>>> flushing
> > > > > > > > >>>>>>>> it
> > > > > > > > >>>>>>>>>> straight to SST files). This is because the record
> > cache
> > > > > > (the
> > > > > > > > >>>>>>>> ThreadCache
> > > > > > > > >>>>>>>>>> class) is not thread-safe, so every commit would
> > > > invalidate
> > > > > > open
> > > > > > > > >> IQ
> > > > > > > > >>>>>>>>>> Iterators in the same way that RocksDB WriteBatches
> > do.
> > > > > > > > >>>>>>>>>> --
> > > > > > > > >>>>>>>>>> Nick
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > > > > > > > >>>> nick.telford@gmail.com>
> > > > > > > > >>>>>>>>>> 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.
> > > > > > > > >>>>>>>>>>> 2. Making IsolationLevel a query-time constraint,
> > > > rather
> > > > > > than
> > > > > > > > >>>>>>>> linking it
> > > > > > > > >>>>>>>>>>> to the processing.guarantee.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> There's a wrinkle that makes this a challenge:
> > > > Interactive
> > > > > > > > >> Queries
> > > > > > > > >>>>>>>> that
> > > > > > > > >>>>>>>>>>> open an Iterator, when using transactions and
> > > > > > READ_UNCOMMITTED.
> > > > > > > > >>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries
> > > > need
> > > > > > to be
> > > > > > > > >>>> able
> > > > > > > > >>>>>>>> to
> > > > > > > > >>>>>>>>>>> read records from the currently uncommitted
> > transaction
> > > > > > buffer
> > > > > > > > >>>>>>>>>>> (WriteBatch). This includes for Iterators, which
> > should
> > > > > > iterate
> > > > > > > > >>>>>>>> both the
> > > > > > > > >>>>>>>>>>> transaction buffer and underlying database (using
> > > > > > > > >>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> The issue is that when the StreamThread commits, it
> > > > writes
> > > > > > the
> > > > > > > > >>>>>>>> current
> > > > > > > > >>>>>>>>>>> WriteBatch to RocksDB *and then clears the
> > WriteBatch*.
> > > > > > > > Clearing
> > > > > > > > >>>> the
> > > > > > > > >>>>>>>>>>> WriteBatch while an Interactive Query holds an open
> > > > > > Iterator on
> > > > > > > > >> it
> > > > > > > > >>>>>>>> will
> > > > > > > > >>>>>>>>>>> invalidate the Iterator. Worse, it turns out that
> > > > Iterators
> > > > > > > > over
> > > > > > > > >> a
> > > > > > > > >>>>>>>>>>> WriteBatch become invalidated not just when the
> > > > WriteBatch
> > > > > > is
> > > > > > > > >>>>>>>> cleared,
> > > > > > > > >>>>>>>>>> but
> > > > > > > > >>>>>>>>>>> also when the Iterators' current key receives a new
> > > > write.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> Now that I'm writing this, I remember that this is
> > the
> > > > > > major
> > > > > > > > >>>> reason
> > > > > > > > >>>>>>>> that
> > > > > > > > >>>>>>>>>> I
> > > > > > > > >>>>>>>>>>> switched the original design from having a
> > query-time
> > > > > > > > >>>>>>>> IsolationLevel to
> > > > > > > > >>>>>>>>>>> having the IsolationLevel linked to the
> > > > transactionality
> > > > > > of the
> > > > > > > > >>>>>>>> stores
> > > > > > > > >>>>>>>>>>> themselves.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> It *might* be possible to resolve this, by having a
> > > > > > "chain" of
> > > > > > > > >>>>>>>>>>> WriteBatches, with the StreamThread switching to a
> > new
> > > > > > > > WriteBatch
> > > > > > > > >>>>>>>>>> whenever
> > > > > > > > >>>>>>>>>>> a new Interactive Query attempts to read from the
> > > > > > database, but
> > > > > > > > >>>> that
> > > > > > > > >>>>>>>>>> could
> > > > > > > > >>>>>>>>>>> cause some performance problems/memory pressure
> > when
> > > > > > subjected
> > > > > > > > to
> > > > > > > > >>>> a
> > > > > > > > >>>>>>>> high
> > > > > > > > >>>>>>>>>>> Interactive Query load. It would also reduce the
> > > > > > efficiency of
> > > > > > > > >>>>>>>>>> WriteBatches
> > > > > > > > >>>>>>>>>>> on-commit, as we'd have to write N WriteBatches,
> > where
> > > > N
> > > > > > is the
> > > > > > > > >>>>>>>> number of
> > > > > > > > >>>>>>>>>>> Interactive Queries since the last commit.
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> I realise this is getting into the weeds of the
> > > > > > implementation,
> > > > > > > > >>>> and
> > > > > > > > >>>>>>>> you'd
> > > > > > > > >>>>>>>>>>> rather we focus on the API for now, but I think
> > it's
> > > > > > important
> > > > > > > > to
> > > > > > > > >>>>>>>>>> consider
> > > > > > > > >>>>>>>>>>> how to implement the desired API, in case we come
> > up
> > > > with
> > > > > > an
> > > > > > > > API
> > > > > > > > >>>>>>>> that
> > > > > > > > >>>>>>>>>>> cannot be implemented efficiently, or even at all!
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> Thoughts?
> > > > > > > > >>>>>>>>>>> --
> > > > > > > > >>>>>>>>>>> Nick
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> > > > > > > > cadonna@apache.org
> > > > > > > > >>>
> > > > > > > > >>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> Hi Nick,
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> 6.
> > > > > > > > >>>>>>>>>>>> Of course, you are right! My bad!
> > > > > > > > >>>>>>>>>>>> Wiping out the state in the downgrading case is
> > fine.
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> 3a.
> > > > > > > > >>>>>>>>>>>> Focus on the public facing changes for the KIP. We
> > > > will
> > > > > > manage
> > > > > > > > >> 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.
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> Best,
> > > > > > > > >>>>>>>>>>>> Bruno
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > > > > > > > >>>>>>>>>>>>> Hi Bruno,
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> Thanks for getting back to me!
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> 2.
> > > > > > > > >>>>>>>>>>>>> The fact that implementations can always track
> > > > estimated
> > > > > > > > memory
> > > > > > > > >>>>>>>> usage
> > > > > > > > >>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as
> > an
> > > > > > option,
> > > > > > > > and
> > > > > > > > >>>>>>>> I'll
> > > > > > > > >>>>>>>>>>>> clarify
> > > > > > > > >>>>>>>>>>>>> the JavaDoc that 0 is not just for
> > non-transactional
> > > > > > stores,
> > > > > > > > >>>>>>>> which is
> > > > > > > > >>>>>>>>>>>>> currently misleading.
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> 6.
> > > > > > > > >>>>>>>>>>>>> The problem with catching the exception in the
> > > > downgrade
> > > > > > > > >> process
> > > > > > > > >>>>>>>> is
> > > > > > > > >>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>> would require new code in the Kafka version being
> > > > > > downgraded
> > > > > > > > >> to.
> > > > > > > > >>>>>>>> Since
> > > > > > > > >>>>>>>>>>>>> users could conceivably downgrade to almost *any*
> > > > older
> > > > > > > > version
> > > > > > > > >>>>>>>> of
> > > > > > > > >>>>>>>>>> Kafka
> > > > > > > > >>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
> > > > > > > > >>>>>>>>>>>>> The only way I can think of doing it would be to
> > > > provide
> > > > > > a
> > > > > > > > >>>>>>>> dedicated
> > > > > > > > >>>>>>>>>>>>> downgrade tool, that goes through every local
> > store
> > > > and
> > > > > > > > removes
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>>> offsets column families. But that seems like an
> > > > > > unnecessary
> > > > > > > > >>>>>>>> amount of
> > > > > > > > >>>>>>>>>>>> extra
> > > > > > > > >>>>>>>>>>>>> code to maintain just to handle a somewhat niche
> > > > > > situation,
> > > > > > > > >> when
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>>> alternative (automatically wipe and restore
> > stores)
> > > > > > should be
> > > > > > > > >>>>>>>>>>>> acceptable.
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've
> > > > requested.
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> 3a.
> > > > > > > > >>>>>>>>>>>>> I agree that IsolationLevel makes more sense at
> > > > > > query-time,
> > > > > > > > and
> > > > > > > > >>>> I
> > > > > > > > >>>>>>>>>>>> actually
> > > > > > > > >>>>>>>>>>>>> initially attempted to place the IsolationLevel
> > at
> > > > > > > > query-time,
> > > > > > > > >>>>>>>> but I
> > > > > > > > >>>>>>>>>> ran
> > > > > > > > >>>>>>>>>>>>> into some problems:
> > > > > > > > >>>>>>>>>>>>> - The key issue is that, under ALOS we're not
> > staging
> > > > > > writes
> > > > > > > > in
> > > > > > > > >>>>>>>>>>>>> transactions, so can't perform writes at the
> > > > > > READ_COMMITTED
> > > > > > > > >>>>>>>> isolation
> > > > > > > > >>>>>>>>>>>>> level. However, this may be addressed if we
> > decide to
> > > > > > > > *always*
> > > > > > > > >>>>>>>> use
> > > > > > > > >>>>>>>>>>>>> transactions as discussed under 3b.
> > > > > > > > >>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> > > > implementations. I
> > > > > > > > >> remember
> > > > > > > > >>>>>>>>>> having
> > > > > > > > >>>>>>>>>>>>> some difficulty understanding the IQv1 internals,
> > > > which
> > > > > > made
> > > > > > > > it
> > > > > > > > >>>>>>>>>>>> difficult
> > > > > > > > >>>>>>>>>>>>> to determine what needed to be changed. However,
> > I
> > > > > > *think*
> > > > > > > > this
> > > > > > > > >>>>>>>> can be
> > > > > > > > >>>>>>>>>>>>> addressed for both implementations by wrapping
> > the
> > > > > > > > RocksDBStore
> > > > > > > > >>>>>>>> in an
> > > > > > > > >>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides
> > read
> > > > > > methods
> > > > > > > > >>>>>>>> (get,
> > > > > > > > >>>>>>>>>>>> etc.)
> > > > > > > > >>>>>>>>>>>>> to either read directly from the database or
> > from the
> > > > > > ongoing
> > > > > > > > >>>>>>>>>>>> transaction.
> > > > > > > > >>>>>>>>>>>>> But IQv1 might still be difficult.
> > > > > > > > >>>>>>>>>>>>> - If IsolationLevel becomes a query constraint,
> > then
> > > > all
> > > > > > > > other
> > > > > > > > >>>>>>>>>>>> StateStores
> > > > > > > > >>>>>>>>>>>>> will need to respect it, including the in-memory
> > > > stores.
> > > > > > This
> > > > > > > > >>>>>>>> would
> > > > > > > > >>>>>>>>>>>> require
> > > > > > > > >>>>>>>>>>>>> us to adapt in-memory stores to stage their
> > writes so
> > > > > > they
> > > > > > > > can
> > > > > > > > >>>> be
> > > > > > > > >>>>>>>>>>>> isolated
> > > > > > > > >>>>>>>>>>>>> from READ_COMMITTTED queries. It would also
> > become an
> > > > > > > > important
> > > > > > > > >>>>>>>>>>>>> consideration for third-party stores on upgrade,
> > as
> > > > > > without
> > > > > > > > >>>>>>>> changes,
> > > > > > > > >>>>>>>>>>>> they
> > > > > > > > >>>>>>>>>>>>> would not support READ_COMMITTED queries
> > correctly.
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> Ultimately, I may need some help making the
> > necessary
> > > > > > change
> > > > > > > > to
> > > > > > > > >>>>>>>> IQv1
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>> support this, but I don't think it's
> > fundamentally
> > > > > > > > impossible,
> > > > > > > > >>>>>>>> if we
> > > > > > > > >>>>>>>>>>>> want
> > > > > > > > >>>>>>>>>>>>> to pursue this route.
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> 3b.
> > > > > > > > >>>>>>>>>>>>> The main reason I chose to keep ALOS
> > un-transactional
> > > > > > was to
> > > > > > > > >>>>>>>> minimize
> > > > > > > > >>>>>>>>>>>>> behavioural change for most users (I believe most
> > > > Streams
> > > > > > > > users
> > > > > > > > >>>>>>>> use
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>> default configuration, which is ALOS). That said,
> > > > it's
> > > > > > clear
> > > > > > > > >>>>>>>> that if
> > > > > > > > >>>>>>>>>>>> ALOS
> > > > > > > > >>>>>>>>>>>>> also used transactional stores, the only change
> > in
> > > > > > behaviour
> > > > > > > > >>>>>>>> would be
> > > > > > > > >>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>> it would become *more correct*, which could be
> > > > > > considered a
> > > > > > > > >> "bug
> > > > > > > > >>>>>>>> fix"
> > > > > > > > >>>>>>>>>> by
> > > > > > > > >>>>>>>>>>>>> users, rather than a change they need to handle.
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> I believe that performance using transactions
> > (aka.
> > > > > > RocksDB
> > > > > > > > >>>>>>>>>>>> WriteBatches)
> > > > > > > > >>>>>>>>>>>>> should actually be *better* than the un-batched
> > > > > > write-path
> > > > > > > > that
> > > > > > > > >>>>>>>> is
> > > > > > > > >>>>>>>>>>>>> currently used[1]. The only "performance"
> > > > consideration
> > > > > > will
> > > > > > > > be
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>>> increased memory usage that transactions require.
> > > > Given
> > > > > > the
> > > > > > > > >>>>>>>>>> mitigations
> > > > > > > > >>>>>>>>>>>> for
> > > > > > > > >>>>>>>>>>>>> this memory that we have in place, I would expect
> > > > that
> > > > > > this
> > > > > > > > is
> > > > > > > > >>>>>>>> not a
> > > > > > > > >>>>>>>>>>>>> problem for most users.
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> If we're happy to do so, we can make ALOS also
> > use
> > > > > > > > >> transactions.
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> Regards,
> > > > > > > > >>>>>>>>>>>>> Nick
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> Link 1:
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>
> > > > > > > > >>
> > > > > >
> > > >
> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> > > > > > > > >>>> cadonna@apache.org
> > > > > > > > >>>>>>>>>
> > > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> Hi Nick,
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> Thanks for the updates and sorry for the delay
> > on my
> > > > > > side!
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> 1.
> > > > > > > > >>>>>>>>>>>>>> Making the default implementation for flush() a
> > > > no-op
> > > > > > sounds
> > > > > > > > >>>>>>>> good to
> > > > > > > > >>>>>>>>>>>> me.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> 2.
> > > > > > > > >>>>>>>>>>>>>> I think what was bugging me here is that a
> > > > third-party
> > > > > > state
> > > > > > > > >>>>>>>> store
> > > > > > > > >>>>>>>>>>>> needs
> > > > > > > > >>>>>>>>>>>>>> to implement the state store interface. That
> > means
> > > > they
> > > > > > need
> > > > > > > > >> to
> > > > > > > > >>>>>>>>>>>>>> implement a wrapper around the actual state
> > store
> > > > as we
> > > > > > do
> > > > > > > > for
> > > > > > > > >>>>>>>>>> RocksDB
> > > > > > > > >>>>>>>>>>>>>> with RocksDBStore. So, a third-party state
> > store can
> > > > > > always
> > > > > > > > >>>>>>>> estimate
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>> uncommitted bytes, if it wants, because the
> > wrapper
> > > > can
> > > > > > > > record
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>> added
> > > > > > > > >>>>>>>>>>>>>> bytes.
> > > > > > > > >>>>>>>>>>>>>> One case I can think of where returning -1 makes
> > > > sense
> > > > > > is
> > > > > > > > when
> > > > > > > > >>>>>>>>>> Streams
> > > > > > > > >>>>>>>>>>>>>> does not need to estimate the size of the write
> > > > batch
> > > > > > and
> > > > > > > > >>>>>>>> trigger
> > > > > > > > >>>>>>>>>>>>>> extraordinary commits, because the third-party
> > state
> > > > > > store
> > > > > > > > >>>>>>>> takes care
> > > > > > > > >>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>> memory. But in that case the method could also
> > just
> > > > > > return
> > > > > > > > 0.
> > > > > > > > >>>>>>>> Even
> > > > > > > > >>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>> case would be better solved with a method that
> > > > returns
> > > > > > > > whether
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>> state
> > > > > > > > >>>>>>>>>>>>>> store manages itself the memory used for
> > uncommitted
> > > > > > bytes
> > > > > > > > or
> > > > > > > > >>>>>>>> not.
> > > > > > > > >>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return
> > > > value,
> > > > > > I was
> > > > > > > > >>>>>>>> just
> > > > > > > > >>>>>>>>>>>>>> wondering when and if it will be used.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> Regarding returning 0 for transactional state
> > stores
> > > > > > when
> > > > > > > > the
> > > > > > > > >>>>>>>> batch
> > > > > > > > >>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>> empty, I was just wondering because you
> > explicitly
> > > > > > stated
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> "or {@code 0} if this StateStore does not
> > support
> > > > > > > > >>>> transactions."
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> So it seemed to me returning 0 could only
> > happen for
> > > > > > > > >>>>>>>>>> non-transactional
> > > > > > > > >>>>>>>>>>>>>> state stores.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> 3.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> a) What do you think if we move the isolation
> > level
> > > > to
> > > > > > IQ
> > > > > > > > (v1
> > > > > > > > >>>>>>>> and
> > > > > > > > >>>>>>>>>> v2)?
> > > > > > > > >>>>>>>>>>>>>> In the end this is the only component that
> > really
> > > > needs
> > > > > > to
> > > > > > > > >>>>>>>> specify
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>> isolation level. It is similar to the Kafka
> > consumer
> > > > > > that
> > > > > > > > can
> > > > > > > > >>>>>>>> choose
> > > > > > > > >>>>>>>>>>>>>> with what isolation level to read the input
> > topic.
> > > > > > > > >>>>>>>>>>>>>> For IQv1 the isolation level should go into
> > > > > > > > >>>>>>>> StoreQueryParameters. For
> > > > > > > > >>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> b) Point a) raises the question what should
> > happen
> > > > > > during
> > > > > > > > >>>>>>>>>> at-least-once
> > > > > > > > >>>>>>>>>>>>>> processing when the state store does not use
> > > > > > transactions?
> > > > > > > > >> John
> > > > > > > > >>>>>>>> in
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>> past proposed to also use transactions on state
> > > > stores
> > > > > > for
> > > > > > > > >>>>>>>>>>>>>> at-least-once. I like that idea, because it
> > avoids
> > > > > > > > aggregating
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>> same
> > > > > > > > >>>>>>>>>>>>>> records over and over again in the case of a
> > > > failure. We
> > > > > > > > had a
> > > > > > > > >>>>>>>> case
> > > > > > > > >>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>> the past where a Streams applications in
> > > > at-least-once
> > > > > > mode
> > > > > > > > >> was
> > > > > > > > >>>>>>>>>> failing
> > > > > > > > >>>>>>>>>>>>>> continuously for some reasons I do not remember
> > > > before
> > > > > > > > >>>>>>>> committing the
> > > > > > > > >>>>>>>>>>>>>> offsets. After each failover, the app aggregated
> > > > again
> > > > > > and
> > > > > > > > >>>>>>>> again the
> > > > > > > > >>>>>>>>>>>>>> same records. Of course the aggregate increased
> > to
> > > > very
> > > > > > > > wrong
> > > > > > > > >>>>>>>> values
> > > > > > > > >>>>>>>>>>>>>> just because of the failover. With transactions
> > on
> > > > the
> > > > > > state
> > > > > > > > >>>>>>>> stores
> > > > > > > > >>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>> could have avoided this. The app would have
> > output
> > > > the
> > > > > > same
> > > > > > > > >>>>>>>> aggregate
> > > > > > > > >>>>>>>>>>>>>> multiple times (i.e., after each failover) but
> > at
> > > > least
> > > > > > the
> > > > > > > > >>>>>>>> value of
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>> aggregate would not depend on the number of
> > > > failovers.
> > > > > > > > >>>>>>>> Outputting the
> > > > > > > > >>>>>>>>>>>>>> same aggregate multiple times would be incorrect
> > > > under
> > > > > > > > >>>>>>>> exactly-once
> > > > > > > > >>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>> it is OK for at-least-once.
> > > > > > > > >>>>>>>>>>>>>> If it makes sense to add a config to turn on
> > and off
> > > > > > > > >>>>>>>> transactions on
> > > > > > > > >>>>>>>>>>>>>> state stores under at-least-once or just use
> > > > > > transactions in
> > > > > > > > >>>>>>>> any case
> > > > > > > > >>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>> a question we should also discuss in this KIP.
> > It
> > > > > > depends a
> > > > > > > > >> bit
> > > > > > > > >>>>>>>> on
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would
> > > > add a
> > > > > > > > config.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> 4.
> > > > > > > > >>>>>>>>>>>>>> Your points are all valid. I tend to say to
> > keep the
> > > > > > metrics
> > > > > > > > >>>>>>>> around
> > > > > > > > >>>>>>>>>>>>>> flush() until we remove flush() completely from
> > the
> > > > > > > > interface.
> > > > > > > > >>>>>>>> Calls
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>> flush() might still exist since existing
> > processors
> > > > > > might
> > > > > > > > >> still
> > > > > > > > >>>>>>>> call
> > > > > > > > >>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For
> > > > sure, we
> > > > > > need
> > > > > > > > >> to
> > > > > > > > >>>>>>>>>>>> document
> > > > > > > > >>>>>>>>>>>>>> how the metrics change due to the transactions
> > in
> > > > the
> > > > > > > > upgrade
> > > > > > > > >>>>>>>> notes.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> 5.
> > > > > > > > >>>>>>>>>>>>>> I see. Then you should describe how the
> > .position
> > > > files
> > > > > > are
> > > > > > > > >>>>>>>> handled
> > > > > > > > >>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>> a dedicated section of the KIP or incorporate
> > the
> > > > > > > > description
> > > > > > > > >>>>>>>> in the
> > > > > > > > >>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only
> > > > > > mentioning it
> > > > > > > > >> in
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>> "Compatibility, 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:
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>
> > > > > > > > >>
> > > > > > > >
> > > > > >
> > > >
> > https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> It is a bit brittle, but it works.
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> Best,
> > > > > > > > >>>>>>>>>>>>>> Bruno
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > > > > > > > >>>>>>>>>>>>>>> Hi Bruno,
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> Thanks for taking the time to review the KIP.
> > I'm
> > > > back
> > > > > > from
> > > > > > > > >>>>>>>> leave
> > > > > > > > >>>>>>>>>> now
> > > > > > > > >>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>> intend to move this forwards as quickly as I
> > can.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> Addressing your points:
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> 1.
> > > > > > > > >>>>>>>>>>>>>>> Because flush() is part of the StateStore API,
> > it's
> > > > > > exposed
> > > > > > > > >> to
> > > > > > > > >>>>>>>>>> custom
> > > > > > > > >>>>>>>>>>>>>>> Processors, which might be making calls to
> > flush().
> > > > > > This
> > > > > > > > was
> > > > > > > > >>>>>>>>>> actually
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> case in a few integration tests.
> > > > > > > > >>>>>>>>>>>>>>> To maintain as much compatibility as possible,
> > I'd
> > > > > > prefer
> > > > > > > > not
> > > > > > > > >>>>>>>> to
> > > > > > > > >>>>>>>>>> make
> > > > > > > > >>>>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>> an UnsupportedOperationException, as it will
> > cause
> > > > > > > > previously
> > > > > > > > >>>>>>>>>> working
> > > > > > > > >>>>>>>>>>>>>>> Processors to start throwing exceptions at
> > runtime.
> > > > > > > > >>>>>>>>>>>>>>> I agree that it doesn't make sense for it to
> > proxy
> > > > > > > > commit(),
> > > > > > > > >>>>>>>> though,
> > > > > > > > >>>>>>>>>>>> as
> > > > > > > > >>>>>>>>>>>>>>> that would cause it to violate the "StateStores
> > > > commit
> > > > > > only
> > > > > > > > >>>>>>>> when the
> > > > > > > > >>>>>>>>>>>> Task
> > > > > > > > >>>>>>>>>>>>>>> commits" rule.
> > > > > > > > >>>>>>>>>>>>>>> Instead, I think we should make this a no-op.
> > That
> > > > way,
> > > > > > > > >>>>>>>> existing
> > > > > > > > >>>>>>>>>> user
> > > > > > > > >>>>>>>>>>>>>>> Processors will continue to work as-before,
> > without
> > > > > > > > violation
> > > > > > > > >>>>>>>> of
> > > > > > > > >>>>>>>>>> store
> > > > > > > > >>>>>>>>>>>>>>> consistency that would be caused by premature
> > > > > > flush/commit
> > > > > > > > of
> > > > > > > > >>>>>>>>>>>> StateStore
> > > > > > > > >>>>>>>>>>>>>>> data to disk.
> > > > > > > > >>>>>>>>>>>>>>> What do you think?
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> 2.
> > > > > > > > >>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
> > > > > > implementation
> > > > > > > > is
> > > > > > > > >>>>>>>>>>>>>>> transactional, but is unable to estimate the
> > > > > > uncommitted
> > > > > > > > >>>> memory
> > > > > > > > >>>>>>>>>> usage,
> > > > > > > > >>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> method will return -1.
> > > > > > > > >>>>>>>>>>>>>>> The intention here is to permit third-party
> > > > > > implementations
> > > > > > > > >>>>>>>> that may
> > > > > > > > >>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>> able to estimate memory usage.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been
> > written to
> > > > the
> > > > > > > > store
> > > > > > > > >>>>>>>> yet. I
> > > > > > > > >>>>>>>>>>>>>> thought
> > > > > > > > >>>>>>>>>>>>>>> that was implied by "This method will return an
> > > > > > > > approximation
> > > > > > > > >>>>>>>> of the
> > > > > > > > >>>>>>>>>>>>>> memory
> > > > > > > > >>>>>>>>>>>>>>> would be freed by the next call to {@link
> > > > > > #commit(Map)}"
> > > > > > > > and
> > > > > > > > >>>>>>>>>> "@return
> > > > > > > > >>>>>>>>>>>> The
> > > > > > > > >>>>>>>>>>>>>>> approximate size of all records awaiting {@link
> > > > > > > > >>>> #commit(Map)}",
> > > > > > > > >>>>>>>>>>>> however,
> > > > > > > > >>>>>>>>>>>>>> I
> > > > > > > > >>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you
> > think
> > > > this
> > > > > > is
> > > > > > > > >>>>>>>> unclear?
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> 3.
> > > > > > > > >>>>>>>>>>>>>>> I realise this is probably the most contentious
> > > > point
> > > > > > in my
> > > > > > > > >>>>>>>> design,
> > > > > > > > >>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>> I'm
> > > > > > > > >>>>>>>>>>>>>>> open to changing it if I'm unable to convince
> > you
> > > > of
> > > > > > the
> > > > > > > > >>>>>>>> benefits.
> > > > > > > > >>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > > > > > > > >>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly
> > > > provided
> > > > > > > > >>>>>>>> StateStores
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>> query, and it may be important for users to
> > > > > > > > programmatically
> > > > > > > > >>>>>>>> know
> > > > > > > > >>>>>>>>>>>> which
> > > > > > > > >>>>>>>>>>>>>>> mode the StateStore is operating under. If we
> > > > simply
> > > > > > > > provide
> > > > > > > > >>>> an
> > > > > > > > >>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the
> > > > internal
> > > > > > > > streams
> > > > > > > > >>>>>>>>>>>> engine), or
> > > > > > > > >>>>>>>>>>>>>>> similar, then users will need to understand the
> > > > > > operation
> > > > > > > > and
> > > > > > > > >>>>>>>>>>>>>> consequences
> > > > > > > > >>>>>>>>>>>>>>> of each available processing mode and how it
> > > > pertains
> > > > > > to
> > > > > > > > >> their
> > > > > > > > >>>>>>>>>>>>>> StateStore.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> Interactive Query users aren't the only people
> > that
> > > > > > care
> > > > > > > > >> about
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
> > > > > > > > implementers
> > > > > > > > >>>> of
> > > > > > > > >>>>>>>>>> custom
> > > > > > > > >>>>>>>>>>>>>>> StateStores also need to understand the
> > behaviour
> > > > > > expected
> > > > > > > > of
> > > > > > > > >>>>>>>> their
> > > > > > > > >>>>>>>>>>>>>>> implementation. KIP-892 introduces some
> > assumptions
> > > > > > into
> > > > > > > > the
> > > > > > > > >>>>>>>> Streams
> > > > > > > > >>>>>>>>>>>>>> Engine
> > > > > > > > >>>>>>>>>>>>>>> about how StateStores operate under each
> > processing
> > > > > > mode,
> > > > > > > > and
> > > > > > > > >>>>>>>> it's
> > > > > > > > >>>>>>>>>>>>>>> important that custom implementations adhere to
> > > > those
> > > > > > > > >>>>>>>> assumptions in
> > > > > > > > >>>>>>>>>>>>>> order
> > > > > > > > >>>>>>>>>>>>>>> to maintain the consistency guarantees.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> IsolationLevels provide a high-level contract
> > on
> > > > the
> > > > > > > > >> behaviour
> > > > > > > > >>>>>>>> of
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>> StateStore: a user knows that under
> > READ_COMMITTED,
> > > > > > they
> > > > > > > > will
> > > > > > > > >>>>>>>> see
> > > > > > > > >>>>>>>>>>>> writes
> > > > > > > > >>>>>>>>>>>>>>> only after the Task has committed, and under
> > > > > > > > READ_UNCOMMITTED
> > > > > > > > >>>>>>>> they
> > > > > > > > >>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>> see
> > > > > > > > >>>>>>>>>>>>>>> writes immediately. No understanding of the
> > > > details of
> > > > > > each
> > > > > > > > >>>>>>>>>>>>>> processing.mode
> > > > > > > > >>>>>>>>>>>>>>> is required, either for IQ users or StateStore
> > > > > > > > implementers.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> An argument can be made that these contractual
> > > > > > guarantees
> > > > > > > > can
> > > > > > > > >>>>>>>> simply
> > > > > > > > >>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>> documented for the processing.mode (i.e. that
> > > > > > exactly-once
> > > > > > > > >> and
> > > > > > > > >>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
> > > > > > > > at-least-once
> > > > > > > > >>>>>>>> behaves
> > > > > > > > >>>>>>>>>>>> like
> > > > > > > > >>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small
> > > > issues
> > > > > > with
> > > > > > > > >>>>>>>> this I'd
> > > > > > > > >>>>>>>>>>>>>> prefer
> > > > > > > > >>>>>>>>>>>>>>> to avoid:
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>         - Where would we document these
> > contracts,
> > > > in
> > > > > > a way
> > > > > > > > >> that
> > > > > > > > >>>>>>>> is
> > > > > > > > >>>>>>>>>>>> difficult
> > > > > > > > >>>>>>>>>>>>>>>         for users/implementers to miss/ignore?
> > > > > > > > >>>>>>>>>>>>>>>         - It's not clear to users that the
> > > > processing
> > > > > > mode
> > > > > > > > is
> > > > > > > > >>>>>>>>>>>> communicating
> > > > > > > > >>>>>>>>>>>>>>>         an expectation of read isolation,
> > unless
> > > > they
> > > > > > read
> > > > > > > > the
> > > > > > > > >>>>>>>>>>>>>> documentation. Users
> > > > > > > > >>>>>>>>>>>>>>>         rarely consult documentation unless
> > they
> > > > feel
> > > > > > they
> > > > > > > > >> need
> > > > > > > > >>>>>>>> to, so
> > > > > > > > >>>>>>>>>>>> it's
> > > > > > > > >>>>>>>>>>>>>> likely
> > > > > > > > >>>>>>>>>>>>>>>         this detail would get missed by many
> > users.
> > > > > > > > >>>>>>>>>>>>>>>         - It tightly couples processing modes
> > to
> > > > read
> > > > > > > > >> isolation.
> > > > > > > > >>>>>>>> Adding
> > > > > > > > >>>>>>>>>>>> new
> > > > > > > > >>>>>>>>>>>>>>>         processing modes, or changing the read
> > > > > > isolation of
> > > > > > > > >>>>>>>> existing
> > > > > > > > >>>>>>>>>>>>>> processing
> > > > > > > > >>>>>>>>>>>>>>>         modes would be difficult/impossible.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> Ultimately, the cost of introducing
> > > > IsolationLevels is
> > > > > > > > just a
> > > > > > > > >>>>>>>> single
> > > > > > > > >>>>>>>>>>>>>>> method, since we re-use the existing
> > IsolationLevel
> > > > > > enum
> > > > > > > > from
> > > > > > > > >>>>>>>> Kafka.
> > > > > > > > >>>>>>>>>>>> This
> > > > > > > > >>>>>>>>>>>>>>> gives us a clear place to document the
> > contractual
> > > > > > > > guarantees
> > > > > > > > >>>>>>>>>> expected
> > > > > > > > >>>>>>>>>>>>>>> of/provided by StateStores, that is accessible
> > > > both by
> > > > > > the
> > > > > > > > >>>>>>>>>> StateStore
> > > > > > > > >>>>>>>>>>>>>>> itself, and by IQ users.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> (Writing this I've just realised that the
> > > > StateStore
> > > > > > and IQ
> > > > > > > > >>>>>>>> APIs
> > > > > > > > >>>>>>>>>>>> actually
> > > > > > > > >>>>>>>>>>>>>>> don't provide access to StateStoreContext that
> > IQ
> > > > users
> > > > > > > > would
> > > > > > > > >>>>>>>> have
> > > > > > > > >>>>>>>>>>>> direct
> > > > > > > > >>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> > > > > > > > >> isolationLevel()
> > > > > > > > >>>>>>>>>> itself
> > > > > > > > >>>>>>>>>>>>>> too?)
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> 4.
> > > > > > > > >>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics
> > > > in-place
> > > > > > > > >>>>>>>> either, as
> > > > > > > > >>>>>>>>>>>> it's a
> > > > > > > > >>>>>>>>>>>>>>> backwards incompatible change. My concern is
> > that,
> > > > if
> > > > > > we
> > > > > > > > >> leave
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>> existing
> > > > > > > > >>>>>>>>>>>>>>> "flush" metrics in place, they will be
> > confusing to
> > > > > > users.
> > > > > > > > >>>>>>>> Right
> > > > > > > > >>>>>>>>>> now,
> > > > > > > > >>>>>>>>>>>>>>> "flush" metrics record explicit flushes to
> > disk,
> > > > but
> > > > > > under
> > > > > > > > >>>>>>>> KIP-892,
> > > > > > > > >>>>>>>>>>>> even
> > > > > > > > >>>>>>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>> commit() will not explicitly flush data to
> > disk -
> > > > > > RocksDB
> > > > > > > > >> will
> > > > > > > > >>>>>>>>>> decide
> > > > > > > > >>>>>>>>>>>> on
> > > > > > > > >>>>>>>>>>>>>>> when to flush memtables to disk itself.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd
> > have
> > > > two
> > > > > > > > >>>> options,
> > > > > > > > >>>>>>>>>> which
> > > > > > > > >>>>>>>>>>>>>> both
> > > > > > > > >>>>>>>>>>>>>>> seem pretty bad to me:
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>         1. Have them record calls to commit(),
> > > > which
> > > > > > would
> > > > > > > > be
> > > > > > > > >>>>>>>>>>>> misleading, as
> > > > > > > > >>>>>>>>>>>>>>>         data is no longer explicitly "flushed"
> > to
> > > > disk
> > > > > > by
> > > > > > > > this
> > > > > > > > >>>>>>>> call.
> > > > > > > > >>>>>>>>>>>>>>>         2. Have them record nothing at all,
> > which
> > > > is
> > > > > > > > >> equivalent
> > > > > > > > >>>> to
> > > > > > > > >>>>>>>>>>>> removing
> > > > > > > > >>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>         metrics, except that users will see the
> > > > metric
> > > > > > > > still
> > > > > > > > >>>>>>>> exists and
> > > > > > > > >>>>>>>>>>>> so
> > > > > > > > >>>>>>>>>>>>>> assume
> > > > > > > > >>>>>>>>>>>>>>>         that the metric is correct, and that
> > > > there's a
> > > > > > > > problem
> > > > > > > > >>>>>>>> with
> > > > > > > > >>>>>>>>>> their
> > > > > > > > >>>>>>>>>>>>>> system
> > > > > > > > >>>>>>>>>>>>>>>         when there isn't.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> I agree that removing them is also a bad
> > solution,
> > > > and
> > > > > > I'd
> > > > > > > > >>>>>>>> like some
> > > > > > > > >>>>>>>>>>>>>>> guidance on the best path forward here.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> 5.
> > > > > > > > >>>>>>>>>>>>>>> Position files are updated on every write to a
> > > > > > StateStore.
> > > > > > > > >>>>>>>> Since our
> > > > > > > > >>>>>>>>>>>>>> writes
> > > > > > > > >>>>>>>>>>>>>>> are now buffered until commit(), we can't
> > update
> > > > the
> > > > > > > > Position
> > > > > > > > >>>>>>>> file
> > > > > > > > >>>>>>>>>>>> until
> > > > > > > > >>>>>>>>>>>>>>> commit() has been called, otherwise it would be
> > > > > > > > inconsistent
> > > > > > > > >>>>>>>> with
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>> data
> > > > > > > > >>>>>>>>>>>>>>> in the event of a rollback. Consequently, we
> > need
> > > > to
> > > > > > manage
> > > > > > > > >>>>>>>> these
> > > > > > > > >>>>>>>>>>>> offsets
> > > > > > > > >>>>>>>>>>>>>>> the same way we manage the checkpoint offsets,
> > and
> > > > > > ensure
> > > > > > > > >>>>>>>> they're
> > > > > > > > >>>>>>>>>> only
> > > > > > > > >>>>>>>>>>>>>>> written on commit().
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> 6.
> > > > > > > > >>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what
> > > > tests to
> > > > > > > > >> write.
> > > > > > > > >>>>>>>> How
> > > > > > > > >>>>>>>>>>>>>> explicit
> > > > > > > > >>>>>>>>>>>>>>> do we need to be here in the KIP?
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed
> > to be
> > > > > > > > seamless,
> > > > > > > > >>>>>>>> and we
> > > > > > > > >>>>>>>>>>>>>> should
> > > > > > > > >>>>>>>>>>>>>>> definitely add some tests around that.
> > Downgrade,
> > > > it
> > > > > > > > >>>>>>>> transpires,
> > > > > > > > >>>>>>>>>> isn't
> > > > > > > > >>>>>>>>>>>>>>> currently possible, as the extra column family
> > for
> > > > > > offset
> > > > > > > > >>>>>>>> storage is
> > > > > > > > >>>>>>>>>>>>>>> incompatible with the pre-KIP-892
> > implementation:
> > > > when
> > > > > > you
> > > > > > > > >>>>>>>> open a
> > > > > > > > >>>>>>>>>>>> RocksDB
> > > > > > > > >>>>>>>>>>>>>>> database, you must open all available column
> > > > families
> > > > > > or
> > > > > > > > >>>>>>>> receive an
> > > > > > > > >>>>>>>>>>>>>> error.
> > > > > > > > >>>>>>>>>>>>>>> What currently happens on downgrade is that it
> > > > > > attempts to
> > > > > > > > >>>>>>>> open the
> > > > > > > > >>>>>>>>>>>>>> store,
> > > > > > > > >>>>>>>>>>>>>>> throws an error about the offsets column
> > family not
> > > > > > being
> > > > > > > > >>>>>>>> opened,
> > > > > > > > >>>>>>>>>>>> which
> > > > > > > > >>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given
> > that
> > > > > > > > >> downgrades
> > > > > > > > >>>>>>>>>> should
> > > > > > > > >>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>> uncommon, I think this is acceptable
> > behaviour, as
> > > > the
> > > > > > > > >>>>>>>> end-state is
> > > > > > > > >>>>>>>>>>>>>>> consistent, even if it results in an
> > undesirable
> > > > state
> > > > > > > > >>>> restore.
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> Should I document the upgrade/downgrade
> > behaviour
> > > > > > > > explicitly
> > > > > > > > >>>>>>>> in the
> > > > > > > > >>>>>>>>>>>> KIP?
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> --
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> Regards,
> > > > > > > > >>>>>>>>>>>>>>> Nick
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > > > > > > > >>>>>>>> cadonna@apache.org>
> > > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> Hi Nick!
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> Thanks for the updates!
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> 1.
> > > > > > > > >>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> > > > > > > > >>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > > > > > > > >>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore
> > > > after
> > > > > > this
> > > > > > > > KIP
> > > > > > > > >>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>> released, I would rather throw an unsupported
> > > > > > operation
> > > > > > > > >>>>>>>> exception
> > > > > > > > >>>>>>>>>> by
> > > > > > > > >>>>>>>>>>>>>>>> default.
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> 2.
> > > > > > > > >>>>>>>>>>>>>>>> When would a state store return -1 from
> > > > > > > > >>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
> > while
> > > > > > being
> > > > > > > > >>>>>>>>>>>> transactional?
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> Wouldn't
> > > > StateStore#approximateNumUncommittedBytes()
> > > > > > also
> > > > > > > > >>>>>>>> return 0
> > > > > > > > >>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>> the state store is transactional but nothing
> > has
> > > > been
> > > > > > > > >> written
> > > > > > > > >>>>>>>> to
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>> state store yet?
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> 3.
> > > > > > > > >>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this
> > KIP
> > > > really
> > > > > > > > need
> > > > > > > > >>>> to
> > > > > > > > >>>>>>>>>>>> introduce
> > > > > > > > >>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> > > > StateStoreContext
> > > > > > has
> > > > > > > > >>>>>>>> already
> > > > > > > > >>>>>>>>>>>>>>>> appConfigs() which basically exposes the same
> > > > > > information,
> > > > > > > > >>>>>>>> i.e., if
> > > > > > > > >>>>>>>>>>>> EOS
> > > > > > > > >>>>>>>>>>>>>>>> is enabled or not.
> > > > > > > > >>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore
> > > > interface
> > > > > > as
> > > > > > > > >>>>>>>> loosely
> > > > > > > > >>>>>>>>>>>> coupled
> > > > > > > > >>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > > > > > implementers
> > > > > > > > >>>> more
> > > > > > > > >>>>>>>>>>>> freedom,
> > > > > > > > >>>>>>>>>>>>>>>> and reduce the amount of internal knowledge
> > > > required."
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> While I understand the intent, I doubt that it
> > > > > > decreases
> > > > > > > > the
> > > > > > > > >>>>>>>>>>>> coupling of
> > > > > > > > >>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> > > > > > > > >> READ_COMMITTED
> > > > > > > > >>>>>>>> only
> > > > > > > > >>>>>>>>>>>>>>>> applies to IQ but not to reads by processors.
> > > > Thus,
> > > > > > > > >>>>>>>> implementers
> > > > > > > > >>>>>>>>>>>> need to
> > > > > > > > >>>>>>>>>>>>>>>> understand how Streams accesses the state
> > stores.
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> I would like to hear what others think about
> > this.
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> 4.
> > > > > > > > >>>>>>>>>>>>>>>> Great exposing new metrics for transactional
> > state
> > > > > > stores!
> > > > > > > > >>>>>>>>>> However, I
> > > > > > > > >>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate
> > (in
> > > > the
> > > > > > > > docs)
> > > > > > > > >>>>>>>> the 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 also need to adapt/add
> > integration
> > > > > > tests
> > > > > > > > >>>>>>>> besides
> > > > > > > > >>>>>>>>>> unit
> > > > > > > > >>>>>>>>>>>>>>>> tests. Additionally, we probably need
> > integration
> > > > or
> > > > > > > > system
> > > > > > > > >>>>>>>> tests
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>> verify that upgrades and downgrades between
> > > > > > transactional
> > > > > > > > >> and
> > > > > > > > >>>>>>>>>>>>>>>> non-transactional state stores work as
> > expected.
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> Best,
> > > > > > > > >>>>>>>>>>>>>>>> Bruno
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > > > > > > > >>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in
> > the
> > > > KIP,
> > > > > > > > under
> > > > > > > > >>>>>>>>>>>> "Rejected
> > > > > > > > >>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea
> > worth
> > > > > > pursuing,
> > > > > > > > >>>>>>>> but I
> > > > > > > > >>>>>>>>>>>> believe
> > > > > > > > >>>>>>>>>>>>>>>>> that it's out of the scope of this KIP,
> > because
> > > > it
> > > > > > > > solves a
> > > > > > > > >>>>>>>>>>>> different
> > > > > > > > >>>>>>>>>>>>>> set
> > > > > > > > >>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of
> > this
> > > > one
> > > > > > has
> > > > > > > > >>>>>>>> already
> > > > > > > > >>>>>>>>>> grown
> > > > > > > > >>>>>>>>>>>>>>>> quite
> > > > > > > > >>>>>>>>>>>>>>>>> large!
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > > > > > > > >>>>>>>>>> nick.telford@gmail.com>
> > > > > > > > >>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> Hi everyone,
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> I've updated the KIP (
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>
> > > > > > > > >>
> > > > > > > >
> > > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > > > > > >>>>>>>>>>>>>>>> )
> > > > > > > > >>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing
> > back
> > > > > > "Atomic
> > > > > > > > >>>>>>>>>>>> Checkpointing"
> > > > > > > > >>>>>>>>>>>>>>>> (for
> > > > > > > > >>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think
> > the one
> > > > > > thing
> > > > > > > > >>>>>>>> missing is
> > > > > > > > >>>>>>>>>>>> some
> > > > > > > > >>>>>>>>>>>>>>>>>> changes to metrics (notably the store
> > "flush"
> > > > > > metrics
> > > > > > > > will
> > > > > > > > >>>>>>>> need
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>> renamed to "commit").
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> The reason I brought back Atomic
> > Checkpointing
> > > > was
> > > > > > to
> > > > > > > > >>>>>>>> decouple
> > > > > > > > >>>>>>>>>>>> store
> > > > > > > > >>>>>>>>>>>>>>>> flush
> > > > > > > > >>>>>>>>>>>>>>>>>> from store commit. This is important,
> > because
> > > > with
> > > > > > > > >>>>>>>> Transactional
> > > > > > > > >>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on
> > > > *every*
> > > > > > Task
> > > > > > > > >>>>>>>> commit,
> > > > > > > > >>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>> just when the StateStore is closing,
> > otherwise
> > > > our
> > > > > > > > >>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>> buffer
> > > > > > > > >>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>> never be written and persisted, instead
> > growing
> > > > > > > > unbounded!
> > > > > > > > >>>> I
> > > > > > > > >>>>>>>>>>>>>>>> experimented
> > > > > > > > >>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a
> > store
> > > > > > flush
> > > > > > > > >>>>>>>> whenever
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its
> > > > > > configured
> > > > > > > > >>>>>>>> size, but
> > > > > > > > >>>>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>> was
> > > > > > > > >>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer
> > > > from
> > > > > > being
> > > > > > > > >>>>>>>>>> configured
> > > > > > > > >>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>> unbounded, and it still would have required
> > > > explicit
> > > > > > > > >>>>>>>> flushes of
> > > > > > > > >>>>>>>>>>>>>> RocksDB,
> > > > > > > > >>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory
> > > > > > utilization.
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the
> > "right"
> > > > way
> > > > > > to
> > > > > > > > >>>>>>>> resolve
> > > > > > > > >>>>>>>>>> this
> > > > > > > > >>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog
> > offsets
> > > > that
> > > > > > > > >>>>>>>> correspond
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>> most
> > > > > > > > >>>>>>>>>>>>>>>>>> recently written records are always
> > atomically
> > > > > > written
> > > > > > > > to
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>> StateStore
> > > > > > > > >>>>>>>>>>>>>>>>>> (by writing them to the same transaction
> > > > buffer),
> > > > > > we can
> > > > > > > > >>>>>>>> avoid
> > > > > > > > >>>>>>>>>>>>>> forcibly
> > > > > > > > >>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk,
> > letting
> > > > > > RocksDB
> > > > > > > > >>>>>>>> flush
> > > > > > > > >>>>>>>>>> them
> > > > > > > > >>>>>>>>>>>>>> only
> > > > > > > > >>>>>>>>>>>>>>>>>> when necessary, without losing any of our
> > > > > > consistency
> > > > > > > > >>>>>>>> guarantees.
> > > > > > > > >>>>>>>>>>>> See
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>> updated KIP for more info.
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> I have fully implemented these changes,
> > > > although I'm
> > > > > > > > still
> > > > > > > > >>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>> entirely
> > > > > > > > >>>>>>>>>>>>>>>>>> happy with the implementation for segmented
> > > > > > StateStores,
> > > > > > > > >> so
> > > > > > > > >>>>>>>> I
> > > > > > > > >>>>>>>>>> plan
> > > > > > > > >>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests
> > pass. If
> > > > > > you'd
> > > > > > > > like
> > > > > > > > >>>>>>>> to try
> > > > > > > > >>>>>>>>>>>> out
> > > > > > > > >>>>>>>>>>>>>> or
> > > > > > > > >>>>>>>>>>>>>>>>>> review this highly experimental and
> > incomplete
> > > > > > branch,
> > > > > > > > >> it's
> > > > > > > > >>>>>>>>>>>> available
> > > > > > > > >>>>>>>>>>>>>>>> here:
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > > > > > > > .
> > > > > > > > >>>>>>>> Note:
> > > > > > > > >>>>>>>>>>>> it's
> > > > > > > > >>>>>>>>>>>>>>>> built
> > > > > > > > >>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable
> > base
> > > > to
> > > > > > build
> > > > > > > > >>>>>>>> and test
> > > > > > > > >>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>> on,
> > > > > > > > >>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples
> > comparisons
> > > > in a
> > > > > > > > live
> > > > > > > > >>>>>>>>>>>>>> environment. I
> > > > > > > > >>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's
> > nearer
> > > > > > > > >> completion
> > > > > > > > >>>>>>>> and
> > > > > > > > >>>>>>>>>> has
> > > > > > > > >>>>>>>>>>>>>> been
> > > > > > > > >>>>>>>>>>>>>>>>>> proven on our main application.
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing
> > and
> > > > > > testing:
> > > > > > > > >>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window)
> > > > stores
> > > > > > > > >>>>>>>>>>>>>>>>>> - Global stores
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> As I do not currently use either of these,
> > so my
> > > > > > primary
> > > > > > > > >>>>>>>> test
> > > > > > > > >>>>>>>>>>>>>>>> environment
> > > > > > > > >>>>>>>>>>>>>>>>>> doesn't test these areas.
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next
> > week
> > > > for
> > > > > > a few
> > > > > > > > >>>>>>>> weeks,
> > > > > > > > >>>>>>>>>> so
> > > > > > > > >>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>> not have time to move this forward until
> > late
> > > > > > August.
> > > > > > > > That
> > > > > > > > >>>>>>>> said,
> > > > > > > > >>>>>>>>>>>> your
> > > > > > > > >>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just
> > > > won't be
> > > > > > > > able
> > > > > > > > >>>> to
> > > > > > > > >>>>>>>>>>>> respond
> > > > > > > > >>>>>>>>>>>>>> as
> > > > > > > > >>>>>>>>>>>>>>>>>> quickly as usual.
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> Regards,
> > > > > > > > >>>>>>>>>>>>>>>>>> Nick
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > > > > > > > >>>>>>>>>> nick.telford@gmail.com>
> > > > > > > > >>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>> Hi Bruno
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact
> > on IQ
> > > > is
> > > > > > not
> > > > > > > > >>>>>>>> something
> > > > > > > > >>>>>>>>>> I
> > > > > > > > >>>>>>>>>>>> had
> > > > > > > > >>>>>>>>>>>>>>>>>>> considered.
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>> What about atomically updating the state
> > store
> > > > > > from the
> > > > > > > > >>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> > the
> > > > > > > > checkpoint
> > > > > > > > >>>>>>>> (thus,
> > > > > > > > >>>>>>>>>>>>>>>> flushing
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> > data
> > > > > > and/or
> > > > > > > > >>>>>>>> number of
> > > > > > > > >>>>>>>>>>>>>> commit
> > > > > > > > >>>>>>>>>>>>>>>>>>>> intervals?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you
> > suggesting
> > > > > > that we
> > > > > > > > >>>>>>>> add an
> > > > > > > > >>>>>>>>>>>>>>>> additional
> > > > > > > > >>>>>>>>>>>>>>>>>>> config for the max number of commit
> > intervals
> > > > > > between
> > > > > > > > >>>>>>>>>> checkpoints?
> > > > > > > > >>>>>>>>>>>>>> That
> > > > > > > > >>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the
> > > > > > transaction
> > > > > > > > >>>>>>>> buffers
> > > > > > > > >>>>>>>>>> are
> > > > > > > > >>>>>>>>>>>>>>>> nearly
> > > > > > > > >>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of
> > commit
> > > > > > > > intervals
> > > > > > > > >>>>>>>> have
> > > > > > > > >>>>>>>>>>>>>> elapsed,
> > > > > > > > >>>>>>>>>>>>>>>>>>> whichever comes first?
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although
> > this
> > > > > > > > re-ignites
> > > > > > > > >>>>>>>> an
> > > > > > > > >>>>>>>>>>>> earlier
> > > > > > > > >>>>>>>>>>>>>>>>>>> debate about whether a config should be
> > > > measured in
> > > > > > > > >>>>>>>> "number of
> > > > > > > > >>>>>>>>>>>> commit
> > > > > > > > >>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute
> > time.
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the
> > reason
> > > > I
> > > > > > was
> > > > > > > > >>>>>>>> pursuing
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>> Atomic
> > > > > > > > >>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable
> > flush
> > > > from
> > > > > > > > >>>>>>>> checkpointing,
> > > > > > > > >>>>>>>>>>>>>> which
> > > > > > > > >>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every
> > commit
> > > > > > without
> > > > > > > > any
> > > > > > > > >>>>>>>>>>>> performance
> > > > > > > > >>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely
> > the
> > > > > > "best"
> > > > > > > > >>>>>>>> solution,
> > > > > > > > >>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>> I'm not
> > > > > > > > >>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back
> > into
> > > > this
> > > > > > KIP.
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
> > > > > > transactional
> > > > > > > > >>>> logic
> > > > > > > > >>>>>>>>>>>> directly
> > > > > > > > >>>>>>>>>>>>>>>> into
> > > > > > > > >>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with
> > the
> > > > > > > > >>>>>>>>>>>>>> StateStore#newTransaction
> > > > > > > > >>>>>>>>>>>>>>>>>>> method, and reduces the number of new
> > classes
> > > > > > > > introduced,
> > > > > > > > >>>>>>>>>>>>>> significantly
> > > > > > > > >>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and
> > the
> > > > > > > > complexity
> > > > > > > > >>>> is
> > > > > > > > >>>>>>>>>>>>>> drastically
> > > > > > > > >>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic
> > > > Checkpoints
> > > > > > > > into
> > > > > > > > >>>>>>>> this
> > > > > > > > >>>>>>>>>> KIP.
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>> Regards,
> > > > > > > > >>>>>>>>>>>>>>>>>>> Nick
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna
> > <
> > > > > > > > >>>>>>>> cadonna@apache.org>
> > > > > > > > >>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> As far as I understand, you want to
> > atomically
> > > > > > update
> > > > > > > > >> the
> > > > > > > > >>>>>>>> state
> > > > > > > > >>>>>>>>>>>>>> store
> > > > > > > > >>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the
> > > > memtable
> > > > > > of a
> > > > > > > > >>>> state
> > > > > > > > >>>>>>>>>> store
> > > > > > > > >>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit
> > time
> > > > > > elapsed
> > > > > > > > >>>> but
> > > > > > > > >>>>>>>>>> after
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that
> > would
> > > > lead
> > > > > > to
> > > > > > > > >>>>>>>> exceeding
> > > > > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > before the
> > > > > > > > next
> > > > > > > > >>>>>>>> commit
> > > > > > > > >>>>>>>>>>>>>>>> interval
> > > > > > > > >>>>>>>>>>>>>>>>>>>> ends.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would
> > commit
> > > > > > every
> > > > > > > > >>>>>>>> commit
> > > > > > > > >>>>>>>>>>>> interval
> > > > > > > > >>>>>>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the state store will only be atomically
> > > > updated
> > > > > > > > roughly
> > > > > > > > >>>>>>>> every
> > > > > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of
> > > > data.
> > > > > > Also
> > > > > > > > IQ
> > > > > > > > >>>>>>>> would
> > > > > > > > >>>>>>>>>>>> then
> > > > > > > > >>>>>>>>>>>>>>>> only
> > > > > > > > >>>>>>>>>>>>>>>>>>>> see new data roughly every
> > > > > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> After a failure the state store needs to
> > > > restore
> > > > > > up to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Is this correct?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> What about atomically updating the state
> > store
> > > > > > from
> > > > > > > > the
> > > > > > > > >>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> > the
> > > > > > > > checkpoint
> > > > > > > > >>>>>>>> (thus,
> > > > > > > > >>>>>>>>>>>>>>>> flushing
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> > data
> > > > > > and/or
> > > > > > > > >>>>>>>> number of
> > > > > > > > >>>>>>>>>>>>>> commit
> > > > > > > > >>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have
> > the
> > > > same
> > > > > > delay
> > > > > > > > >>>> for
> > > > > > > > >>>>>>>>>>>> records
> > > > > > > > >>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because
> > both
> > > > > > would
> > > > > > > > >>>>>>>> appear
> > > > > > > > >>>>>>>>>> when
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However,
> > > > after a
> > > > > > > > failure
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>> state
> > > > > > > > >>>>>>>>>>>>>>>> store
> > > > > > > > >>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > > > > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > > > > > >>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>> it might restore data that is already in
> > the
> > > > state
> > > > > > > > store
> > > > > > > > >>>>>>>>>> because
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable
> > offset
> > > > > > (i.e.
> > > > > > > > the
> > > > > > > > >>>>>>>> last
> > > > > > > > >>>>>>>>>>>>>> committed
> > > > > > > > >>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring
> > > > data
> > > > > > that
> > > > > > > > is
> > > > > > > > >>>>>>>> already
> > > > > > > > >>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should
> > not
> > > > > > violated.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> This solution needs at least one new
> > config to
> > > > > > specify
> > > > > > > > >>>>>>>> when a
> > > > > > > > >>>>>>>>>>>>>>>> checkpoint
> > > > > > > > >>>>>>>>>>>>>>>>>>>> should be written.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail
> > > > that
> > > > > > does
> > > > > > > > not
> > > > > > > > >>>>>>>> change
> > > > > > > > >>>>>>>>>>>>>>>> anything
> > > > > > > > >>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit
> > > > interval
> > > > > > is 30
> > > > > > > > >>>>>>>> seconds,
> > > > > > > > >>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>> five
> > > > > > > > >>>>>>>>>>>>>>>>>>>> seconds.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Best,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Bruno
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch
> > on
> > > > our
> > > > > > > > staging
> > > > > > > > >>>>>>>>>>>>>> environment,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
> > > > non-trivial
> > > > > > > > >>>>>>>> application.
> > > > > > > > >>>>>>>>>> I'm
> > > > > > > > >>>>>>>>>>>>>>>>>>>> already
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate
> > that
> > > > we
> > > > > > saw
> > > > > > > > the
> > > > > > > > >>>>>>>> last
> > > > > > > > >>>>>>>>>>>> time
> > > > > > > > >>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP,
> > but
> > > > this
> > > > > > > > time,
> > > > > > > > >> I
> > > > > > > > >>>>>>>>>> think I
> > > > > > > > >>>>>>>>>>>>>> know
> > > > > > > > >>>>>>>>>>>>>>>>>>>> why.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit,
> > which is
> > > > > > called
> > > > > > > > at
> > > > > > > > >>>>>>>> the end
> > > > > > > > >>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Task
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> commit process, has the following
> > behaviour:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           - Under ALOS: checkpoint the
> > state
> > > > > > stores.
> > > > > > > > >> This
> > > > > > > > >>>>>>>>>> includes
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           flushing memtables in RocksDB.
> > > > This is
> > > > > > > > >>>> acceptable
> > > > > > > > >>>>>>>>>>>> because the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> default
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           commit.interval.ms is 5
> > seconds,
> > > > so
> > > > > > > > forcibly
> > > > > > > > >>>>>>>> flushing
> > > > > > > > >>>>>>>>>>>>>> memtables
> > > > > > > > >>>>>>>>>>>>>>>>>>>> every 5
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           seconds is acceptable for most
> > > > > > > > applications.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           - Under EOS: checkpointing is
> > not
> > > > done,
> > > > > > > > >> *unless*
> > > > > > > > >>>>>>>> it's
> > > > > > > > >>>>>>>>>>>> being
> > > > > > > > >>>>>>>>>>>>>>>>>>>> forced, due
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           to e.g. the Task closing or
> > being
> > > > > > revoked.
> > > > > > > > >> This
> > > > > > > > >>>>>>>> means
> > > > > > > > >>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>> under
> > > > > > > > >>>>>>>>>>>>>>>>>>>> normal
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           processing conditions, the
> > state
> > > > stores
> > > > > > > > will
> > > > > > > > >> not
> > > > > > > > >>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>> checkpointed,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> and will
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           not have memtables flushed at
> > all ,
> > > > > > unless
> > > > > > > > >>>> RocksDB
> > > > > > > > >>>>>>>>>>>> decides to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> flush them on
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           its own. Checkpointing stores
> > and
> > > > > > > > >> force-flushing
> > > > > > > > >>>>>>>> their
> > > > > > > > >>>>>>>>>>>>>> memtables
> > > > > > > > >>>>>>>>>>>>>>>>>>>> is only
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           done when a Task is being
> > closed.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint
> > > > stores on
> > > > > > at
> > > > > > > > >>>> least
> > > > > > > > >>>>>>>>>> *some*
> > > > > > > > >>>>>>>>>>>>>>>> normal
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the
> > RocksDB
> > > > > > > > transaction
> > > > > > > > >>>>>>>>>> buffers
> > > > > > > > >>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets
> > are
> > > > > > synced to
> > > > > > > > >>>>>>>> disk to
> > > > > > > > >>>>>>>>>>>>>> prevent
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
> > > > Consequently,
> > > > > > my
> > > > > > > > >>>>>>>> current
> > > > > > > > >>>>>>>>>>>>>>>>>>>> implementation
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task
> > commit,
> > > > > > which
> > > > > > > > is
> > > > > > > > >>>>>>>> far too
> > > > > > > > >>>>>>>>>>>>>>>>>>>> frequent.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000
> > records,
> > > > > > which
> > > > > > > > is
> > > > > > > > >> a
> > > > > > > > >>>>>>>>>> change
> > > > > > > > >>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>> flush
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
> > performance
> > > > > > problems
> > > > > > > > for
> > > > > > > > >>>>>>>> some
> > > > > > > > >>>>>>>>>>>>>>>>>>>> applications.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and
> > I'm
> > > > > > > > currently
> > > > > > > > >>>>>>>> leaning
> > > > > > > > >>>>>>>>>>>>>>>> towards
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> using the
> > > > statestore.transaction.buffer.max.bytes
> > > > > > > > >>>>>>>>>> configuration
> > > > > > > > >>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to
> > > > exceed it.
> > > > > > > > This
> > > > > > > > >>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>> complement the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
> > functionality
> > > > that
> > > > > > this
> > > > > > > > >>>>>>>>>>>> configuration
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           - Currently, we use
> > > > > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > > > > > >>>>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> force an
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           early Task commit if processing
> > > > more
> > > > > > > > records
> > > > > > > > >>>> would
> > > > > > > > >>>>>>>>>> cause
> > > > > > > > >>>>>>>>>>>> our
> > > > > > > > >>>>>>>>>>>>>>>> state
> > > > > > > > >>>>>>>>>>>>>>>>>>>> store
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           transactions to exceed the
> > memory
> > > > > > assigned
> > > > > > > > to
> > > > > > > > >>>>>>>> them.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           - New functionality: when a
> > Task
> > > > *does*
> > > > > > > > >> commit,
> > > > > > > > >>>>>>>> we will
> > > > > > > > >>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>>>> checkpoint
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           the stores (and hence flush the
> > > > > > transaction
> > > > > > > > >>>>>>>> buffers)
> > > > > > > > >>>>>>>>>>>> unless
> > > > > > > > >>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>> expect to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           cross the
> > > > > > > > >>>> statestore.transaction.buffer.max.bytes
> > > > > > > > >>>>>>>>>>>> threshold
> > > > > > > > >>>>>>>>>>>>>>>> before
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the next
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>           commit
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> Nick
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick
> > Telford <
> > > > > > > > >>>>>>>>>>>> nick.telford@gmail.com
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I
> > meant in
> > > > > > terms of
> > > > > > > > >>>>>>>>>>>> understanding
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> performance profile under various
> > > > > > circumstances. The
> > > > > > > > >>>>>>>> more
> > > > > > > > >>>>>>>>>>>> complex
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would
> > be for
> > > > > > users
> > > > > > > > to
> > > > > > > > >>>>>>>>>>>> understand
> > > > > > > > >>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> performance they see. For example,
> > spilling
> > > > > > records
> > > > > > > > to
> > > > > > > > >>>>>>>> disk
> > > > > > > > >>>>>>>>>>>> when
> > > > > > > > >>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold
> > > > would, I
> > > > > > > > >> expect,
> > > > > > > > >>>>>>>>>> reduce
> > > > > > > > >>>>>>>>>>>>>> write
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
> > > > throughput
> > > > > > could
> > > > > > > > >> be
> > > > > > > > >>>>>>>>>>>>>> unexpected,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> > > > diagnose/understand for
> > > > > > > > >> users.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early
> > commit"
> > > > > > concept is
> > > > > > > > >>>>>>>>>> relatively
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document,
> > and
> > > > > > > > >>>> conceptually
> > > > > > > > >>>>>>>>>> fairly
> > > > > > > > >>>>>>>>>>>>>>>>>>>> obvious to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to
> > > > make it
> > > > > > > > >> easier
> > > > > > > > >>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>> understand
> > > > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> it happens though.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an
> > > > indirect
> > > > > > way
> > > > > > > > of
> > > > > > > > >>>>>>>>>> telling
> > > > > > > > >>>>>>>>>>>>>>>>>>>> StateStores
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
> > > > > > READ_COMMITTED
> > > > > > > > >>>>>>>>>>>> essentially
> > > > > > > > >>>>>>>>>>>>>>>>>>>> requires
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that
> > two
> > > > > > threads
> > > > > > > > >>>>>>>> calling
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes
> > > > from
> > > > > > the
> > > > > > > > >> other
> > > > > > > > >>>>>>>>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>>>>>>>> until
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> > > > > > READ_UNCOMMITTED, all
> > > > > > > > >>>>>>>> bets are
> > > > > > > > >>>>>>>>>>>> off,
> > > > > > > > >>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe
> > written
> > > > > > records
> > > > > > > > at
> > > > > > > > >>>>>>>> any
> > > > > > > > >>>>>>>>>>>> time,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> which is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That
> > said,
> > > > > > > > StateStores
> > > > > > > > >>>>>>>> are
> > > > > > > > >>>>>>>>>> free
> > > > > > > > >>>>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>> implement
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> these guarantees however they can,
> > which is
> > > > a
> > > > > > bit
> > > > > > > > more
> > > > > > > > >>>>>>>>>> relaxed
> > > > > > > > >>>>>>>>>>>>>> than
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions".
> > For
> > > > > > example,
> > > > > > > > >>>> with
> > > > > > > > >>>>>>>>>>>> RocksDB
> > > > > > > > >>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>> would
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED ==
> > > > WBWI-based
> > > > > > > > >>>>>>>>>> "transactions",
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the
> > > > > > database.
> > > > > > > > But
> > > > > > > > >>>>>>>> with
> > > > > > > > >>>>>>>>>>>> other
> > > > > > > > >>>>>>>>>>>>>>>>>>>> storage
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to
> > *always*
> > > > use
> > > > > > > > >>>>>>>> transactions,
> > > > > > > > >>>>>>>>>>>> even
> > > > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage
> > engines
> > > > > > that
> > > > > > > > >> don't
> > > > > > > > >>>>>>>>>> provide
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation
> > guarantees
> > > > can
> > > > > > be
> > > > > > > > met
> > > > > > > > >>>>>>>> using a
> > > > > > > > >>>>>>>>>>>>>>>>>>>> different
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> technique.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the
> > StateStore
> > > > > > interface
> > > > > > > > as
> > > > > > > > >>>>>>>>>> loosely
> > > > > > > > >>>>>>>>>>>>>>>> coupled
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to
> > give
> > > > > > > > >>>>>>>> implementers
> > > > > > > > >>>>>>>>>> more
> > > > > > > > >>>>>>>>>>>>>>>>>>>> freedom, and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge
> > > > > > required.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> > > > "IsolationLevel"
> > > > > > might
> > > > > > > > >> not
> > > > > > > > >>>>>>>> be
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>> right
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it
> > much
> > > > more
> > > > > > > > >>>>>>>> explicit if
> > > > > > > > >>>>>>>>>>>>>>>>>>>> required, e.g.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> 7-8.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> I can make these changes either later
> > today
> > > > or
> > > > > > > > >>>> tomorrow.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Small update:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and
> > fixed a
> > > > > > bunch of
> > > > > > > > >>>>>>>> issues
> > > > > > > > >>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>> needed
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests
> > pass,
> > > > > > which is
> > > > > > > > >>>>>>>>>> promising,
> > > > > > > > >>>>>>>>>>>> but
> > > > > > > > >>>>>>>>>>>>>>>> it
> > > > > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> need to undergo some performance
> > testing. I
> > > > > > haven't
> > > > > > > > >>>>>>>> (yet)
> > > > > > > > >>>>>>>>>>>> worked
> > > > > > > > >>>>>>>>>>>>>> on
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff,
> > but I
> > > > > > would
> > > > > > > > >>>>>>>> expect
> > > > > > > > >>>>>>>>>> that,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
> > > > difference. The
> > > > > > > > >> branch
> > > > > > > > >>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>> available
> > > > > > > > >>>>>>>>>>>>>>>>>>>> at
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > https://github.com/nicktelford/kafka/tree/KIP-892-c
> > > > > > > > >> if
> > > > > > > > >>>>>>>>>> anyone
> > > > > > > > >>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno
> > Cadonna
> > > > <
> > > > > > > > >>>>>>>>>>>> cadonna@apache.org>
> > > > > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was
> > actually
> > > > also
> > > > > > my
> > > > > > > > >>>>>>>> point. I
> > > > > > > > >>>>>>>>>>>>>>>> understood
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion
> > path
> > > > as
> > > > > > a way
> > > > > > > > >> to
> > > > > > > > >>>>>>>> avoid
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> early
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the
> > > > intent.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually
> > it is
> > > > > > public
> > > > > > > > >>>>>>>> API. My
> > > > > > > > >>>>>>>>>>>>>> question
> > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal
> > > > > > processing.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the
> > size
> > > > of the
> > > > > > > > >>>>>>>> transaction
> > > > > > > > >>>>>>>>>>>>>> buffer
> > > > > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit
> > affects
> > > > the
> > > > > > whole
> > > > > > > > >>>>>>>>>> processing
> > > > > > > > >>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a
> > state
> > > > store
> > > > > > is
> > > > > > > > not
> > > > > > > > >>>>>>>>>>>> confined to
> > > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and
> > > > > > changes the
> > > > > > > > >>>>>>>> behavior
> > > > > > > > >>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>> other
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you
> > that
> > > > it
> > > > > > is a
> > > > > > > > >>>>>>>> decent
> > > > > > > > >>>>>>>>>>>>>>>>>>>> compromise. I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides
> > and
> > > > list
> > > > > > the
> > > > > > > > >>>>>>>> options to
> > > > > > > > >>>>>>>>>>>>>>>> overcome
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all
> > > > options
> > > > > > seem
> > > > > > > > >>>> quite
> > > > > > > > >>>>>>>>>> heavy
> > > > > > > > >>>>>>>>>>>>>>>>>>>> compared
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand
> > what you
> > > > > > mean
> > > > > > > > with
> > > > > > > > >>>>>>>> "less
> > > > > > > > >>>>>>>>>>>>>>>>>>>> predictable
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
> > > > alternatives
> > > > > > > > really
> > > > > > > > >>>>>>>>>>>>>> interesting.
> > > > > > > > >>>>>>>>>>>>>>>>>>>> But I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good
> > and
> > > > we
> > > > > > should
> > > > > > > > >>>>>>>> continue
> > > > > > > > >>>>>>>>>>>> with
> > > > > > > > >>>>>>>>>>>>>>>> it!
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my
> > e-mail on
> > > > > > June
> > > > > > > > >> 20th:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning
> > behind
> > > > > > putting
> > > > > > > > >>>>>>>> isolation
> > > > > > > > >>>>>>>>>>>>>> level
> > > > > > > > >>>>>>>>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should
> > > > that
> > > > > > also
> > > > > > > > be
> > > > > > > > >> a
> > > > > > > > >>>>>>>> way
> > > > > > > > >>>>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>> give
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to
> > decide
> > > > > > whether
> > > > > > > > to
> > > > > > > > >>>>>>>> turn on
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned
> > about
> > > > > > how do
> > > > > > > > >> you
> > > > > > > > >>>>>>>> know
> > > > > > > > >>>>>>>>>>>> if a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written
> > under
> > > > EOS,
> > > > > > if
> > > > > > > > you
> > > > > > > > >>>>>>>> do not
> > > > > > > > >>>>>>>>>>>>>> have a
> > > > > > > > >>>>>>>>>>>>>>>>>>>> way
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
> > > > transactional or
> > > > > > not.
> > > > > > > > >> If
> > > > > > > > >>>>>>>> a
> > > > > > > > >>>>>>>>>> state
> > > > > > > > >>>>>>>>>>>>>>>> store
> > > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can
> > be
> > > > > > written
> > > > > > > > >>>>>>>> during
> > > > > > > > >>>>>>>>>>>> normal
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state
> > store
> > > > is not
> > > > > > > > >>>>>>>>>> transactional,
> > > > > > > > >>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written
> > under
> > > > EOS.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 7.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> My point was about not only
> > considering the
> > > > > > bytes
> > > > > > > > in
> > > > > > > > >>>>>>>> memory
> > > > > > > > >>>>>>>>>> in
> > > > > > > > >>>>>>>>>>>>>>>> config
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but
> > also
> > > > > > bytes
> > > > > > > > that
> > > > > > > > >>>>>>>> might
> > > > > > > > >>>>>>>>>> be
> > > > > > > > >>>>>>>>>>>>>>>>>>>> spilled
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering
> > > > whether you
> > > > > > > > >> should
> > > > > > > > >>>>>>>>>> remove
> > > > > > > > >>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory
> > > > bytes to
> > > > > > be
> > > > > > > > >> used
> > > > > > > > >>>>>>>> to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store
> > records." My
> > > > > > > > thinking
> > > > > > > > >>>>>>>> was
> > > > > > > > >>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>> even
> > > > > > > > >>>>>>>>>>>>>>>>>>>> if a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to
> > > > disk,
> > > > > > > > >> limiting
> > > > > > > > >>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>> overall
> > > > > > > > >>>>>>>>>>>>>>>>>>>> bytes
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it
> > again
> > > > and
> > > > > > > > >>>>>>>> considering
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>> recent
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much
> > > > sense
> > > > > > > > anymore.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I like the name
> > > > > > > > >>>>>>>> statestore.transaction.buffer.max.bytes that
> > > > > > > > >>>>>>>>>>>> you
> > > > > > > > >>>>>>>>>>>>>>>>>>>> proposed.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 8.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> > > > > > implementation
> > > > > > > > >>>>>>>> details) of
> > > > > > > > >>>>>>>>>>>> how
> > > > > > > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of
> > changelog
> > > > > > > > >>>>>>>> transactions,
> > > > > > > > >>>>>>>>>>>> state
> > > > > > > > >>>>>>>>>>>>>>>> store
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be
> > > > great.
> > > > > > > > Would
> > > > > > > > >>>> be
> > > > > > > > >>>>>>>>>> great
> > > > > > > > >>>>>>>>>>>> if
> > > > > > > > >>>>>>>>>>>>>>>> you
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the
> > > > > > behavior in
> > > > > > > > >>>>>>>> case of
> > > > > > > > >>>>>>>>>> a
> > > > > > > > >>>>>>>>>>>>>>>>>>>> failure.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional
> > state
> > > > > > store
> > > > > > > > >>>>>>>> recover
> > > > > > > > >>>>>>>>>>>> after a
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
> > > > transaction
> > > > > > > > buffer,
> > > > > > > > >>>>>>>> etc.
> > > > > > > > >>>>>>>>>>>> (that
> > > > > > > > >>>>>>>>>>>>>> is
> > > > > > > > >>>>>>>>>>>>>>>>>>>> what
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> > > > > > > > >>>>>>>> WriteBatchWithIndex
> > > > > > > > >>>>>>>>>>>>>>>>>>>> transactions
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to
> > > > exceed)
> > > > > > > > >>>>>>>> configured
> > > > > > > > >>>>>>>>>>>> memory
> > > > > > > > >>>>>>>>>>>>>>>>>>>> needs to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
> > > > Ultimately,
> > > > > > any
> > > > > > > > >>>>>>>> approach
> > > > > > > > >>>>>>>>>>>> based
> > > > > > > > >>>>>>>>>>>>>> on
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> cracking
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in
> > > > ways
> > > > > > it's
> > > > > > > > not
> > > > > > > > >>>>>>>> really
> > > > > > > > >>>>>>>>>>>>>>>> designed
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for is
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen
> > performance
> > > > or
> > > > > > > > >>>>>>>> consistency
> > > > > > > > >>>>>>>>>>>> issues.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing
> > these
> > > > > > early
> > > > > > > > >>>>>>>> commits?
> > > > > > > > >>>>>>>>>>>> While
> > > > > > > > >>>>>>>>>>>>>> not
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to
> > > > ensure
> > > > > > > > >>>>>>>> consistency
> > > > > > > > >>>>>>>>>>>> whilst
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> maintaining
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem
> > *very*
> > > > > > > > >>>>>>>> complicated, and
> > > > > > > > >>>>>>>>>>>> might
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for
> > users
> > > > as a
> > > > > > > > >>>>>>>> consequence.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of
> > this
> > > > > > KIP is
> > > > > > > > >>>>>>>> growing a
> > > > > > > > >>>>>>>>>>>> bit
> > > > > > > > >>>>>>>>>>>>>>>> out
> > > > > > > > >>>>>>>>>>>>>>>>>>>> of
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss
> > ideas
> > > > for
> > > > > > > > future
> > > > > > > > >>>>>>>>>>>>>>>> improvements, I
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> think
> > > >
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope
> > down
> > > > to a
> > > > > > > > design
> > > > > > > > >>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>> achieves
> > > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> most
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
> > > > > > restorations
> > > > > > > > >>>>>>>> during
> > > > > > > > >>>>>>>>>> dirty
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design
> > that
> > > > > > this KIP
> > > > > > > > >>>>>>>> produces
> > > > > > > > >>>>>>>>>>>> can
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ultimately
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially
> > if
> > > > the
> > > > > > bulk
> > > > > > > > of
> > > > > > > > >>>>>>>> it is
> > > > > > > > >>>>>>>>>>>>>> internal
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week
> > > > trying
> > > > > > to
> > > > > > > > >>>>>>>> re-work
> > > > > > > > >>>>>>>>>> the
> > > > > > > > >>>>>>>>>>>>>>>>>>>> original
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove
> > the
> > > > > > > > >>>>>>>> newTransaction()
> > > > > > > > >>>>>>>>>>>>>> method,
> > > > > > > > >>>>>>>>>>>>>>>>>>>> such
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
> > > > > > > > RocksDBStore.
> > > > > > > > >>>>>>>> That
> > > > > > > > >>>>>>>>>>>> way, if
> > > > > > > > >>>>>>>>>>>>>>>> we
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>> want to
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the
> > future,
> > > > > > like
> > > > > > > > the
> > > > > > > > >>>>>>>> SST
> > > > > > > > >>>>>>>>>> file
> > > > > > > > >>>>>>>>>>>>>>>>>>>> management
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so
> > with
> > > > > > little/no
> > > > > > > > >>>> API
> > > > > > > > >>>>>>>>>>>> changes.
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>>
> > > > > > > > >>>>>>>>>>>
> > > > > > > > >>>>>>>>>>
> > > > > > > > >>>>>>>>
> > > > > > > > >>>>>>>
> > > > > > > > >>>>>>
> > > > > > > > >>>>>
> > > > > > > > >>>>
> > > > > > > > >>>
> > > > > > > > >>
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > > >
> > > >
> >


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

Posted by Nick Telford <ni...@gmail.com>.
Hi Lucas,

TaskCorruptedException is how Streams signals that the Task state needs to
be wiped, so we can't retain that exception without also wiping state on
timeouts.

Regards,
Nick

On Wed, 18 Oct 2023 at 14:48, Lucas Brutschy <lb...@confluent.io.invalid>
wrote:

> Hi Nick,
>
> I think indeed the better behavior would be to retry commitTransaction
> until we risk running out of time to meet `max.poll.interval.ms`.
>
> However, if it's handled as a `TaskCorruptedException` at the moment,
> I would do the same in this KIP, and leave exception handling
> improvements to future work. 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 <ni...@gmail.com>
> wrote:
> >
> > Hi Lucas,
> >
> > Yeah, this is pretty much the direction I'm thinking of going in now. You
> > make an interesting point about committing on-error under
> > ALOS/READ_COMMITTED, although I haven't had a chance to think through the
> > implications yet.
> >
> > Something that I ran into earlier this week is an issue with the new
> > handling of TimeoutException. Without TX stores, TimeoutException under
> EOS
> > throws a TaskCorruptedException, which wipes the stores. However, with TX
> > stores, TimeoutException is now just bubbled up and dealt with as it is
> > under ALOS. The problem arises when the Producer#commitTransaction call
> > times out: Streams attempts to ignore the error and continue producing,
> > which causes the next call to Producer#send to throw
> > "IllegalStateException: Cannot attempt operation `send` because the
> > previous call to `commitTransaction` timed out and must be retried".
> >
> > I'm not sure what we should do here: retrying the commitTransaction seems
> > logical, but what if it times out again? Where do we draw the line and
> > shutdown the instance?
> >
> > Regards,
> > Nick
> >
> > On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <lbrutschy@confluent.io
> .invalid>
> > wrote:
> >
> > > Hi all,
> > >
> > > I think I liked your suggestion of allowing EOS with READ_UNCOMMITTED,
> > > but keep wiping the state on error, and I'd vote for this solution
> > > when introducing `default.state.isolation.level`. This way, we'd have
> > > the most low-risk roll-out of this feature (no behavior change without
> > > reconfiguration), with the possibility of switching to the most sane /
> > > battle-tested default settings in 4.0. Essentially, we'd have a
> > > feature flag but call it `default.state.isolation.level` and don't
> > > have to deprecate it later.
> > >
> > > So the possible configurations would then be this:
> > >
> > > 1. ALOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
> > > reads from DB.
> > > 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> > > WriteBatch/DB. Flush on error (see note below).
> > > 3. EOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
> > > reads from DB. Wipe state on error.
> > > 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> > > WriteBatch/DB.
> > >
> > > I believe the feature is important enough that we will see good
> > > adoption even without changing the default. In 4.0, when we have seen
> > > this being adopted and is battle-tested, we make READ_COMMITTED the
> > > default for EOS, or even READ_COMITTED always the default, depending
> > > on our experiences. And we could add a clever implementation of
> > > READ_UNCOMITTED with WriteBatches later.
> > >
> > > The only smell here is that `default.state.isolation.level` wouldn't
> > > be purely an IQ setting, but it would also (slightly) change the
> > > behavior of the processing, but that seems unavoidable as long as we
> > > haven't solve READ_UNCOMITTED IQ with WriteBatches.
> > >
> > > Minor: As for Bruno's point 4, I think if we are concerned about this
> > > 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).
> > >
> > > Cheers,
> > > Lucas
> > >
> > > On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <ni...@gmail.com>
> > > wrote:
> > > >
> > > > Hi Guozhang,
> > > >
> > > > The KIP as it stands introduces a new configuration,
> > > > default.state.isolation.level, which is independent of
> processing.mode.
> > > > It's intended that this new configuration be used to configure a
> global
> > > IQ
> > > > isolation level in the short term, with a future KIP introducing the
> > > > capability to change the isolation level on a per-query basis,
> falling
> > > back
> > > > to the "default" defined by this config. That's why I called it
> > > "default",
> > > > for future-proofing.
> > > >
> > > > However, it currently includes the caveat that READ_UNCOMMITTED is
> not
> > > > available under EOS. I think this is the coupling you are alluding
> to?
> > > >
> > > > This isn't intended to be a restriction of the API, but is currently
> a
> > > > technical limitation. However, after discussing with some users about
> > > > use-cases that would require READ_UNCOMMITTED under EOS, I'm
> inclined to
> > > > remove that clause and put in the necessary work to make that
> combination
> > > > possible now.
> > > >
> > > > I currently see two possible approaches:
> > > >
> > > >    1. Disable TX StateStores internally when the IsolationLevel is
> > > >    READ_UNCOMMITTED and the processing.mode is EOS. This is more
> > > difficult
> > > >    than it sounds, as there are many assumptions being made
> throughout
> > > the
> > > >    internals about the guarantees StateStores provide. It would
> > > definitely add
> > > >    a lot of extra "if (read_uncommitted && eos)" branches,
> complicating
> > > >    maintenance and testing.
> > > >    2. Invest the time *now* to make READ_UNCOMMITTED of EOS
> StateStores
> > > >    possible. I have some ideas on how this could be achieved, but
> they
> > > would
> > > >    need testing and could introduce some additional issues. The
> benefit
> > > of
> > > >    this approach is that it would make query-time IsolationLevels
> much
> > > simpler
> > > >    to implement in the future.
> > > >
> > > > Unfortunately, both will require considerable work that will further
> > > delay
> > > > this KIP, which was the reason I placed the restriction in the KIP
> in the
> > > > first place.
> > > >
> > > > Regards,
> > > > Nick
> > > >
> > > > On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <
> guozhang.wang.us@gmail.com>
> > > > wrote:
> > > >
> > > > > Hello Nick,
> > > > >
> > > > > First of all, thanks a lot for the great effort you've put in
> driving
> > > > > this KIP! I really like it coming through finally, as many people
> in
> > > > > the community have raised this. At the same time I honestly feel a
> bit
> > > > > ashamed for not putting enough of my time supporting it and
> pushing it
> > > > > through the finish line (you raised this KIP almost a year ago).
> > > > >
> > > > > I briefly passed through the DISCUSS thread so far, not sure I've
> 100
> > > > > percent digested all the bullet points. But with the goal of
> trying to
> > > > > help take it through the finish line in mind, I'd want to throw
> > > > > thoughts on top of my head only on the point #4 above which I felt
> may
> > > > > be the main hurdle for the current KIP to drive to a consensus now.
> > > > >
> > > > > The general question I asked myself is, whether we want to couple
> "IQ
> > > > > reading mode" with "processing mode". While technically I tend to
> > > > > agree with you that, it's feels like a bug if some single user
> chose
> > > > > "EOS" for processing mode while choosing "read uncommitted" for IQ
> > > > > reading mode, at the same time, I'm thinking if it's possible that
> > > > > there could be two different persons (or even two teams) that
> would be
> > > > > using the stream API to build the app, and the IQ API to query the
> > > > > running state of the app. I know this is less of a technical thing
> but
> > > > > rather a more design stuff, but if it could be ever the case, I'm
> > > > > wondering if the personale using the IQ API knows about the risks
> of
> > > > > using read uncommitted but still chose so for the favor of
> > > > > performance, no matter if the underlying stream processing mode
> > > > > configured by another personale is EOS or not. In that regard, I'm
> > > > > leaning towards a "leaving the door open, and close it later if we
> > > > > found it's a bad idea" aspect with a configuration 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 can still enforce
> > > > > coupling them.
> > > > >
> > > > > Again, just my 2c here. Thanks again for the great patience and
> > > > > diligence on this KIP.
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <
> nick.telford@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > Hi Bruno,
> > > > > >
> > > > > > 4.
> > > > > > I'll hold off on making that change until we have a consensus as
> to
> > > what
> > > > > > configuration to use to control all of this, as it'll be
> affected by
> > > the
> > > > > > decision on EOS isolation levels.
> > > > > >
> > > > > > 5.
> > > > > > Done. I've chosen "committedOffsets".
> > > > > >
> > > > > > Regards,
> > > > > > Nick
> > > > > >
> > > > > > On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <ca...@apache.org>
> > > wrote:
> > > > > >
> > > > > > > Hi Nick,
> > > > > > >
> > > > > > > 1.
> > > > > > > Yeah, you are probably right that it does not make too much
> sense.
> > > > > > > Thanks for the clarification!
> > > > > > >
> > > > > > >
> > > > > > > 4.
> > > > > > > Yes, sorry for the back and forth, but I think for the sake of
> the
> > > KIP
> > > > > > > it is better to let the ALOS behavior as it is for now due to
> the
> > > > > > > possible issues you would run into. Maybe we can find a
> solution
> > > in the
> > > > > > > future. Now the question returns to whether we really need
> > > > > > > default.state.isolation.level. Maybe the config could be the
> > > feature
> > > > > > > flag Sophie requested.
> > > > > > >
> > > > > > >
> > > > > > > 5.
> > > > > > > There 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 committedOffsetFor().
> > > > > > >
> > > > > > >
> > > > > > > Best,
> > > > > > > Bruno
> > > > > > >
> > > > > > >
> > > > > > > On 10/13/23 3:21 PM, Nick Telford wrote:
> > > > > > > > Hi Bruno,
> > > > > > > >
> > > > > > > > Thanks for getting back to me.
> > > > > > > >
> > > > > > > > 1.
> > > > > > > > I think this should be possible. Are you thinking of the
> > > situation
> > > > > where
> > > > > > > a
> > > > > > > > user may downgrade to a previous version of Kafka Streams? In
> > > that
> > > > > case,
> > > > > > > > sadly, the RocksDBStore would get wiped by the older version
> of
> > > Kafka
> > > > > > > > Streams anyway, because that version wouldn't understand the
> > > extra
> > > > > column
> > > > > > > > family (that holds offsets), so the missing Position file
> would
> > > > > > > > automatically get rebuilt when the store is rebuilt from the
> > > > > changelog.
> > > > > > > > Are there other situations than downgrade where a
> transactional
> > > store
> > > > > > > could
> > > > > > > > be replaced by a non-transactional one? I can't think of any.
> > > > > > > >
> > > > > > > > 2.
> > > > > > > > Ahh yes, the Test Plan - my Kryptonite! This section
> definitely
> > > > > needs to
> > > > > > > be
> > > > > > > > fleshed out. I'll work on that. How much detail do you need?
> > > > > > > >
> > > > > > > > 3.
> > > > > > > > See my previous email discussing this.
> > > > > > > >
> > > > > > > > 4.
> > > > > > > > Hmm, this is an interesting point. Are you suggesting that
> under
> > > ALOS
> > > > > > > > READ_COMMITTED should not be supported?
> > > > > > > >
> > > > > > > > Regards,
> > > > > > > > Nick
> > > > > > > >
> > > > > > > > On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <
> cadonna@apache.org>
> > > > > wrote:
> > > > > > > >
> > > > > > > >> Hi Nick,
> > > > > > > >>
> > > > > > > >> I think the KIP is converging!
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> 1.
> > > > > > > >> I am wondering whether it makes sense to write the position
> file
> > > > > during
> > > > > > > >> close as we do for the checkpoint file, so that in case the
> > > state
> > > > > store
> > > > > > > >> is replaced with a non-transactional state store the
> > > > > non-transactional
> > > > > > > >> state store finds the position file. I think, this is not
> > > strictly
> > > > > > > >> needed, but would be a nice behavior instead of just
> deleting
> > > the
> > > > > > > >> position file.
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> 2.
> > > > > > > >> The test plan does not mention integration tests. Do you not
> > > need to
> > > > > > > >> extend existing ones and add new ones. Also for upgrading
> and
> > > > > > > >> downgrading you might need integration and/or system tests.
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> 3.
> > > > > > > >> I think Sophie made a point. Although, IQ reading from
> > > uncommitted
> > > > > data
> > > > > > > >> under EOS might be considered a bug by some people. Thus,
> your
> > > KIP
> > > > > would
> > > > > > > >> fix a bug rather than changing the intended behavior.
> However, I
> > > > > also
> > > > > > > >> see that a feature flag would help users that rely on this
> buggy
> > > > > > > >> behavior (at least until AK 4.0).
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> 4.
> > > > > > > >> This is related to the previous point. I assume that the
> > > difference
> > > > > > > >> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is
> that in
> > > the
> > > > > > > >> former you enable transactions on the state store and in the
> > > latter
> > > > > you
> > > > > > > >> disable them. If my assumption is correct, I think that is
> an
> > > issue.
> > > > > > > >> Let's assume under ALOS Streams fails over a couple of times
> > > more or
> > > > > > > >> less at the same step in processing after value 3 is added
> to an
> > > > > > > >> aggregation but the offset of the corresponding input record
> > > was not
> > > > > > > >> committed. Without transactions disabled, the aggregation
> value
> > > > > would
> > > > > > > >> increase by 3 for each failover. With transactions enabled,
> > > value 3
> > > > > > > >> would only be added to the aggregation once when the offset
> of
> > > the
> > > > > input
> > > > > > > >> record is committed and the transaction finally completes.
> So
> > > the
> > > > > > > >> content of the state store would change depending on the
> > > > > configuration
> > > > > > > >> for IQ. IMO, the content of the state store should be
> > > independent
> > > > > from
> > > > > > > >> IQ. Given this issue, I propose to not use transactions with
> > > ALOS at
> > > > > > > >> all. I was a big proponent of using transactions with ALOS,
> but
> > > I
> > > > > > > >> realized that transactions with ALOS is not as easy as
> enabling
> > > > > > > >> transactions on state stores. Another aspect that is
> > > problematic is
> > > > > that
> > > > > > > >> the changelog topic which actually replicates the state
> store
> > > is not
> > > > > > > >> transactional under ALOS. Thus, it might happen that the
> state
> > > > > store and
> > > > > > > >> the changelog differ in their content. All of this is maybe
> > > solvable
> > > > > > > >> somehow, but for the sake of this KIP, I would leave it for
> the
> > > > > future.
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> Best,
> > > > > > > >> Bruno
> > > > > > > >>
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > > > > > > >>> Hey Nick! First of all thanks for taking up this awesome
> > > feature,
> > > > > I'm
> > > > > > > >> sure
> > > > > > > >>> every single
> > > > > > > >>> Kafka Streams user and dev would agree that it is sorely
> > > needed.
> > > > > > > >>>
> > > > > > > >>> I've just been catching up on the KIP and surrounding
> > > discussion,
> > > > > so
> > > > > > > >> please
> > > > > > > >>> forgive me
> > > > > > > >>> for any misunderstandings or misinterpretations of the
> current
> > > > > plan and
> > > > > > > >>> don't hesitate to
> > > > > > > >>> correct me.
> > > > > > > >>>
> > > > > > > >>> Before I jump in, I just want to say that having seen this
> > > drag on
> > > > > for
> > > > > > > so
> > > > > > > >>> long, my singular
> > > > > > > >>> goal in responding is to help this KIP past a perceived
> > > impasse so
> > > > > we
> > > > > > > can
> > > > > > > >>> finally move on
> > > > > > > >>> to voting and implementing it. Long discussions are to be
> > > expected
> > > > > for
> > > > > > > >>> major features like
> > > > > > > >>> this but it's completely on us as the Streams devs to make
> sure
> > > > > there
> > > > > > > is
> > > > > > > >> an
> > > > > > > >>> end in sight
> > > > > > > >>> for any ongoing discussion.
> > > > > > > >>>
> > > > > > > >>> With that said, it's my understanding that the KIP as
> currently
> > > > > > > proposed
> > > > > > > >> is
> > > > > > > >>> just not tenable
> > > > > > > >>> for Kafka Streams, and would prevent some EOS users from
> > > upgrading
> > > > > to
> > > > > > > the
> > > > > > > >>> version it
> > > > > > > >>> first appears in. Given that we can't predict or guarantee
> > > whether
> > > > > any
> > > > > > > of
> > > > > > > >>> the followup KIPs
> > > > > > > >>> would be completed in the same release cycle as this one,
> we
> > > need
> > > > > to
> > > > > > > make
> > > > > > > >>> sure that the
> > > > > > > >>> feature is either compatible with all current users or else
> > > > > > > >> feature-flagged
> > > > > > > >>> so that they may
> > > > > > > >>> opt in/out.
> > > > > > > >>>
> > > > > > > >>> Therefore, IIUC we need to have either (or both) of these
> as
> > > > > > > >>> fully-implemented config options:
> > > > > > > >>> 1. default.state.isolation.level
> > > > > > > >>> 2. enable.transactional.state.stores
> > > > > > > >>>
> > > > > > > >>> This way EOS users for whom read_committed semantics are
> not
> > > > > viable can
> > > > > > > >>> still upgrade,
> > > > > > > >>> and either use the isolation.level config to leverage the
> new
> > > txn
> > > > > state
> > > > > > > >>> stores without sacrificing
> > > > > > > >>> their application semantics, or else simply keep the
> > > transactional
> > > > > > > state
> > > > > > > >>> stores disabled until we
> > > > > > > >>> are able to fully implement the isolation level
> configuration
> > > at
> > > > > either
> > > > > > > >> an
> > > > > > > >>> application or query level.
> > > > > > > >>>
> > > > > > > >>> Frankly you are the expert here and know much more about
> the
> > > > > tradeoffs
> > > > > > > in
> > > > > > > >>> both semantics and
> > > > > > > >>> effort level of implementing one of these configs vs the
> > > other. In
> > > > > my
> > > > > > > >>> opinion, either option would
> > > > > > > >>> be fine and I would leave the decision of which one to
> include
> > > in
> > > > > this
> > > > > > > >> KIP
> > > > > > > >>> completely up to you.
> > > > > > > >>> I just don't see a way for the KIP to proceed without some
> > > > > variation of
> > > > > > > >> the
> > > > > > > >>> above that would allow
> > > > > > > >>> EOS users to opt-out of read_committed.
> > > > > > > >>>
> > > > > > > >>> (If it's all the same to you, I would recommend 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 that gets released.
> > > > > > > >>> So personally, my choice would be to add the feature flag
> and
> > > > > leave it
> > > > > > > >> off
> > > > > > > >>> by default. If all 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
> > > > > > > >>>
> > > > > > > >>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > > > > nick.telford@gmail.com>
> > > > > > > >> wrote:
> > > > > > > >>>
> > > > > > > >>>> Hi Bruno,
> > > > > > > >>>>
> > > > > > > >>>> Agreed, I can live with that for now.
> > > > > > > >>>>
> > > > > > > >>>> In an effort to keep the scope of this KIP from
> expanding, I'm
> > > > > leaning
> > > > > > > >>>> towards just providing a configurable
> > > > > default.state.isolation.level
> > > > > > > and
> > > > > > > >>>> removing IsolationLevel from the StateStoreContext. This
> > > would be
> > > > > > > >>>> compatible with adding support for query-time
> IsolationLevels
> > > in
> > > > > the
> > > > > > > >>>> future, whilst providing a way for users to select an
> > > isolation
> > > > > level
> > > > > > > >> now.
> > > > > > > >>>>
> > > > > > > >>>> The big problem with this, however, is that if a user
> selects
> > > > > > > >>>> processing.mode
> > > > > > > >>>> = "exactly-once(-v2|-beta)", and
> > > default.state.isolation.level =
> > > > > > > >>>> "READ_UNCOMMITTED", we need to guarantee that the data
> isn't
> > > > > written
> > > > > > > to
> > > > > > > >>>> disk until commit() is called, but we also need to permit
> IQ
> > > > > threads
> > > > > > > to
> > > > > > > >>>> read from the ongoing transaction.
> > > > > > > >>>>
> > > > > > > >>>> A simple solution would be to (temporarily) forbid this
> > > > > combination of
> > > > > > > >>>> configuration, and have default.state.isolation.level
> > > > > automatically
> > > > > > > >> switch
> > > > > > > >>>> to READ_COMMITTED when processing.mode is anything other
> than
> > > > > > > >>>> at-least-once. Do you think this would be acceptable?
> > > > > > > >>>>
> > > > > > > >>>> In a later KIP, we can add support for query-time
> isolation
> > > > > levels and
> > > > > > > >>>> solve this particular problem there, which would relax
> this
> > > > > > > restriction.
> > > > > > > >>>>
> > > > > > > >>>> Regards,
> > > > > > > >>>> Nick
> > > > > > > >>>>
> > > > > > > >>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> > > cadonna@apache.org>
> > > > > > > >> 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,
> > > > > > > >>>>> Bruno
> > > > > > > >>>>>
> > > > > > > >>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > > > > > > >>>>>> 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 <
> > > > > nick.telford@gmail.com
> > > > > > > >
> > > > > > > >>>>> wrote:
> > > > > > > >>>>>>
> > > > > > > >>>>>>> Hi everyone,
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> I agree that having IsolationLevel be determined at
> > > query-time
> > > > > is
> > > > > > > the
> > > > > > > >>>>>>> ideal design, but there are a few sticking points:
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> 1.
> > > > > > > >>>>>>> There needs to be some way to communicate the
> > > IsolationLevel
> > > > > down
> > > > > > > to
> > > > > > > >>>> the
> > > > > > > >>>>>>> RocksDBStore itself, so that the query can respect it.
> > > Since
> > > > > stores
> > > > > > > >>>> are
> > > > > > > >>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
> > > > > MeteredStore,
> > > > > > > >>>>> etc.),
> > > > > > > >>>>>>> we need some way to deliver that information to the
> bottom
> > > > > layer.
> > > > > > > For
> > > > > > > >>>>> IQv2,
> > > > > > > >>>>>>> we can use the existing State#query() method, but IQv1
> has
> > > no
> > > > > way
> > > > > > > to
> > > > > > > >>>> do
> > > > > > > >>>>>>> this.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> A simple approach, which would potentially open up
> other
> > > > > options,
> > > > > > > >>>> would
> > > > > > > >>>>> be
> > > > > > > >>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> > > > > > > >>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > > > > > > ReadOnlyKeyValueStore
> > > > > > > >>>>> (and
> > > > > > > >>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore,
> > > etc.).
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> 2.
> > > > > > > >>>>>>> As mentioned above, RocksDB WriteBatches are not
> > > thread-safe,
> > > > > which
> > > > > > > >>>>> causes
> > > > > > > >>>>>>> a problem if we want to provide READ_UNCOMMITTED
> > > Iterators. I
> > > > > also
> > > > > > > >>>> had a
> > > > > > > >>>>>>> look at RocksDB Transactions[1], but they solve a very
> > > > > different
> > > > > > > >>>>> problem,
> > > > > > > >>>>>>> and have the same thread-safety issue.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> One possible approach that I mentioned is chaining
> > > > > WriteBatches:
> > > > > > > >> every
> > > > > > > >>>>>>> time a new Interactive Query is received (i.e.
> > > readOnlyView,
> > > > > see
> > > > > > > >>>> above,
> > > > > > > >>>>>>> is called) we "freeze" the existing WriteBatch, and
> start a
> > > > > new one
> > > > > > > >>>> for
> > > > > > > >>>>> new
> > > > > > > >>>>>>> writes. The Interactive Query queries the "chain" of
> > > previous
> > > > > > > >>>>> WriteBatches
> > > > > > > >>>>>>> + the underlying database; while the StreamThread
> starts
> > > > > writing to
> > > > > > > >>>> the
> > > > > > > >>>>>>> *new* WriteBatch. On-commit, the StreamThread would
> write
> > > *all*
> > > > > > > >>>>>>> WriteBatches in the chain to the database (that have
> not
> > > yet
> > > > > been
> > > > > > > >>>>> written).
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> WriteBatches would be closed/freed only when they have
> been
> > > > > both
> > > > > > > >>>>>>> committed, and all open Interactive Queries on them
> have
> > > been
> > > > > > > closed.
> > > > > > > >>>>> This
> > > > > > > >>>>>>> would require some reference counting.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Obviously a drawback of this approach is the potential
> for
> > > > > > > increased
> > > > > > > >>>>>>> memory usage: if an Interactive Query is long-lived,
> for
> > > > > example by
> > > > > > > >>>>> doing a
> > > > > > > >>>>>>> full scan over a large database, or even just pausing
> in
> > > the
> > > > > middle
> > > > > > > >> of
> > > > > > > >>>>> an
> > > > > > > >>>>>>> iteration, then the existing chain of WriteBatches
> could be
> > > > > kept
> > > > > > > >>>> around
> > > > > > > >>>>> for
> > > > > > > >>>>>>> a long time, potentially forever.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> --
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> A.
> > > > > > > >>>>>>> Going off on a tangent, it looks like in addition to
> > > supporting
> > > > > > > >>>>>>> READ_COMMITTED queries, we could go further and support
> > > > > > > >>>> REPEATABLE_READ
> > > > > > > >>>>>>> queries (i.e. where subsequent reads to the same key
> in the
> > > > > same
> > > > > > > >>>>>>> Interactive Query are guaranteed to yield the same
> value)
> > > by
> > > > > making
> > > > > > > >>>> use
> > > > > > > >>>>> of
> > > > > > > >>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so
> the
> > > > > > > >> performance
> > > > > > > >>>>>>> impact is likely to be negligible, but they do require
> > > that the
> > > > > > > >>>>> Interactive
> > > > > > > >>>>>>> Query session can be explicitly closed.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> This could be achieved if we made the above
> readOnlyView
> > > > > interface
> > > > > > > >>>> look
> > > > > > > >>>>>>> more like:
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> > > > > > > >>>> ReadOnlyKeyValueStore<K,
> > > > > > > >>>>>>> V>, AutoCloseable {}
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > > > > > > >>>>>>>        ...
> > > > > > > >>>>>>>        ReadOnlyKeyValueView<K, V>
> > > readOnlyView(IsolationLevel
> > > > > > > >>>>> isolationLevel);
> > > > > > > >>>>>>> }
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> But this would be a breaking change, as existing IQv1
> > > queries
> > > > > are
> > > > > > > >>>>>>> guaranteed to never call store.close(), and therefore
> these
> > > > > would
> > > > > > > >> leak
> > > > > > > >>>>>>> memory under REPEATABLE_READ.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> B.
> > > > > > > >>>>>>> One thing that's notable: MyRocks states that they
> support
> > > > > > > >>>>> READ_COMMITTED
> > > > > > > >>>>>>> and REPEATABLE_READ, but they make no mention of
> > > > > > > >>>> READ_UNCOMMITTED[3][4].
> > > > > > > >>>>>>> This could be because doing so is technically
> > > > > difficult/impossible
> > > > > > > >>>> using
> > > > > > > >>>>>>> the primitives available in RocksDB.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> --
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Lucas, to address your points:
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> U1.
> > > > > > > >>>>>>> It's only "SHOULD" to permit alternative (i.e.
> non-RocksDB)
> > > > > > > >>>>>>> implementations of StateStore that do not support
> atomic
> > > > > writes.
> > > > > > > >>>>> Obviously
> > > > > > > >>>>>>> in those cases, the guarantees Kafka Streams
> > > provides/expects
> > > > > would
> > > > > > > >> be
> > > > > > > >>>>>>> relaxed. Do you think we should require all
> > > implementations to
> > > > > > > >> support
> > > > > > > >>>>>>> atomic writes?
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> U2.
> > > > > > > >>>>>>> Stores can support multiple IsolationLevels. As we've
> > > discussed
> > > > > > > >> above,
> > > > > > > >>>>> the
> > > > > > > >>>>>>> ideal scenario would be to specify the IsolationLevel
> at
> > > > > > > query-time.
> > > > > > > >>>>>>> Failing that, I think the second-best approach is to
> > > define the
> > > > > > > >>>>>>> IsolationLevel for *all* queries based on the
> > > processing.mode,
> > > > > > > which
> > > > > > > >>>> is
> > > > > > > >>>>>>> what the default StateStoreContext#isolationLevel()
> > > achieves.
> > > > > Would
> > > > > > > >>>> you
> > > > > > > >>>>>>> prefer an alternative?
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> While the existing implementation is equivalent to
> > > > > > > READ_UNCOMMITTED,
> > > > > > > >>>>> this
> > > > > > > >>>>>>> can yield unexpected results/errors under EOS, if a
> > > > > transaction is
> > > > > > > >>>>> rolled
> > > > > > > >>>>>>> back. While this would be a change in behaviour for
> users,
> > > it
> > > > > would
> > > > > > > >>>> look
> > > > > > > >>>>>>> more like a bug fix than a breaking change. That said,
> we
> > > > > *could*
> > > > > > > >> make
> > > > > > > >>>>> it
> > > > > > > >>>>>>> configurable, and default to the existing behaviour
> > > > > > > >> (READ_UNCOMMITTED)
> > > > > > > >>>>>>> instead of inferring it from the processing.mode?
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> N1, N2.
> > > > > > > >>>>>>> These were only primitives to avoid boxing costs, but
> since
> > > > > this is
> > > > > > > >>>> not
> > > > > > > >>>>> a
> > > > > > > >>>>>>> performance sensitive area, it should be fine to
> change if
> > > > > that's
> > > > > > > >>>>> desirable.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> N3.
> > > > > > > >>>>>>> It's because the store "manages its own offsets", which
> > > > > includes
> > > > > > > both
> > > > > > > >>>>>>> committing the offset, *and providing it* via
> > > > > getCommittedOffset().
> > > > > > > >>>>>>> Personally, I think "managesOffsets" conveys this best,
> > > but I
> > > > > don't
> > > > > > > >>>> mind
> > > > > > > >>>>>>> changing it if the nomenclature is unclear.
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> Sorry for the massive emails/essays!
> > > > > > > >>>>>>> --
> > > > > > > >>>>>>> Nick
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> 1:
> https://github.com/facebook/rocksdb/wiki/Transactions
> > > > > > > >>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > > > > > > >>>>>>> 3:
> > > > > > >
> https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > > > > > >>>>>>> 4:
> > > https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > > > > > >>>>>>>
> > > > > > > >>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > > > > > > >>>>>>> <lb...@confluent.io.invalid> wrote:
> > > > > > > >>>>>>>
> > > > > > > >>>>>>>> Hi Nick,
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> since I last read it in April, the KIP has become much
> > > > > cleaner and
> > > > > > > >>>>>>>> easier to read. Great work!
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> It feels to me the last big open point is whether we
> can
> > > > > implement
> > > > > > > >>>>>>>> isolation level as a query parameter. I understand
> that
> > > there
> > > > > are
> > > > > > > >>>>>>>> implementation concerns, but as Colt says, it would
> be a
> > > great
> > > > > > > >>>>>>>> addition, and would also simplify the migration path
> for
> > > this
> > > > > > > >> change.
> > > > > > > >>>>>>>> Is the implementation problem you mentioned caused by
> the
> > > > > > > WriteBatch
> > > > > > > >>>>>>>> not having a notion of a snapshot, as the underlying
> DB
> > > > > iterator
> > > > > > > >>>> does?
> > > > > > > >>>>>>>> In that case, I am not sure a chain of WriteBatches
> as you
> > > > > propose
> > > > > > > >>>>>>>> would fully solve the problem, but maybe I didn't dig
> > > enough
> > > > > into
> > > > > > > >> the
> > > > > > > >>>>>>>> details to fully understand it.
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> If it's not possible to implement it now, would it be
> an
> > > > > option to
> > > > > > > >>>>>>>> make sure in this KIP that we do not fully close the
> door
> > > on
> > > > > > > >>>> per-query
> > > > > > > >>>>>>>> isolation levels in the interface, as it may be
> possible
> > > to
> > > > > > > >> implement
> > > > > > > >>>>>>>> the missing primitives in RocksDB or Speedb in the
> future.
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> Understanding:
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to
> be
> > > > > persisted
> > > > > > > >>>>>>>> atomically with the records?
> > > > > > > >>>>>>>> * U2) Don't understand the default implementation of
> > > > > > > >>>> `isolationLevel`.
> > > > > > > >>>>>>>> The isolation level should be a property of the
> underlying
> > > > > store,
> > > > > > > >> and
> > > > > > > >>>>>>>> not be defined by the default config? Existing stores
> > > probably
> > > > > > > don't
> > > > > > > >>>>>>>> guarantee READ_COMMITTED, so the default should be to
> > > return
> > > > > > > >>>>>>>> READ_UNCOMMITTED.
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> Nits:
> > > > > > > >>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong`
> > > return
> > > > > type,
> > > > > > > >> to
> > > > > > > >>>>>>>> avoid the `null`?
> > > > > > > >>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
> > > > > `OptionalLong`
> > > > > > > >>>>>>>> return type, to avoid the `-1`?
> > > > > > > >>>>>>>> * N3) I don't understand why `managesOffsets` uses the
> > > > > 'manage'
> > > > > > > >> verb,
> > > > > > > >>>>>>>> whereas all other methods use the "commits" verb. I'd
> > > suggest
> > > > > > > >>>>>>>> `commitsOffsets`.
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> Either way, it feels this KIP is very close to the
> finish
> > > > > line,
> > > > > > > I'm
> > > > > > > >>>>>>>> looking forward to seeing this in production!
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> Cheers,
> > > > > > > >>>>>>>> Lucas
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> > > > > colt@littlehorse.io
> > > > > > > >
> > > > > > > >>>>> wrote:
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>>> Making IsolationLevel a query-time constraint,
> rather
> > > than
> > > > > > > linking
> > > > > > > >>>> it
> > > > > > > >>>>>>>> to
> > > > > > > >>>>>>>>> the processing.guarantee.
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>> As I understand it, would this allow even a user of
> EOS
> > > to
> > > > > > > control
> > > > > > > >>>>>>>> whether
> > > > > > > >>>>>>>>> reading committed or uncommitted records? If so, I am
> > > highly
> > > > > in
> > > > > > > >>>> favor
> > > > > > > >>>>> of
> > > > > > > >>>>>>>>> this.
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>> I know that I was one of the early people to point
> out
> > > the
> > > > > > > current
> > > > > > > >>>>>>>>> shortcoming that IQ reads uncommitted records, but
> just
> > > this
> > > > > > > >>>> morning I
> > > > > > > >>>>>>>>> realized a pattern we use which means that (for
> certain
> > > > > queries)
> > > > > > > >> our
> > > > > > > >>>>>>>> system
> > > > > > > >>>>>>>>> needs to be able to read uncommitted records, which
> is
> > > the
> > > > > > > current
> > > > > > > >>>>>>>> behavior
> > > > > > > >>>>>>>>> of Kafka Streams in EOS.***
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>> If IsolationLevel being a query-time decision allows
> for
> > > > > this,
> > > > > > > then
> > > > > > > >>>>> that
> > > > > > > >>>>>>>>> would be amazing. I would also vote that the default
> > > behavior
> > > > > > > >> should
> > > > > > > >>>>> be
> > > > > > > >>>>>>>> for
> > > > > > > >>>>>>>>> reading uncommitted records, because it is totally
> > > possible
> > > > > for a
> > > > > > > >>>>> valid
> > > > > > > >>>>>>>>> application to depend on that behavior, and breaking
> it
> > > in a
> > > > > > > minor
> > > > > > > >>>>>>>> release
> > > > > > > >>>>>>>>> might be a bit strong.
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>> *** (Note, for the curious reader....) Our
> use-case/query
> > > > > pattern
> > > > > > > >>>> is a
> > > > > > > >>>>>>>> bit
> > > > > > > >>>>>>>>> complex, but reading "uncommitted" records is
> actually
> > > safe
> > > > > in
> > > > > > > our
> > > > > > > >>>>> case
> > > > > > > >>>>>>>>> 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 soon as the processor processes
> the
> > > > > result.
> > > > > > > If
> > > > > > > >>>>> they
> > > > > > > >>>>>>>>> can't read uncommitted records, they may receive a
> "201 -
> > > > > > > Created"
> > > > > > > >>>>>>>>> response, immediately followed by a "404 - 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 <
> > > > > > > >>>> nick.telford@gmail.com>
> > > > > > > >>>>>>>> wrote:
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>>> Addendum:
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>> I think we would also face the same problem with the
> > > > > approach
> > > > > > > John
> > > > > > > >>>>>>>> outlined
> > > > > > > >>>>>>>>>> earlier (using the record cache as a transaction
> buffer
> > > and
> > > > > > > >>>> flushing
> > > > > > > >>>>>>>> it
> > > > > > > >>>>>>>>>> straight to SST files). This is because the record
> cache
> > > > > (the
> > > > > > > >>>>>>>> ThreadCache
> > > > > > > >>>>>>>>>> class) is not thread-safe, so every commit would
> > > invalidate
> > > > > open
> > > > > > > >> IQ
> > > > > > > >>>>>>>>>> Iterators in the same way that RocksDB WriteBatches
> do.
> > > > > > > >>>>>>>>>> --
> > > > > > > >>>>>>>>>> Nick
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > > > > > > >>>> nick.telford@gmail.com>
> > > > > > > >>>>>>>>>> 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.
> > > > > > > >>>>>>>>>>> 2. Making IsolationLevel a query-time constraint,
> > > rather
> > > > > than
> > > > > > > >>>>>>>> linking it
> > > > > > > >>>>>>>>>>> to the processing.guarantee.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> There's a wrinkle that makes this a challenge:
> > > Interactive
> > > > > > > >> Queries
> > > > > > > >>>>>>>> that
> > > > > > > >>>>>>>>>>> open an Iterator, when using transactions and
> > > > > READ_UNCOMMITTED.
> > > > > > > >>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries
> > > need
> > > > > to be
> > > > > > > >>>> able
> > > > > > > >>>>>>>> to
> > > > > > > >>>>>>>>>>> read records from the currently uncommitted
> transaction
> > > > > buffer
> > > > > > > >>>>>>>>>>> (WriteBatch). This includes for Iterators, which
> should
> > > > > iterate
> > > > > > > >>>>>>>> both the
> > > > > > > >>>>>>>>>>> transaction buffer and underlying database (using
> > > > > > > >>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> The issue is that when the StreamThread commits, it
> > > writes
> > > > > the
> > > > > > > >>>>>>>> current
> > > > > > > >>>>>>>>>>> WriteBatch to RocksDB *and then clears the
> WriteBatch*.
> > > > > > > Clearing
> > > > > > > >>>> the
> > > > > > > >>>>>>>>>>> WriteBatch while an Interactive Query holds an open
> > > > > Iterator on
> > > > > > > >> it
> > > > > > > >>>>>>>> will
> > > > > > > >>>>>>>>>>> invalidate the Iterator. Worse, it turns out that
> > > Iterators
> > > > > > > over
> > > > > > > >> a
> > > > > > > >>>>>>>>>>> WriteBatch become invalidated not just when the
> > > WriteBatch
> > > > > is
> > > > > > > >>>>>>>> cleared,
> > > > > > > >>>>>>>>>> but
> > > > > > > >>>>>>>>>>> also when the Iterators' current key receives a new
> > > write.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> Now that I'm writing this, I remember that this is
> the
> > > > > major
> > > > > > > >>>> reason
> > > > > > > >>>>>>>> that
> > > > > > > >>>>>>>>>> I
> > > > > > > >>>>>>>>>>> switched the original design from having a
> query-time
> > > > > > > >>>>>>>> IsolationLevel to
> > > > > > > >>>>>>>>>>> having the IsolationLevel linked to the
> > > transactionality
> > > > > of the
> > > > > > > >>>>>>>> stores
> > > > > > > >>>>>>>>>>> themselves.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> It *might* be possible to resolve this, by having a
> > > > > "chain" of
> > > > > > > >>>>>>>>>>> WriteBatches, with the StreamThread switching to a
> new
> > > > > > > WriteBatch
> > > > > > > >>>>>>>>>> whenever
> > > > > > > >>>>>>>>>>> a new Interactive Query attempts to read from the
> > > > > database, but
> > > > > > > >>>> that
> > > > > > > >>>>>>>>>> could
> > > > > > > >>>>>>>>>>> cause some performance problems/memory pressure
> when
> > > > > subjected
> > > > > > > to
> > > > > > > >>>> a
> > > > > > > >>>>>>>> high
> > > > > > > >>>>>>>>>>> Interactive Query load. It would also reduce the
> > > > > efficiency of
> > > > > > > >>>>>>>>>> WriteBatches
> > > > > > > >>>>>>>>>>> on-commit, as we'd have to write N WriteBatches,
> where
> > > N
> > > > > is the
> > > > > > > >>>>>>>> number of
> > > > > > > >>>>>>>>>>> Interactive Queries since the last commit.
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> I realise this is getting into the weeds of the
> > > > > implementation,
> > > > > > > >>>> and
> > > > > > > >>>>>>>> you'd
> > > > > > > >>>>>>>>>>> rather we focus on the API for now, but I think
> it's
> > > > > important
> > > > > > > to
> > > > > > > >>>>>>>>>> consider
> > > > > > > >>>>>>>>>>> how to implement the desired API, in case we come
> up
> > > with
> > > > > an
> > > > > > > API
> > > > > > > >>>>>>>> that
> > > > > > > >>>>>>>>>>> cannot be implemented efficiently, or even at all!
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> Thoughts?
> > > > > > > >>>>>>>>>>> --
> > > > > > > >>>>>>>>>>> Nick
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> > > > > > > cadonna@apache.org
> > > > > > > >>>
> > > > > > > >>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> Hi Nick,
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> 6.
> > > > > > > >>>>>>>>>>>> Of course, you are right! My bad!
> > > > > > > >>>>>>>>>>>> Wiping out the state in the downgrading case is
> fine.
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> 3a.
> > > > > > > >>>>>>>>>>>> Focus on the public facing changes for the KIP. We
> > > will
> > > > > manage
> > > > > > > >> 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.
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> Best,
> > > > > > > >>>>>>>>>>>> Bruno
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > > > > > > >>>>>>>>>>>>> Hi Bruno,
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> Thanks for getting back to me!
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> 2.
> > > > > > > >>>>>>>>>>>>> The fact that implementations can always track
> > > estimated
> > > > > > > memory
> > > > > > > >>>>>>>> usage
> > > > > > > >>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as
> an
> > > > > option,
> > > > > > > and
> > > > > > > >>>>>>>> I'll
> > > > > > > >>>>>>>>>>>> clarify
> > > > > > > >>>>>>>>>>>>> the JavaDoc that 0 is not just for
> non-transactional
> > > > > stores,
> > > > > > > >>>>>>>> which is
> > > > > > > >>>>>>>>>>>>> currently misleading.
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> 6.
> > > > > > > >>>>>>>>>>>>> The problem with catching the exception in the
> > > downgrade
> > > > > > > >> process
> > > > > > > >>>>>>>> is
> > > > > > > >>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>> would require new code in the Kafka version being
> > > > > downgraded
> > > > > > > >> to.
> > > > > > > >>>>>>>> Since
> > > > > > > >>>>>>>>>>>>> users could conceivably downgrade to almost *any*
> > > older
> > > > > > > version
> > > > > > > >>>>>>>> of
> > > > > > > >>>>>>>>>> Kafka
> > > > > > > >>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
> > > > > > > >>>>>>>>>>>>> The only way I can think of doing it would be to
> > > provide
> > > > > a
> > > > > > > >>>>>>>> dedicated
> > > > > > > >>>>>>>>>>>>> downgrade tool, that goes through every local
> store
> > > and
> > > > > > > removes
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>>> offsets column families. But that seems like an
> > > > > unnecessary
> > > > > > > >>>>>>>> amount of
> > > > > > > >>>>>>>>>>>> extra
> > > > > > > >>>>>>>>>>>>> code to maintain just to handle a somewhat niche
> > > > > situation,
> > > > > > > >> when
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>>> alternative (automatically wipe and restore
> stores)
> > > > > should be
> > > > > > > >>>>>>>>>>>> acceptable.
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've
> > > requested.
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> 3a.
> > > > > > > >>>>>>>>>>>>> I agree that IsolationLevel makes more sense at
> > > > > query-time,
> > > > > > > and
> > > > > > > >>>> I
> > > > > > > >>>>>>>>>>>> actually
> > > > > > > >>>>>>>>>>>>> initially attempted to place the IsolationLevel
> at
> > > > > > > query-time,
> > > > > > > >>>>>>>> but I
> > > > > > > >>>>>>>>>> ran
> > > > > > > >>>>>>>>>>>>> into some problems:
> > > > > > > >>>>>>>>>>>>> - The key issue is that, under ALOS we're not
> staging
> > > > > writes
> > > > > > > in
> > > > > > > >>>>>>>>>>>>> transactions, so can't perform writes at the
> > > > > READ_COMMITTED
> > > > > > > >>>>>>>> isolation
> > > > > > > >>>>>>>>>>>>> level. However, this may be addressed if we
> decide to
> > > > > > > *always*
> > > > > > > >>>>>>>> use
> > > > > > > >>>>>>>>>>>>> transactions as discussed under 3b.
> > > > > > > >>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> > > implementations. I
> > > > > > > >> remember
> > > > > > > >>>>>>>>>> having
> > > > > > > >>>>>>>>>>>>> some difficulty understanding the IQv1 internals,
> > > which
> > > > > made
> > > > > > > it
> > > > > > > >>>>>>>>>>>> difficult
> > > > > > > >>>>>>>>>>>>> to determine what needed to be changed. However,
> I
> > > > > *think*
> > > > > > > this
> > > > > > > >>>>>>>> can be
> > > > > > > >>>>>>>>>>>>> addressed for both implementations by wrapping
> the
> > > > > > > RocksDBStore
> > > > > > > >>>>>>>> in an
> > > > > > > >>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides
> read
> > > > > methods
> > > > > > > >>>>>>>> (get,
> > > > > > > >>>>>>>>>>>> etc.)
> > > > > > > >>>>>>>>>>>>> to either read directly from the database or
> from the
> > > > > ongoing
> > > > > > > >>>>>>>>>>>> transaction.
> > > > > > > >>>>>>>>>>>>> But IQv1 might still be difficult.
> > > > > > > >>>>>>>>>>>>> - If IsolationLevel becomes a query constraint,
> then
> > > all
> > > > > > > other
> > > > > > > >>>>>>>>>>>> StateStores
> > > > > > > >>>>>>>>>>>>> will need to respect it, including the in-memory
> > > stores.
> > > > > This
> > > > > > > >>>>>>>> would
> > > > > > > >>>>>>>>>>>> require
> > > > > > > >>>>>>>>>>>>> us to adapt in-memory stores to stage their
> writes so
> > > > > they
> > > > > > > can
> > > > > > > >>>> be
> > > > > > > >>>>>>>>>>>> isolated
> > > > > > > >>>>>>>>>>>>> from READ_COMMITTTED queries. It would also
> become an
> > > > > > > important
> > > > > > > >>>>>>>>>>>>> consideration for third-party stores on upgrade,
> as
> > > > > without
> > > > > > > >>>>>>>> changes,
> > > > > > > >>>>>>>>>>>> they
> > > > > > > >>>>>>>>>>>>> would not support READ_COMMITTED queries
> correctly.
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> Ultimately, I may need some help making the
> necessary
> > > > > change
> > > > > > > to
> > > > > > > >>>>>>>> IQv1
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>> support this, but I don't think it's
> fundamentally
> > > > > > > impossible,
> > > > > > > >>>>>>>> if we
> > > > > > > >>>>>>>>>>>> want
> > > > > > > >>>>>>>>>>>>> to pursue this route.
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> 3b.
> > > > > > > >>>>>>>>>>>>> The main reason I chose to keep ALOS
> un-transactional
> > > > > was to
> > > > > > > >>>>>>>> minimize
> > > > > > > >>>>>>>>>>>>> behavioural change for most users (I believe most
> > > Streams
> > > > > > > users
> > > > > > > >>>>>>>> use
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>> default configuration, which is ALOS). That said,
> > > it's
> > > > > clear
> > > > > > > >>>>>>>> that if
> > > > > > > >>>>>>>>>>>> ALOS
> > > > > > > >>>>>>>>>>>>> also used transactional stores, the only change
> in
> > > > > behaviour
> > > > > > > >>>>>>>> would be
> > > > > > > >>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>> it would become *more correct*, which could be
> > > > > considered a
> > > > > > > >> "bug
> > > > > > > >>>>>>>> fix"
> > > > > > > >>>>>>>>>> by
> > > > > > > >>>>>>>>>>>>> users, rather than a change they need to handle.
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> I believe that performance using transactions
> (aka.
> > > > > RocksDB
> > > > > > > >>>>>>>>>>>> WriteBatches)
> > > > > > > >>>>>>>>>>>>> should actually be *better* than the un-batched
> > > > > write-path
> > > > > > > that
> > > > > > > >>>>>>>> is
> > > > > > > >>>>>>>>>>>>> currently used[1]. The only "performance"
> > > consideration
> > > > > will
> > > > > > > be
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>>> increased memory usage that transactions require.
> > > Given
> > > > > the
> > > > > > > >>>>>>>>>> mitigations
> > > > > > > >>>>>>>>>>>> for
> > > > > > > >>>>>>>>>>>>> this memory that we have in place, I would expect
> > > that
> > > > > this
> > > > > > > is
> > > > > > > >>>>>>>> not a
> > > > > > > >>>>>>>>>>>>> problem for most users.
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> If we're happy to do so, we can make ALOS also
> use
> > > > > > > >> transactions.
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> Regards,
> > > > > > > >>>>>>>>>>>>> Nick
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> Link 1:
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>
> > > > > > > >>>>>
> > > > > > > >>
> > > > >
> > >
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> > > > > > > >>>> cadonna@apache.org
> > > > > > > >>>>>>>>>
> > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> Hi Nick,
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> Thanks for the updates and sorry for the delay
> on my
> > > > > side!
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> 1.
> > > > > > > >>>>>>>>>>>>>> Making the default implementation for flush() a
> > > no-op
> > > > > sounds
> > > > > > > >>>>>>>> good to
> > > > > > > >>>>>>>>>>>> me.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> 2.
> > > > > > > >>>>>>>>>>>>>> I think what was bugging me here is that a
> > > third-party
> > > > > state
> > > > > > > >>>>>>>> store
> > > > > > > >>>>>>>>>>>> needs
> > > > > > > >>>>>>>>>>>>>> to implement the state store interface. That
> means
> > > they
> > > > > need
> > > > > > > >> to
> > > > > > > >>>>>>>>>>>>>> implement a wrapper around the actual state
> store
> > > as we
> > > > > do
> > > > > > > for
> > > > > > > >>>>>>>>>> RocksDB
> > > > > > > >>>>>>>>>>>>>> with RocksDBStore. So, a third-party state
> store can
> > > > > always
> > > > > > > >>>>>>>> estimate
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>> uncommitted bytes, if it wants, because the
> wrapper
> > > can
> > > > > > > record
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>> added
> > > > > > > >>>>>>>>>>>>>> bytes.
> > > > > > > >>>>>>>>>>>>>> One case I can think of where returning -1 makes
> > > sense
> > > > > is
> > > > > > > when
> > > > > > > >>>>>>>>>> Streams
> > > > > > > >>>>>>>>>>>>>> does not need to estimate the size of the write
> > > batch
> > > > > and
> > > > > > > >>>>>>>> trigger
> > > > > > > >>>>>>>>>>>>>> extraordinary commits, because the third-party
> state
> > > > > store
> > > > > > > >>>>>>>> takes care
> > > > > > > >>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>> memory. But in that case the method could also
> just
> > > > > return
> > > > > > > 0.
> > > > > > > >>>>>>>> Even
> > > > > > > >>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>> case would be better solved with a method that
> > > returns
> > > > > > > whether
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>> state
> > > > > > > >>>>>>>>>>>>>> store manages itself the memory used for
> uncommitted
> > > > > bytes
> > > > > > > or
> > > > > > > >>>>>>>> not.
> > > > > > > >>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return
> > > value,
> > > > > I was
> > > > > > > >>>>>>>> just
> > > > > > > >>>>>>>>>>>>>> wondering when and if it will be used.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> Regarding returning 0 for transactional state
> stores
> > > > > when
> > > > > > > the
> > > > > > > >>>>>>>> batch
> > > > > > > >>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>> empty, I was just wondering because you
> explicitly
> > > > > stated
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> "or {@code 0} if this StateStore does not
> support
> > > > > > > >>>> transactions."
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> So it seemed to me returning 0 could only
> happen for
> > > > > > > >>>>>>>>>> non-transactional
> > > > > > > >>>>>>>>>>>>>> state stores.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> 3.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> a) What do you think if we move the isolation
> level
> > > to
> > > > > IQ
> > > > > > > (v1
> > > > > > > >>>>>>>> and
> > > > > > > >>>>>>>>>> v2)?
> > > > > > > >>>>>>>>>>>>>> In the end this is the only component that
> really
> > > needs
> > > > > to
> > > > > > > >>>>>>>> specify
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>> isolation level. It is similar to the Kafka
> consumer
> > > > > that
> > > > > > > can
> > > > > > > >>>>>>>> choose
> > > > > > > >>>>>>>>>>>>>> with what isolation level to read the input
> topic.
> > > > > > > >>>>>>>>>>>>>> For IQv1 the isolation level should go into
> > > > > > > >>>>>>>> StoreQueryParameters. For
> > > > > > > >>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> b) Point a) raises the question what should
> happen
> > > > > during
> > > > > > > >>>>>>>>>> at-least-once
> > > > > > > >>>>>>>>>>>>>> processing when the state store does not use
> > > > > transactions?
> > > > > > > >> John
> > > > > > > >>>>>>>> in
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>> past proposed to also use transactions on state
> > > stores
> > > > > for
> > > > > > > >>>>>>>>>>>>>> at-least-once. I like that idea, because it
> avoids
> > > > > > > aggregating
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>> same
> > > > > > > >>>>>>>>>>>>>> records over and over again in the case of a
> > > failure. We
> > > > > > > had a
> > > > > > > >>>>>>>> case
> > > > > > > >>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>> the past where a Streams applications in
> > > at-least-once
> > > > > mode
> > > > > > > >> was
> > > > > > > >>>>>>>>>> failing
> > > > > > > >>>>>>>>>>>>>> continuously for some reasons I do not remember
> > > before
> > > > > > > >>>>>>>> committing the
> > > > > > > >>>>>>>>>>>>>> offsets. After each failover, the app aggregated
> > > again
> > > > > and
> > > > > > > >>>>>>>> again the
> > > > > > > >>>>>>>>>>>>>> same records. Of course the aggregate increased
> to
> > > very
> > > > > > > wrong
> > > > > > > >>>>>>>> values
> > > > > > > >>>>>>>>>>>>>> just because of the failover. With transactions
> on
> > > the
> > > > > state
> > > > > > > >>>>>>>> stores
> > > > > > > >>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>> could have avoided this. The app would have
> output
> > > the
> > > > > same
> > > > > > > >>>>>>>> aggregate
> > > > > > > >>>>>>>>>>>>>> multiple times (i.e., after each failover) but
> at
> > > least
> > > > > the
> > > > > > > >>>>>>>> value of
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>> aggregate would not depend on the number of
> > > failovers.
> > > > > > > >>>>>>>> Outputting the
> > > > > > > >>>>>>>>>>>>>> same aggregate multiple times would be incorrect
> > > under
> > > > > > > >>>>>>>> exactly-once
> > > > > > > >>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>> it is OK for at-least-once.
> > > > > > > >>>>>>>>>>>>>> If it makes sense to add a config to turn on
> and off
> > > > > > > >>>>>>>> transactions on
> > > > > > > >>>>>>>>>>>>>> state stores under at-least-once or just use
> > > > > transactions in
> > > > > > > >>>>>>>> any case
> > > > > > > >>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>> a question we should also discuss in this KIP.
> It
> > > > > depends a
> > > > > > > >> bit
> > > > > > > >>>>>>>> on
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would
> > > add a
> > > > > > > config.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> 4.
> > > > > > > >>>>>>>>>>>>>> Your points are all valid. I tend to say to
> keep the
> > > > > metrics
> > > > > > > >>>>>>>> around
> > > > > > > >>>>>>>>>>>>>> flush() until we remove flush() completely from
> the
> > > > > > > interface.
> > > > > > > >>>>>>>> Calls
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>> flush() might still exist since existing
> processors
> > > > > might
> > > > > > > >> still
> > > > > > > >>>>>>>> call
> > > > > > > >>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For
> > > sure, we
> > > > > need
> > > > > > > >> to
> > > > > > > >>>>>>>>>>>> document
> > > > > > > >>>>>>>>>>>>>> how the metrics change due to the transactions
> in
> > > the
> > > > > > > upgrade
> > > > > > > >>>>>>>> notes.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> 5.
> > > > > > > >>>>>>>>>>>>>> I see. Then you should describe how the
> .position
> > > files
> > > > > are
> > > > > > > >>>>>>>> handled
> > > > > > > >>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>> a dedicated section of the KIP or incorporate
> the
> > > > > > > description
> > > > > > > >>>>>>>> in the
> > > > > > > >>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only
> > > > > mentioning it
> > > > > > > >> in
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>>>> "Compatibility, 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:
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>
> > > > > > > >>>>>
> > > > > > > >>>>
> > > > > > > >>
> > > > > > >
> > > > >
> > >
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> It is a bit brittle, but it works.
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> Best,
> > > > > > > >>>>>>>>>>>>>> Bruno
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > > > > > > >>>>>>>>>>>>>>> Hi Bruno,
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> Thanks for taking the time to review the KIP.
> I'm
> > > back
> > > > > from
> > > > > > > >>>>>>>> leave
> > > > > > > >>>>>>>>>> now
> > > > > > > >>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>> intend to move this forwards as quickly as I
> can.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> Addressing your points:
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> 1.
> > > > > > > >>>>>>>>>>>>>>> Because flush() is part of the StateStore API,
> it's
> > > > > exposed
> > > > > > > >> to
> > > > > > > >>>>>>>>>> custom
> > > > > > > >>>>>>>>>>>>>>> Processors, which might be making calls to
> flush().
> > > > > This
> > > > > > > was
> > > > > > > >>>>>>>>>> actually
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> case in a few integration tests.
> > > > > > > >>>>>>>>>>>>>>> To maintain as much compatibility as possible,
> I'd
> > > > > prefer
> > > > > > > not
> > > > > > > >>>>>>>> to
> > > > > > > >>>>>>>>>> make
> > > > > > > >>>>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>> an UnsupportedOperationException, as it will
> cause
> > > > > > > previously
> > > > > > > >>>>>>>>>> working
> > > > > > > >>>>>>>>>>>>>>> Processors to start throwing exceptions at
> runtime.
> > > > > > > >>>>>>>>>>>>>>> I agree that it doesn't make sense for it to
> proxy
> > > > > > > commit(),
> > > > > > > >>>>>>>> though,
> > > > > > > >>>>>>>>>>>> as
> > > > > > > >>>>>>>>>>>>>>> that would cause it to violate the "StateStores
> > > commit
> > > > > only
> > > > > > > >>>>>>>> when the
> > > > > > > >>>>>>>>>>>> Task
> > > > > > > >>>>>>>>>>>>>>> commits" rule.
> > > > > > > >>>>>>>>>>>>>>> Instead, I think we should make this a no-op.
> That
> > > way,
> > > > > > > >>>>>>>> existing
> > > > > > > >>>>>>>>>> user
> > > > > > > >>>>>>>>>>>>>>> Processors will continue to work as-before,
> without
> > > > > > > violation
> > > > > > > >>>>>>>> of
> > > > > > > >>>>>>>>>> store
> > > > > > > >>>>>>>>>>>>>>> consistency that would be caused by premature
> > > > > flush/commit
> > > > > > > of
> > > > > > > >>>>>>>>>>>> StateStore
> > > > > > > >>>>>>>>>>>>>>> data to disk.
> > > > > > > >>>>>>>>>>>>>>> What do you think?
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> 2.
> > > > > > > >>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
> > > > > implementation
> > > > > > > is
> > > > > > > >>>>>>>>>>>>>>> transactional, but is unable to estimate the
> > > > > uncommitted
> > > > > > > >>>> memory
> > > > > > > >>>>>>>>>> usage,
> > > > > > > >>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> method will return -1.
> > > > > > > >>>>>>>>>>>>>>> The intention here is to permit third-party
> > > > > implementations
> > > > > > > >>>>>>>> that may
> > > > > > > >>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>> able to estimate memory usage.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been
> written to
> > > the
> > > > > > > store
> > > > > > > >>>>>>>> yet. I
> > > > > > > >>>>>>>>>>>>>> thought
> > > > > > > >>>>>>>>>>>>>>> that was implied by "This method will return an
> > > > > > > approximation
> > > > > > > >>>>>>>> of the
> > > > > > > >>>>>>>>>>>>>> memory
> > > > > > > >>>>>>>>>>>>>>> would be freed by the next call to {@link
> > > > > #commit(Map)}"
> > > > > > > and
> > > > > > > >>>>>>>>>> "@return
> > > > > > > >>>>>>>>>>>> The
> > > > > > > >>>>>>>>>>>>>>> approximate size of all records awaiting {@link
> > > > > > > >>>> #commit(Map)}",
> > > > > > > >>>>>>>>>>>> however,
> > > > > > > >>>>>>>>>>>>>> I
> > > > > > > >>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you
> think
> > > this
> > > > > is
> > > > > > > >>>>>>>> unclear?
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> 3.
> > > > > > > >>>>>>>>>>>>>>> I realise this is probably the most contentious
> > > point
> > > > > in my
> > > > > > > >>>>>>>> design,
> > > > > > > >>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>> I'm
> > > > > > > >>>>>>>>>>>>>>> open to changing it if I'm unable to convince
> you
> > > of
> > > > > the
> > > > > > > >>>>>>>> benefits.
> > > > > > > >>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > > > > > > >>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly
> > > provided
> > > > > > > >>>>>>>> StateStores
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>> query, and it may be important for users to
> > > > > > > programmatically
> > > > > > > >>>>>>>> know
> > > > > > > >>>>>>>>>>>> which
> > > > > > > >>>>>>>>>>>>>>> mode the StateStore is operating under. If we
> > > simply
> > > > > > > provide
> > > > > > > >>>> an
> > > > > > > >>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the
> > > internal
> > > > > > > streams
> > > > > > > >>>>>>>>>>>> engine), or
> > > > > > > >>>>>>>>>>>>>>> similar, then users will need to understand the
> > > > > operation
> > > > > > > and
> > > > > > > >>>>>>>>>>>>>> consequences
> > > > > > > >>>>>>>>>>>>>>> of each available processing mode and how it
> > > pertains
> > > > > to
> > > > > > > >> their
> > > > > > > >>>>>>>>>>>>>> StateStore.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> Interactive Query users aren't the only people
> that
> > > > > care
> > > > > > > >> about
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
> > > > > > > implementers
> > > > > > > >>>> of
> > > > > > > >>>>>>>>>> custom
> > > > > > > >>>>>>>>>>>>>>> StateStores also need to understand the
> behaviour
> > > > > expected
> > > > > > > of
> > > > > > > >>>>>>>> their
> > > > > > > >>>>>>>>>>>>>>> implementation. KIP-892 introduces some
> assumptions
> > > > > into
> > > > > > > the
> > > > > > > >>>>>>>> Streams
> > > > > > > >>>>>>>>>>>>>> Engine
> > > > > > > >>>>>>>>>>>>>>> about how StateStores operate under each
> processing
> > > > > mode,
> > > > > > > and
> > > > > > > >>>>>>>> it's
> > > > > > > >>>>>>>>>>>>>>> important that custom implementations adhere to
> > > those
> > > > > > > >>>>>>>> assumptions in
> > > > > > > >>>>>>>>>>>>>> order
> > > > > > > >>>>>>>>>>>>>>> to maintain the consistency guarantees.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> IsolationLevels provide a high-level contract
> on
> > > the
> > > > > > > >> behaviour
> > > > > > > >>>>>>>> of
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>> StateStore: a user knows that under
> READ_COMMITTED,
> > > > > they
> > > > > > > will
> > > > > > > >>>>>>>> see
> > > > > > > >>>>>>>>>>>> writes
> > > > > > > >>>>>>>>>>>>>>> only after the Task has committed, and under
> > > > > > > READ_UNCOMMITTED
> > > > > > > >>>>>>>> they
> > > > > > > >>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>> see
> > > > > > > >>>>>>>>>>>>>>> writes immediately. No understanding of the
> > > details of
> > > > > each
> > > > > > > >>>>>>>>>>>>>> processing.mode
> > > > > > > >>>>>>>>>>>>>>> is required, either for IQ users or StateStore
> > > > > > > implementers.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> An argument can be made that these contractual
> > > > > guarantees
> > > > > > > can
> > > > > > > >>>>>>>> simply
> > > > > > > >>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>> documented for the processing.mode (i.e. that
> > > > > exactly-once
> > > > > > > >> and
> > > > > > > >>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
> > > > > > > at-least-once
> > > > > > > >>>>>>>> behaves
> > > > > > > >>>>>>>>>>>> like
> > > > > > > >>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small
> > > issues
> > > > > with
> > > > > > > >>>>>>>> this I'd
> > > > > > > >>>>>>>>>>>>>> prefer
> > > > > > > >>>>>>>>>>>>>>> to avoid:
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>         - Where would we document these
> contracts,
> > > in
> > > > > a way
> > > > > > > >> that
> > > > > > > >>>>>>>> is
> > > > > > > >>>>>>>>>>>> difficult
> > > > > > > >>>>>>>>>>>>>>>         for users/implementers to miss/ignore?
> > > > > > > >>>>>>>>>>>>>>>         - It's not clear to users that the
> > > processing
> > > > > mode
> > > > > > > is
> > > > > > > >>>>>>>>>>>> communicating
> > > > > > > >>>>>>>>>>>>>>>         an expectation of read isolation,
> unless
> > > they
> > > > > read
> > > > > > > the
> > > > > > > >>>>>>>>>>>>>> documentation. Users
> > > > > > > >>>>>>>>>>>>>>>         rarely consult documentation unless
> they
> > > feel
> > > > > they
> > > > > > > >> need
> > > > > > > >>>>>>>> to, so
> > > > > > > >>>>>>>>>>>> it's
> > > > > > > >>>>>>>>>>>>>> likely
> > > > > > > >>>>>>>>>>>>>>>         this detail would get missed by many
> users.
> > > > > > > >>>>>>>>>>>>>>>         - It tightly couples processing modes
> to
> > > read
> > > > > > > >> isolation.
> > > > > > > >>>>>>>> Adding
> > > > > > > >>>>>>>>>>>> new
> > > > > > > >>>>>>>>>>>>>>>         processing modes, or changing the read
> > > > > isolation of
> > > > > > > >>>>>>>> existing
> > > > > > > >>>>>>>>>>>>>> processing
> > > > > > > >>>>>>>>>>>>>>>         modes would be difficult/impossible.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> Ultimately, the cost of introducing
> > > IsolationLevels is
> > > > > > > just a
> > > > > > > >>>>>>>> single
> > > > > > > >>>>>>>>>>>>>>> method, since we re-use the existing
> IsolationLevel
> > > > > enum
> > > > > > > from
> > > > > > > >>>>>>>> Kafka.
> > > > > > > >>>>>>>>>>>> This
> > > > > > > >>>>>>>>>>>>>>> gives us a clear place to document the
> contractual
> > > > > > > guarantees
> > > > > > > >>>>>>>>>> expected
> > > > > > > >>>>>>>>>>>>>>> of/provided by StateStores, that is accessible
> > > both by
> > > > > the
> > > > > > > >>>>>>>>>> StateStore
> > > > > > > >>>>>>>>>>>>>>> itself, and by IQ users.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> (Writing this I've just realised that the
> > > StateStore
> > > > > and IQ
> > > > > > > >>>>>>>> APIs
> > > > > > > >>>>>>>>>>>> actually
> > > > > > > >>>>>>>>>>>>>>> don't provide access to StateStoreContext that
> IQ
> > > users
> > > > > > > would
> > > > > > > >>>>>>>> have
> > > > > > > >>>>>>>>>>>> direct
> > > > > > > >>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> > > > > > > >> isolationLevel()
> > > > > > > >>>>>>>>>> itself
> > > > > > > >>>>>>>>>>>>>> too?)
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> 4.
> > > > > > > >>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics
> > > in-place
> > > > > > > >>>>>>>> either, as
> > > > > > > >>>>>>>>>>>> it's a
> > > > > > > >>>>>>>>>>>>>>> backwards incompatible change. My concern is
> that,
> > > if
> > > > > we
> > > > > > > >> leave
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>>>> existing
> > > > > > > >>>>>>>>>>>>>>> "flush" metrics in place, they will be
> confusing to
> > > > > users.
> > > > > > > >>>>>>>> Right
> > > > > > > >>>>>>>>>> now,
> > > > > > > >>>>>>>>>>>>>>> "flush" metrics record explicit flushes to
> disk,
> > > but
> > > > > under
> > > > > > > >>>>>>>> KIP-892,
> > > > > > > >>>>>>>>>>>> even
> > > > > > > >>>>>>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>> commit() will not explicitly flush data to
> disk -
> > > > > RocksDB
> > > > > > > >> will
> > > > > > > >>>>>>>>>> decide
> > > > > > > >>>>>>>>>>>> on
> > > > > > > >>>>>>>>>>>>>>> when to flush memtables to disk itself.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd
> have
> > > two
> > > > > > > >>>> options,
> > > > > > > >>>>>>>>>> which
> > > > > > > >>>>>>>>>>>>>> both
> > > > > > > >>>>>>>>>>>>>>> seem pretty bad to me:
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>         1. Have them record calls to commit(),
> > > which
> > > > > would
> > > > > > > be
> > > > > > > >>>>>>>>>>>> misleading, as
> > > > > > > >>>>>>>>>>>>>>>         data is no longer explicitly "flushed"
> to
> > > disk
> > > > > by
> > > > > > > this
> > > > > > > >>>>>>>> call.
> > > > > > > >>>>>>>>>>>>>>>         2. Have them record nothing at all,
> which
> > > is
> > > > > > > >> equivalent
> > > > > > > >>>> to
> > > > > > > >>>>>>>>>>>> removing
> > > > > > > >>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>         metrics, except that users will see the
> > > metric
> > > > > > > still
> > > > > > > >>>>>>>> exists and
> > > > > > > >>>>>>>>>>>> so
> > > > > > > >>>>>>>>>>>>>> assume
> > > > > > > >>>>>>>>>>>>>>>         that the metric is correct, and that
> > > there's a
> > > > > > > problem
> > > > > > > >>>>>>>> with
> > > > > > > >>>>>>>>>> their
> > > > > > > >>>>>>>>>>>>>> system
> > > > > > > >>>>>>>>>>>>>>>         when there isn't.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> I agree that removing them is also a bad
> solution,
> > > and
> > > > > I'd
> > > > > > > >>>>>>>> like some
> > > > > > > >>>>>>>>>>>>>>> guidance on the best path forward here.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> 5.
> > > > > > > >>>>>>>>>>>>>>> Position files are updated on every write to a
> > > > > StateStore.
> > > > > > > >>>>>>>> Since our
> > > > > > > >>>>>>>>>>>>>> writes
> > > > > > > >>>>>>>>>>>>>>> are now buffered until commit(), we can't
> update
> > > the
> > > > > > > Position
> > > > > > > >>>>>>>> file
> > > > > > > >>>>>>>>>>>> until
> > > > > > > >>>>>>>>>>>>>>> commit() has been called, otherwise it would be
> > > > > > > inconsistent
> > > > > > > >>>>>>>> with
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>> data
> > > > > > > >>>>>>>>>>>>>>> in the event of a rollback. Consequently, we
> need
> > > to
> > > > > manage
> > > > > > > >>>>>>>> these
> > > > > > > >>>>>>>>>>>> offsets
> > > > > > > >>>>>>>>>>>>>>> the same way we manage the checkpoint offsets,
> and
> > > > > ensure
> > > > > > > >>>>>>>> they're
> > > > > > > >>>>>>>>>> only
> > > > > > > >>>>>>>>>>>>>>> written on commit().
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> 6.
> > > > > > > >>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what
> > > tests to
> > > > > > > >> write.
> > > > > > > >>>>>>>> How
> > > > > > > >>>>>>>>>>>>>> explicit
> > > > > > > >>>>>>>>>>>>>>> do we need to be here in the KIP?
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed
> to be
> > > > > > > seamless,
> > > > > > > >>>>>>>> and we
> > > > > > > >>>>>>>>>>>>>> should
> > > > > > > >>>>>>>>>>>>>>> definitely add some tests around that.
> Downgrade,
> > > it
> > > > > > > >>>>>>>> transpires,
> > > > > > > >>>>>>>>>> isn't
> > > > > > > >>>>>>>>>>>>>>> currently possible, as the extra column family
> for
> > > > > offset
> > > > > > > >>>>>>>> storage is
> > > > > > > >>>>>>>>>>>>>>> incompatible with the pre-KIP-892
> implementation:
> > > when
> > > > > you
> > > > > > > >>>>>>>> open a
> > > > > > > >>>>>>>>>>>> RocksDB
> > > > > > > >>>>>>>>>>>>>>> database, you must open all available column
> > > families
> > > > > or
> > > > > > > >>>>>>>> receive an
> > > > > > > >>>>>>>>>>>>>> error.
> > > > > > > >>>>>>>>>>>>>>> What currently happens on downgrade is that it
> > > > > attempts to
> > > > > > > >>>>>>>> open the
> > > > > > > >>>>>>>>>>>>>> store,
> > > > > > > >>>>>>>>>>>>>>> throws an error about the offsets column
> family not
> > > > > being
> > > > > > > >>>>>>>> opened,
> > > > > > > >>>>>>>>>>>> which
> > > > > > > >>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given
> that
> > > > > > > >> downgrades
> > > > > > > >>>>>>>>>> should
> > > > > > > >>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>> uncommon, I think this is acceptable
> behaviour, as
> > > the
> > > > > > > >>>>>>>> end-state is
> > > > > > > >>>>>>>>>>>>>>> consistent, even if it results in an
> undesirable
> > > state
> > > > > > > >>>> restore.
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> Should I document the upgrade/downgrade
> behaviour
> > > > > > > explicitly
> > > > > > > >>>>>>>> in the
> > > > > > > >>>>>>>>>>>> KIP?
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> --
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> Regards,
> > > > > > > >>>>>>>>>>>>>>> Nick
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > > > > > > >>>>>>>> cadonna@apache.org>
> > > > > > > >>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> Hi Nick!
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> Thanks for the updates!
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> 1.
> > > > > > > >>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> > > > > > > >>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > > > > > > >>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore
> > > after
> > > > > this
> > > > > > > KIP
> > > > > > > >>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>> released, I would rather throw an unsupported
> > > > > operation
> > > > > > > >>>>>>>> exception
> > > > > > > >>>>>>>>>> by
> > > > > > > >>>>>>>>>>>>>>>> default.
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> 2.
> > > > > > > >>>>>>>>>>>>>>>> When would a state store return -1 from
> > > > > > > >>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
> while
> > > > > being
> > > > > > > >>>>>>>>>>>> transactional?
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> Wouldn't
> > > StateStore#approximateNumUncommittedBytes()
> > > > > also
> > > > > > > >>>>>>>> return 0
> > > > > > > >>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>> the state store is transactional but nothing
> has
> > > been
> > > > > > > >> written
> > > > > > > >>>>>>>> to
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>> state store yet?
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> 3.
> > > > > > > >>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this
> KIP
> > > really
> > > > > > > need
> > > > > > > >>>> to
> > > > > > > >>>>>>>>>>>> introduce
> > > > > > > >>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> > > StateStoreContext
> > > > > has
> > > > > > > >>>>>>>> already
> > > > > > > >>>>>>>>>>>>>>>> appConfigs() which basically exposes the same
> > > > > information,
> > > > > > > >>>>>>>> i.e., if
> > > > > > > >>>>>>>>>>>> EOS
> > > > > > > >>>>>>>>>>>>>>>> is enabled or not.
> > > > > > > >>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore
> > > interface
> > > > > as
> > > > > > > >>>>>>>> loosely
> > > > > > > >>>>>>>>>>>> coupled
> > > > > > > >>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > > > > implementers
> > > > > > > >>>> more
> > > > > > > >>>>>>>>>>>> freedom,
> > > > > > > >>>>>>>>>>>>>>>> and reduce the amount of internal knowledge
> > > required."
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> While I understand the intent, I doubt that it
> > > > > decreases
> > > > > > > the
> > > > > > > >>>>>>>>>>>> coupling of
> > > > > > > >>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> > > > > > > >> READ_COMMITTED
> > > > > > > >>>>>>>> only
> > > > > > > >>>>>>>>>>>>>>>> applies to IQ but not to reads by processors.
> > > Thus,
> > > > > > > >>>>>>>> implementers
> > > > > > > >>>>>>>>>>>> need to
> > > > > > > >>>>>>>>>>>>>>>> understand how Streams accesses the state
> stores.
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> I would like to hear what others think about
> this.
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> 4.
> > > > > > > >>>>>>>>>>>>>>>> Great exposing new metrics for transactional
> state
> > > > > stores!
> > > > > > > >>>>>>>>>> However, I
> > > > > > > >>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate
> (in
> > > the
> > > > > > > docs)
> > > > > > > >>>>>>>> the 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 also need to adapt/add
> integration
> > > > > tests
> > > > > > > >>>>>>>> besides
> > > > > > > >>>>>>>>>> unit
> > > > > > > >>>>>>>>>>>>>>>> tests. Additionally, we probably need
> integration
> > > or
> > > > > > > system
> > > > > > > >>>>>>>> tests
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>> verify that upgrades and downgrades between
> > > > > transactional
> > > > > > > >> and
> > > > > > > >>>>>>>>>>>>>>>> non-transactional state stores work as
> expected.
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> Best,
> > > > > > > >>>>>>>>>>>>>>>> Bruno
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > > > > > > >>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in
> the
> > > KIP,
> > > > > > > under
> > > > > > > >>>>>>>>>>>> "Rejected
> > > > > > > >>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea
> worth
> > > > > pursuing,
> > > > > > > >>>>>>>> but I
> > > > > > > >>>>>>>>>>>> believe
> > > > > > > >>>>>>>>>>>>>>>>> that it's out of the scope of this KIP,
> because
> > > it
> > > > > > > solves a
> > > > > > > >>>>>>>>>>>> different
> > > > > > > >>>>>>>>>>>>>> set
> > > > > > > >>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of
> this
> > > one
> > > > > has
> > > > > > > >>>>>>>> already
> > > > > > > >>>>>>>>>> grown
> > > > > > > >>>>>>>>>>>>>>>> quite
> > > > > > > >>>>>>>>>>>>>>>>> large!
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > > > > > > >>>>>>>>>> nick.telford@gmail.com>
> > > > > > > >>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> Hi everyone,
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> I've updated the KIP (
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>
> > > > > > > >>>>>
> > > > > > > >>>>
> > > > > > > >>
> > > > > > >
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > > > > >>>>>>>>>>>>>>>> )
> > > > > > > >>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing
> back
> > > > > "Atomic
> > > > > > > >>>>>>>>>>>> Checkpointing"
> > > > > > > >>>>>>>>>>>>>>>> (for
> > > > > > > >>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think
> the one
> > > > > thing
> > > > > > > >>>>>>>> missing is
> > > > > > > >>>>>>>>>>>> some
> > > > > > > >>>>>>>>>>>>>>>>>> changes to metrics (notably the store
> "flush"
> > > > > metrics
> > > > > > > will
> > > > > > > >>>>>>>> need
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>> renamed to "commit").
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> The reason I brought back Atomic
> Checkpointing
> > > was
> > > > > to
> > > > > > > >>>>>>>> decouple
> > > > > > > >>>>>>>>>>>> store
> > > > > > > >>>>>>>>>>>>>>>> flush
> > > > > > > >>>>>>>>>>>>>>>>>> from store commit. This is important,
> because
> > > with
> > > > > > > >>>>>>>> Transactional
> > > > > > > >>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on
> > > *every*
> > > > > Task
> > > > > > > >>>>>>>> commit,
> > > > > > > >>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>> just when the StateStore is closing,
> otherwise
> > > our
> > > > > > > >>>>>>>> transaction
> > > > > > > >>>>>>>>>>>> buffer
> > > > > > > >>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>> never be written and persisted, instead
> growing
> > > > > > > unbounded!
> > > > > > > >>>> I
> > > > > > > >>>>>>>>>>>>>>>> experimented
> > > > > > > >>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a
> store
> > > > > flush
> > > > > > > >>>>>>>> whenever
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its
> > > > > configured
> > > > > > > >>>>>>>> size, but
> > > > > > > >>>>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>> was
> > > > > > > >>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer
> > > from
> > > > > being
> > > > > > > >>>>>>>>>> configured
> > > > > > > >>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>> unbounded, and it still would have required
> > > explicit
> > > > > > > >>>>>>>> flushes of
> > > > > > > >>>>>>>>>>>>>> RocksDB,
> > > > > > > >>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory
> > > > > utilization.
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the
> "right"
> > > way
> > > > > to
> > > > > > > >>>>>>>> resolve
> > > > > > > >>>>>>>>>> this
> > > > > > > >>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog
> offsets
> > > that
> > > > > > > >>>>>>>> correspond
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>> most
> > > > > > > >>>>>>>>>>>>>>>>>> recently written records are always
> atomically
> > > > > written
> > > > > > > to
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>>>> StateStore
> > > > > > > >>>>>>>>>>>>>>>>>> (by writing them to the same transaction
> > > buffer),
> > > > > we can
> > > > > > > >>>>>>>> avoid
> > > > > > > >>>>>>>>>>>>>> forcibly
> > > > > > > >>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk,
> letting
> > > > > RocksDB
> > > > > > > >>>>>>>> flush
> > > > > > > >>>>>>>>>> them
> > > > > > > >>>>>>>>>>>>>> only
> > > > > > > >>>>>>>>>>>>>>>>>> when necessary, without losing any of our
> > > > > consistency
> > > > > > > >>>>>>>> guarantees.
> > > > > > > >>>>>>>>>>>> See
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>> updated KIP for more info.
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> I have fully implemented these changes,
> > > although I'm
> > > > > > > still
> > > > > > > >>>>>>>> not
> > > > > > > >>>>>>>>>>>>>> entirely
> > > > > > > >>>>>>>>>>>>>>>>>> happy with the implementation for segmented
> > > > > StateStores,
> > > > > > > >> so
> > > > > > > >>>>>>>> I
> > > > > > > >>>>>>>>>> plan
> > > > > > > >>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests
> pass. If
> > > > > you'd
> > > > > > > like
> > > > > > > >>>>>>>> to try
> > > > > > > >>>>>>>>>>>> out
> > > > > > > >>>>>>>>>>>>>> or
> > > > > > > >>>>>>>>>>>>>>>>>> review this highly experimental and
> incomplete
> > > > > branch,
> > > > > > > >> it's
> > > > > > > >>>>>>>>>>>> available
> > > > > > > >>>>>>>>>>>>>>>> here:
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > > > > > > .
> > > > > > > >>>>>>>> Note:
> > > > > > > >>>>>>>>>>>> it's
> > > > > > > >>>>>>>>>>>>>>>> built
> > > > > > > >>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable
> base
> > > to
> > > > > build
> > > > > > > >>>>>>>> and test
> > > > > > > >>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>> on,
> > > > > > > >>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples
> comparisons
> > > in a
> > > > > > > live
> > > > > > > >>>>>>>>>>>>>> environment. I
> > > > > > > >>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's
> nearer
> > > > > > > >> completion
> > > > > > > >>>>>>>> and
> > > > > > > >>>>>>>>>> has
> > > > > > > >>>>>>>>>>>>>> been
> > > > > > > >>>>>>>>>>>>>>>>>> proven on our main application.
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing
> and
> > > > > testing:
> > > > > > > >>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window)
> > > stores
> > > > > > > >>>>>>>>>>>>>>>>>> - Global stores
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> As I do not currently use either of these,
> so my
> > > > > primary
> > > > > > > >>>>>>>> test
> > > > > > > >>>>>>>>>>>>>>>> environment
> > > > > > > >>>>>>>>>>>>>>>>>> doesn't test these areas.
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next
> week
> > > for
> > > > > a few
> > > > > > > >>>>>>>> weeks,
> > > > > > > >>>>>>>>>> so
> > > > > > > >>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>> not have time to move this forward until
> late
> > > > > August.
> > > > > > > That
> > > > > > > >>>>>>>> said,
> > > > > > > >>>>>>>>>>>> your
> > > > > > > >>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just
> > > won't be
> > > > > > > able
> > > > > > > >>>> to
> > > > > > > >>>>>>>>>>>> respond
> > > > > > > >>>>>>>>>>>>>> as
> > > > > > > >>>>>>>>>>>>>>>>>> quickly as usual.
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> Regards,
> > > > > > > >>>>>>>>>>>>>>>>>> Nick
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > > > > > > >>>>>>>>>> nick.telford@gmail.com>
> > > > > > > >>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>> Hi Bruno
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact
> on IQ
> > > is
> > > > > not
> > > > > > > >>>>>>>> something
> > > > > > > >>>>>>>>>> I
> > > > > > > >>>>>>>>>>>> had
> > > > > > > >>>>>>>>>>>>>>>>>>> considered.
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>> What about atomically updating the state
> store
> > > > > from the
> > > > > > > >>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> the
> > > > > > > checkpoint
> > > > > > > >>>>>>>> (thus,
> > > > > > > >>>>>>>>>>>>>>>> flushing
> > > > > > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> data
> > > > > and/or
> > > > > > > >>>>>>>> number of
> > > > > > > >>>>>>>>>>>>>> commit
> > > > > > > >>>>>>>>>>>>>>>>>>>> intervals?
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you
> suggesting
> > > > > that we
> > > > > > > >>>>>>>> add an
> > > > > > > >>>>>>>>>>>>>>>> additional
> > > > > > > >>>>>>>>>>>>>>>>>>> config for the max number of commit
> intervals
> > > > > between
> > > > > > > >>>>>>>>>> checkpoints?
> > > > > > > >>>>>>>>>>>>>> That
> > > > > > > >>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the
> > > > > transaction
> > > > > > > >>>>>>>> buffers
> > > > > > > >>>>>>>>>> are
> > > > > > > >>>>>>>>>>>>>>>> nearly
> > > > > > > >>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of
> commit
> > > > > > > intervals
> > > > > > > >>>>>>>> have
> > > > > > > >>>>>>>>>>>>>> elapsed,
> > > > > > > >>>>>>>>>>>>>>>>>>> whichever comes first?
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although
> this
> > > > > > > re-ignites
> > > > > > > >>>>>>>> an
> > > > > > > >>>>>>>>>>>> earlier
> > > > > > > >>>>>>>>>>>>>>>>>>> debate about whether a config should be
> > > measured in
> > > > > > > >>>>>>>> "number of
> > > > > > > >>>>>>>>>>>> commit
> > > > > > > >>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute
> time.
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the
> reason
> > > I
> > > > > was
> > > > > > > >>>>>>>> pursuing
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>> Atomic
> > > > > > > >>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable
> flush
> > > from
> > > > > > > >>>>>>>> checkpointing,
> > > > > > > >>>>>>>>>>>>>> which
> > > > > > > >>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every
> commit
> > > > > without
> > > > > > > any
> > > > > > > >>>>>>>>>>>> performance
> > > > > > > >>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely
> the
> > > > > "best"
> > > > > > > >>>>>>>> solution,
> > > > > > > >>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>> I'm not
> > > > > > > >>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back
> into
> > > this
> > > > > KIP.
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
> > > > > transactional
> > > > > > > >>>> logic
> > > > > > > >>>>>>>>>>>> directly
> > > > > > > >>>>>>>>>>>>>>>> into
> > > > > > > >>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with
> the
> > > > > > > >>>>>>>>>>>>>> StateStore#newTransaction
> > > > > > > >>>>>>>>>>>>>>>>>>> method, and reduces the number of new
> classes
> > > > > > > introduced,
> > > > > > > >>>>>>>>>>>>>> significantly
> > > > > > > >>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and
> the
> > > > > > > complexity
> > > > > > > >>>> is
> > > > > > > >>>>>>>>>>>>>> drastically
> > > > > > > >>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic
> > > Checkpoints
> > > > > > > into
> > > > > > > >>>>>>>> this
> > > > > > > >>>>>>>>>> KIP.
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>> Regards,
> > > > > > > >>>>>>>>>>>>>>>>>>> Nick
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna
> <
> > > > > > > >>>>>>>> cadonna@apache.org>
> > > > > > > >>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> As far as I understand, you want to
> atomically
> > > > > update
> > > > > > > >> the
> > > > > > > >>>>>>>> state
> > > > > > > >>>>>>>>>>>>>> store
> > > > > > > >>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the
> > > memtable
> > > > > of a
> > > > > > > >>>> state
> > > > > > > >>>>>>>>>> store
> > > > > > > >>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit
> time
> > > > > elapsed
> > > > > > > >>>> but
> > > > > > > >>>>>>>>>> after
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that
> would
> > > lead
> > > > > to
> > > > > > > >>>>>>>> exceeding
> > > > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > before the
> > > > > > > next
> > > > > > > >>>>>>>> commit
> > > > > > > >>>>>>>>>>>>>>>> interval
> > > > > > > >>>>>>>>>>>>>>>>>>>> ends.
> > > > > > > >>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would
> commit
> > > > > every
> > > > > > > >>>>>>>> commit
> > > > > > > >>>>>>>>>>>> interval
> > > > > > > >>>>>>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>>>>>> the state store will only be atomically
> > > updated
> > > > > > > roughly
> > > > > > > >>>>>>>> every
> > > > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of
> > > data.
> > > > > Also
> > > > > > > IQ
> > > > > > > >>>>>>>> would
> > > > > > > >>>>>>>>>>>> then
> > > > > > > >>>>>>>>>>>>>>>> only
> > > > > > > >>>>>>>>>>>>>>>>>>>> see new data roughly every
> > > > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > > > > >>>>>>>>>>>>>>>>>>>> After a failure the state store needs to
> > > restore
> > > > > up to
> > > > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> Is this correct?
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> What about atomically updating the state
> store
> > > > > from
> > > > > > > the
> > > > > > > >>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing
> the
> > > > > > > checkpoint
> > > > > > > >>>>>>>> (thus,
> > > > > > > >>>>>>>>>>>>>>>> flushing
> > > > > > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of
> data
> > > > > and/or
> > > > > > > >>>>>>>> number of
> > > > > > > >>>>>>>>>>>>>> commit
> > > > > > > >>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have
> the
> > > same
> > > > > delay
> > > > > > > >>>> for
> > > > > > > >>>>>>>>>>>> records
> > > > > > > >>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because
> both
> > > > > would
> > > > > > > >>>>>>>> appear
> > > > > > > >>>>>>>>>> when
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However,
> > > after a
> > > > > > > failure
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>> state
> > > > > > > >>>>>>>>>>>>>>>> store
> > > > > > > >>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > > > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > > > > >>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>> it might restore data that is already in
> the
> > > state
> > > > > > > store
> > > > > > > >>>>>>>>>> because
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable
> offset
> > > > > (i.e.
> > > > > > > the
> > > > > > > >>>>>>>> last
> > > > > > > >>>>>>>>>>>>>> committed
> > > > > > > >>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring
> > > data
> > > > > that
> > > > > > > is
> > > > > > > >>>>>>>> already
> > > > > > > >>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should
> not
> > > > > violated.
> > > > > > > >>>>>>>>>>>>>>>>>>>> This solution needs at least one new
> config to
> > > > > specify
> > > > > > > >>>>>>>> when a
> > > > > > > >>>>>>>>>>>>>>>> checkpoint
> > > > > > > >>>>>>>>>>>>>>>>>>>> should be written.
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail
> > > that
> > > > > does
> > > > > > > not
> > > > > > > >>>>>>>> change
> > > > > > > >>>>>>>>>>>>>>>> anything
> > > > > > > >>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit
> > > interval
> > > > > is 30
> > > > > > > >>>>>>>> seconds,
> > > > > > > >>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>> five
> > > > > > > >>>>>>>>>>>>>>>>>>>> seconds.
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> Best,
> > > > > > > >>>>>>>>>>>>>>>>>>>> Bruno
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch
> on
> > > our
> > > > > > > staging
> > > > > > > >>>>>>>>>>>>>> environment,
> > > > > > > >>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
> > > non-trivial
> > > > > > > >>>>>>>> application.
> > > > > > > >>>>>>>>>> I'm
> > > > > > > >>>>>>>>>>>>>>>>>>>> already
> > > > > > > >>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate
> that
> > > we
> > > > > saw
> > > > > > > the
> > > > > > > >>>>>>>> last
> > > > > > > >>>>>>>>>>>> time
> > > > > > > >>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP,
> but
> > > this
> > > > > > > time,
> > > > > > > >> I
> > > > > > > >>>>>>>>>> think I
> > > > > > > >>>>>>>>>>>>>> know
> > > > > > > >>>>>>>>>>>>>>>>>>>> why.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit,
> which is
> > > > > called
> > > > > > > at
> > > > > > > >>>>>>>> the end
> > > > > > > >>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> Task
> > > > > > > >>>>>>>>>>>>>>>>>>>>> commit process, has the following
> behaviour:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           - Under ALOS: checkpoint the
> state
> > > > > stores.
> > > > > > > >> This
> > > > > > > >>>>>>>>>> includes
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           flushing memtables in RocksDB.
> > > This is
> > > > > > > >>>> acceptable
> > > > > > > >>>>>>>>>>>> because the
> > > > > > > >>>>>>>>>>>>>>>>>>>> default
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           commit.interval.ms is 5
> seconds,
> > > so
> > > > > > > forcibly
> > > > > > > >>>>>>>> flushing
> > > > > > > >>>>>>>>>>>>>> memtables
> > > > > > > >>>>>>>>>>>>>>>>>>>> every 5
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           seconds is acceptable for most
> > > > > > > applications.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           - Under EOS: checkpointing is
> not
> > > done,
> > > > > > > >> *unless*
> > > > > > > >>>>>>>> it's
> > > > > > > >>>>>>>>>>>> being
> > > > > > > >>>>>>>>>>>>>>>>>>>> forced, due
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           to e.g. the Task closing or
> being
> > > > > revoked.
> > > > > > > >> This
> > > > > > > >>>>>>>> means
> > > > > > > >>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>> under
> > > > > > > >>>>>>>>>>>>>>>>>>>> normal
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           processing conditions, the
> state
> > > stores
> > > > > > > will
> > > > > > > >> not
> > > > > > > >>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>> checkpointed,
> > > > > > > >>>>>>>>>>>>>>>>>>>> and will
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           not have memtables flushed at
> all ,
> > > > > unless
> > > > > > > >>>> RocksDB
> > > > > > > >>>>>>>>>>>> decides to
> > > > > > > >>>>>>>>>>>>>>>>>>>> flush them on
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           its own. Checkpointing stores
> and
> > > > > > > >> force-flushing
> > > > > > > >>>>>>>> their
> > > > > > > >>>>>>>>>>>>>> memtables
> > > > > > > >>>>>>>>>>>>>>>>>>>> is only
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           done when a Task is being
> closed.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint
> > > stores on
> > > > > at
> > > > > > > >>>> least
> > > > > > > >>>>>>>>>> *some*
> > > > > > > >>>>>>>>>>>>>>>> normal
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the
> RocksDB
> > > > > > > transaction
> > > > > > > >>>>>>>>>> buffers
> > > > > > > >>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets
> are
> > > > > synced to
> > > > > > > >>>>>>>> disk to
> > > > > > > >>>>>>>>>>>>>> prevent
> > > > > > > >>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
> > > Consequently,
> > > > > my
> > > > > > > >>>>>>>> current
> > > > > > > >>>>>>>>>>>>>>>>>>>> implementation
> > > > > > > >>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task
> commit,
> > > > > which
> > > > > > > is
> > > > > > > >>>>>>>> far too
> > > > > > > >>>>>>>>>>>>>>>>>>>> frequent.
> > > > > > > >>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000
> records,
> > > > > which
> > > > > > > is
> > > > > > > >> a
> > > > > > > >>>>>>>>>> change
> > > > > > > >>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>> flush
> > > > > > > >>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing
> performance
> > > > > problems
> > > > > > > for
> > > > > > > >>>>>>>> some
> > > > > > > >>>>>>>>>>>>>>>>>>>> applications.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and
> I'm
> > > > > > > currently
> > > > > > > >>>>>>>> leaning
> > > > > > > >>>>>>>>>>>>>>>> towards
> > > > > > > >>>>>>>>>>>>>>>>>>>>> using the
> > > statestore.transaction.buffer.max.bytes
> > > > > > > >>>>>>>>>> configuration
> > > > > > > >>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to
> > > exceed it.
> > > > > > > This
> > > > > > > >>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>> complement the
> > > > > > > >>>>>>>>>>>>>>>>>>>>> existing "early Task commit"
> functionality
> > > that
> > > > > this
> > > > > > > >>>>>>>>>>>> configuration
> > > > > > > >>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           - Currently, we use
> > > > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > > > > >>>>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>> force an
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           early Task commit if processing
> > > more
> > > > > > > records
> > > > > > > >>>> would
> > > > > > > >>>>>>>>>> cause
> > > > > > > >>>>>>>>>>>> our
> > > > > > > >>>>>>>>>>>>>>>> state
> > > > > > > >>>>>>>>>>>>>>>>>>>> store
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           transactions to exceed the
> memory
> > > > > assigned
> > > > > > > to
> > > > > > > >>>>>>>> them.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           - New functionality: when a
> Task
> > > *does*
> > > > > > > >> commit,
> > > > > > > >>>>>>>> we will
> > > > > > > >>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>>>> checkpoint
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           the stores (and hence flush the
> > > > > transaction
> > > > > > > >>>>>>>> buffers)
> > > > > > > >>>>>>>>>>>> unless
> > > > > > > >>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>> expect to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           cross the
> > > > > > > >>>> statestore.transaction.buffer.max.bytes
> > > > > > > >>>>>>>>>>>> threshold
> > > > > > > >>>>>>>>>>>>>>>> before
> > > > > > > >>>>>>>>>>>>>>>>>>>> the next
> > > > > > > >>>>>>>>>>>>>>>>>>>>>           commit
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > > > >>>>>>>>>>>>>>>>>>>>> Nick
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick
> Telford <
> > > > > > > >>>>>>>>>>>> nick.telford@gmail.com
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I
> meant in
> > > > > terms of
> > > > > > > >>>>>>>>>>>> understanding
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> performance profile under various
> > > > > circumstances. The
> > > > > > > >>>>>>>> more
> > > > > > > >>>>>>>>>>>> complex
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would
> be for
> > > > > users
> > > > > > > to
> > > > > > > >>>>>>>>>>>> understand
> > > > > > > >>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> performance they see. For example,
> spilling
> > > > > records
> > > > > > > to
> > > > > > > >>>>>>>> disk
> > > > > > > >>>>>>>>>>>> when
> > > > > > > >>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold
> > > would, I
> > > > > > > >> expect,
> > > > > > > >>>>>>>>>> reduce
> > > > > > > >>>>>>>>>>>>>> write
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
> > > throughput
> > > > > could
> > > > > > > >> be
> > > > > > > >>>>>>>>>>>>>> unexpected,
> > > > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> > > diagnose/understand for
> > > > > > > >> users.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early
> commit"
> > > > > concept is
> > > > > > > >>>>>>>>>> relatively
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document,
> and
> > > > > > > >>>> conceptually
> > > > > > > >>>>>>>>>> fairly
> > > > > > > >>>>>>>>>>>>>>>>>>>> obvious to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to
> > > make it
> > > > > > > >> easier
> > > > > > > >>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>> understand
> > > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> it happens though.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an
> > > indirect
> > > > > way
> > > > > > > of
> > > > > > > >>>>>>>>>> telling
> > > > > > > >>>>>>>>>>>>>>>>>>>> StateStores
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
> > > > > READ_COMMITTED
> > > > > > > >>>>>>>>>>>> essentially
> > > > > > > >>>>>>>>>>>>>>>>>>>> requires
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that
> two
> > > > > threads
> > > > > > > >>>>>>>> calling
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes
> > > from
> > > > > the
> > > > > > > >> other
> > > > > > > >>>>>>>>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>>>>>>>> until
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> > > > > READ_UNCOMMITTED, all
> > > > > > > >>>>>>>> bets are
> > > > > > > >>>>>>>>>>>> off,
> > > > > > > >>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe
> written
> > > > > records
> > > > > > > at
> > > > > > > >>>>>>>> any
> > > > > > > >>>>>>>>>>>> time,
> > > > > > > >>>>>>>>>>>>>>>>>>>> which is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That
> said,
> > > > > > > StateStores
> > > > > > > >>>>>>>> are
> > > > > > > >>>>>>>>>> free
> > > > > > > >>>>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>> implement
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> these guarantees however they can,
> which is
> > > a
> > > > > bit
> > > > > > > more
> > > > > > > >>>>>>>>>> relaxed
> > > > > > > >>>>>>>>>>>>>> than
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions".
> For
> > > > > example,
> > > > > > > >>>> with
> > > > > > > >>>>>>>>>>>> RocksDB
> > > > > > > >>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>> would
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED ==
> > > WBWI-based
> > > > > > > >>>>>>>>>> "transactions",
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the
> > > > > database.
> > > > > > > But
> > > > > > > >>>>>>>> with
> > > > > > > >>>>>>>>>>>> other
> > > > > > > >>>>>>>>>>>>>>>>>>>> storage
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to
> *always*
> > > use
> > > > > > > >>>>>>>> transactions,
> > > > > > > >>>>>>>>>>>> even
> > > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage
> engines
> > > > > that
> > > > > > > >> don't
> > > > > > > >>>>>>>>>> provide
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation
> guarantees
> > > can
> > > > > be
> > > > > > > met
> > > > > > > >>>>>>>> using a
> > > > > > > >>>>>>>>>>>>>>>>>>>> different
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> technique.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the
> StateStore
> > > > > interface
> > > > > > > as
> > > > > > > >>>>>>>>>> loosely
> > > > > > > >>>>>>>>>>>>>>>> coupled
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to
> give
> > > > > > > >>>>>>>> implementers
> > > > > > > >>>>>>>>>> more
> > > > > > > >>>>>>>>>>>>>>>>>>>> freedom, and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge
> > > > > required.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> > > "IsolationLevel"
> > > > > might
> > > > > > > >> not
> > > > > > > >>>>>>>> be
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>> right
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it
> much
> > > more
> > > > > > > >>>>>>>> explicit if
> > > > > > > >>>>>>>>>>>>>>>>>>>> required, e.g.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> 7-8.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> I can make these changes either later
> today
> > > or
> > > > > > > >>>> tomorrow.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> Small update:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and
> fixed a
> > > > > bunch of
> > > > > > > >>>>>>>> issues
> > > > > > > >>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>> needed
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests
> pass,
> > > > > which is
> > > > > > > >>>>>>>>>> promising,
> > > > > > > >>>>>>>>>>>> but
> > > > > > > >>>>>>>>>>>>>>>> it
> > > > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> need to undergo some performance
> testing. I
> > > > > haven't
> > > > > > > >>>>>>>> (yet)
> > > > > > > >>>>>>>>>>>> worked
> > > > > > > >>>>>>>>>>>>>> on
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff,
> but I
> > > > > would
> > > > > > > >>>>>>>> expect
> > > > > > > >>>>>>>>>> that,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
> > > difference. The
> > > > > > > >> branch
> > > > > > > >>>>>>>> is
> > > > > > > >>>>>>>>>>>>>> available
> > > > > > > >>>>>>>>>>>>>>>>>>>> at
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > https://github.com/nicktelford/kafka/tree/KIP-892-c
> > > > > > > >> if
> > > > > > > >>>>>>>>>> anyone
> > > > > > > >>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno
> Cadonna
> > > <
> > > > > > > >>>>>>>>>>>> cadonna@apache.org>
> > > > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was
> actually
> > > also
> > > > > my
> > > > > > > >>>>>>>> point. I
> > > > > > > >>>>>>>>>>>>>>>> understood
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion
> path
> > > as
> > > > > a way
> > > > > > > >> to
> > > > > > > >>>>>>>> avoid
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> early
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the
> > > intent.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually
> it is
> > > > > public
> > > > > > > >>>>>>>> API. My
> > > > > > > >>>>>>>>>>>>>> question
> > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal
> > > > > processing.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the
> size
> > > of the
> > > > > > > >>>>>>>> transaction
> > > > > > > >>>>>>>>>>>>>> buffer
> > > > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit
> affects
> > > the
> > > > > whole
> > > > > > > >>>>>>>>>> processing
> > > > > > > >>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a
> state
> > > store
> > > > > is
> > > > > > > not
> > > > > > > >>>>>>>>>>>> confined to
> > > > > > > >>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and
> > > > > changes the
> > > > > > > >>>>>>>> behavior
> > > > > > > >>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>> other
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you
> that
> > > it
> > > > > is a
> > > > > > > >>>>>>>> decent
> > > > > > > >>>>>>>>>>>>>>>>>>>> compromise. I
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides
> and
> > > list
> > > > > the
> > > > > > > >>>>>>>> options to
> > > > > > > >>>>>>>>>>>>>>>> overcome
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all
> > > options
> > > > > seem
> > > > > > > >>>> quite
> > > > > > > >>>>>>>>>> heavy
> > > > > > > >>>>>>>>>>>>>>>>>>>> compared
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand
> what you
> > > > > mean
> > > > > > > with
> > > > > > > >>>>>>>> "less
> > > > > > > >>>>>>>>>>>>>>>>>>>> predictable
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
> > > alternatives
> > > > > > > really
> > > > > > > >>>>>>>>>>>>>> interesting.
> > > > > > > >>>>>>>>>>>>>>>>>>>> But I
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good
> and
> > > we
> > > > > should
> > > > > > > >>>>>>>> continue
> > > > > > > >>>>>>>>>>>> with
> > > > > > > >>>>>>>>>>>>>>>> it!
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my
> e-mail on
> > > > > June
> > > > > > > >> 20th:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning
> behind
> > > > > putting
> > > > > > > >>>>>>>> isolation
> > > > > > > >>>>>>>>>>>>>> level
> > > > > > > >>>>>>>>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should
> > > that
> > > > > also
> > > > > > > be
> > > > > > > >> a
> > > > > > > >>>>>>>> way
> > > > > > > >>>>>>>>>> to
> > > > > > > >>>>>>>>>>>>>> give
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to
> decide
> > > > > whether
> > > > > > > to
> > > > > > > >>>>>>>> turn on
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned
> about
> > > > > how do
> > > > > > > >> you
> > > > > > > >>>>>>>> know
> > > > > > > >>>>>>>>>>>> if a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written
> under
> > > EOS,
> > > > > if
> > > > > > > you
> > > > > > > >>>>>>>> do not
> > > > > > > >>>>>>>>>>>>>> have a
> > > > > > > >>>>>>>>>>>>>>>>>>>> way
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
> > > transactional or
> > > > > not.
> > > > > > > >> If
> > > > > > > >>>>>>>> a
> > > > > > > >>>>>>>>>> state
> > > > > > > >>>>>>>>>>>>>>>> store
> > > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can
> be
> > > > > written
> > > > > > > >>>>>>>> during
> > > > > > > >>>>>>>>>>>> normal
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state
> store
> > > is not
> > > > > > > >>>>>>>>>> transactional,
> > > > > > > >>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written
> under
> > > EOS.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 7.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> My point was about not only
> considering the
> > > > > bytes
> > > > > > > in
> > > > > > > >>>>>>>> memory
> > > > > > > >>>>>>>>>> in
> > > > > > > >>>>>>>>>>>>>>>> config
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but
> also
> > > > > bytes
> > > > > > > that
> > > > > > > >>>>>>>> might
> > > > > > > >>>>>>>>>> be
> > > > > > > >>>>>>>>>>>>>>>>>>>> spilled
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering
> > > whether you
> > > > > > > >> should
> > > > > > > >>>>>>>>>> remove
> > > > > > > >>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory
> > > bytes to
> > > > > be
> > > > > > > >> used
> > > > > > > >>>>>>>> to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store
> records." My
> > > > > > > thinking
> > > > > > > >>>>>>>> was
> > > > > > > >>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>> even
> > > > > > > >>>>>>>>>>>>>>>>>>>> if a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to
> > > disk,
> > > > > > > >> limiting
> > > > > > > >>>>>>>> the
> > > > > > > >>>>>>>>>>>>>> overall
> > > > > > > >>>>>>>>>>>>>>>>>>>> bytes
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it
> again
> > > and
> > > > > > > >>>>>>>> considering
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>> recent
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much
> > > sense
> > > > > > > anymore.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I like the name
> > > > > > > >>>>>>>> statestore.transaction.buffer.max.bytes that
> > > > > > > >>>>>>>>>>>> you
> > > > > > > >>>>>>>>>>>>>>>>>>>> proposed.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> 8.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> > > > > implementation
> > > > > > > >>>>>>>> details) of
> > > > > > > >>>>>>>>>>>> how
> > > > > > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of
> changelog
> > > > > > > >>>>>>>> transactions,
> > > > > > > >>>>>>>>>>>> state
> > > > > > > >>>>>>>>>>>>>>>> store
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be
> > > great.
> > > > > > > Would
> > > > > > > >>>> be
> > > > > > > >>>>>>>>>> great
> > > > > > > >>>>>>>>>>>> if
> > > > > > > >>>>>>>>>>>>>>>> you
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the
> > > > > behavior in
> > > > > > > >>>>>>>> case of
> > > > > > > >>>>>>>>>> a
> > > > > > > >>>>>>>>>>>>>>>>>>>> failure.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional
> state
> > > > > store
> > > > > > > >>>>>>>> recover
> > > > > > > >>>>>>>>>>>> after a
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
> > > transaction
> > > > > > > buffer,
> > > > > > > >>>>>>>> etc.
> > > > > > > >>>>>>>>>>>> (that
> > > > > > > >>>>>>>>>>>>>> is
> > > > > > > >>>>>>>>>>>>>>>>>>>> what
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> > > > > > > >>>>>>>> WriteBatchWithIndex
> > > > > > > >>>>>>>>>>>>>>>>>>>> transactions
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to
> > > exceed)
> > > > > > > >>>>>>>> configured
> > > > > > > >>>>>>>>>>>> memory
> > > > > > > >>>>>>>>>>>>>>>>>>>> needs to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
> > > Ultimately,
> > > > > any
> > > > > > > >>>>>>>> approach
> > > > > > > >>>>>>>>>>>> based
> > > > > > > >>>>>>>>>>>>>> on
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> cracking
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in
> > > ways
> > > > > it's
> > > > > > > not
> > > > > > > >>>>>>>> really
> > > > > > > >>>>>>>>>>>>>>>> designed
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> for is
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen
> performance
> > > or
> > > > > > > >>>>>>>> consistency
> > > > > > > >>>>>>>>>>>> issues.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing
> these
> > > > > early
> > > > > > > >>>>>>>> commits?
> > > > > > > >>>>>>>>>>>> While
> > > > > > > >>>>>>>>>>>>>> not
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to
> > > ensure
> > > > > > > >>>>>>>> consistency
> > > > > > > >>>>>>>>>>>> whilst
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> maintaining
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem
> *very*
> > > > > > > >>>>>>>> complicated, and
> > > > > > > >>>>>>>>>>>> might
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for
> users
> > > as a
> > > > > > > >>>>>>>> consequence.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of
> this
> > > > > KIP is
> > > > > > > >>>>>>>> growing a
> > > > > > > >>>>>>>>>>>> bit
> > > > > > > >>>>>>>>>>>>>>>> out
> > > > > > > >>>>>>>>>>>>>>>>>>>> of
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss
> ideas
> > > for
> > > > > > > future
> > > > > > > >>>>>>>>>>>>>>>> improvements, I
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> think
> > >
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope
> down
> > > to a
> > > > > > > design
> > > > > > > >>>>>>>> that
> > > > > > > >>>>>>>>>>>>>> achieves
> > > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> most
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
> > > > > restorations
> > > > > > > >>>>>>>> during
> > > > > > > >>>>>>>>>> dirty
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design
> that
> > > > > this KIP
> > > > > > > >>>>>>>> produces
> > > > > > > >>>>>>>>>>>> can
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> ultimately
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially
> if
> > > the
> > > > > bulk
> > > > > > > of
> > > > > > > >>>>>>>> it is
> > > > > > > >>>>>>>>>>>>>> internal
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week
> > > trying
> > > > > to
> > > > > > > >>>>>>>> re-work
> > > > > > > >>>>>>>>>> the
> > > > > > > >>>>>>>>>>>>>>>>>>>> original
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove
> the
> > > > > > > >>>>>>>> newTransaction()
> > > > > > > >>>>>>>>>>>>>> method,
> > > > > > > >>>>>>>>>>>>>>>>>>>> such
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
> > > > > > > RocksDBStore.
> > > > > > > >>>>>>>> That
> > > > > > > >>>>>>>>>>>> way, if
> > > > > > > >>>>>>>>>>>>>>>> we
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>> want to
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the
> future,
> > > > > like
> > > > > > > the
> > > > > > > >>>>>>>> SST
> > > > > > > >>>>>>>>>> file
> > > > > > > >>>>>>>>>>>>>>>>>>>> management
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so
> with
> > > > > little/no
> > > > > > > >>>> API
> > > > > > > >>>>>>>>>>>> changes.
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>>
> > > > > > > >>>>>>>>>>>
> > > > > > > >>>>>>>>>>
> > > > > > > >>>>>>>>
> > > > > > > >>>>>>>
> > > > > > > >>>>>>
> > > > > > > >>>>>
> > > > > > > >>>>
> > > > > > > >>>
> > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > >
> > > > >
> > >
>

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

Posted by Lucas Brutschy <lb...@confluent.io.INVALID>.
Hi Nick,

I think indeed the better behavior would be to retry commitTransaction
until we risk running out of time to meet `max.poll.interval.ms`.

However, if it's handled as a `TaskCorruptedException` at the moment,
I would do the same in this KIP, and leave exception handling
improvements to future work. 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 <ni...@gmail.com> wrote:
>
> Hi Lucas,
>
> Yeah, this is pretty much the direction I'm thinking of going in now. You
> make an interesting point about committing on-error under
> ALOS/READ_COMMITTED, although I haven't had a chance to think through the
> implications yet.
>
> Something that I ran into earlier this week is an issue with the new
> handling of TimeoutException. Without TX stores, TimeoutException under EOS
> throws a TaskCorruptedException, which wipes the stores. However, with TX
> stores, TimeoutException is now just bubbled up and dealt with as it is
> under ALOS. The problem arises when the Producer#commitTransaction call
> times out: Streams attempts to ignore the error and continue producing,
> which causes the next call to Producer#send to throw
> "IllegalStateException: Cannot attempt operation `send` because the
> previous call to `commitTransaction` timed out and must be retried".
>
> I'm not sure what we should do here: retrying the commitTransaction seems
> logical, but what if it times out again? Where do we draw the line and
> shutdown the instance?
>
> Regards,
> Nick
>
> On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <lb...@confluent.io.invalid>
> wrote:
>
> > Hi all,
> >
> > I think I liked your suggestion of allowing EOS with READ_UNCOMMITTED,
> > but keep wiping the state on error, and I'd vote for this solution
> > when introducing `default.state.isolation.level`. This way, we'd have
> > the most low-risk roll-out of this feature (no behavior change without
> > reconfiguration), with the possibility of switching to the most sane /
> > battle-tested default settings in 4.0. Essentially, we'd have a
> > feature flag but call it `default.state.isolation.level` and don't
> > have to deprecate it later.
> >
> > So the possible configurations would then be this:
> >
> > 1. ALOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
> > reads from DB.
> > 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> > WriteBatch/DB. Flush on error (see note below).
> > 3. EOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
> > reads from DB. Wipe state on error.
> > 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> > WriteBatch/DB.
> >
> > I believe the feature is important enough that we will see good
> > adoption even without changing the default. In 4.0, when we have seen
> > this being adopted and is battle-tested, we make READ_COMMITTED the
> > default for EOS, or even READ_COMITTED always the default, depending
> > on our experiences. And we could add a clever implementation of
> > READ_UNCOMITTED with WriteBatches later.
> >
> > The only smell here is that `default.state.isolation.level` wouldn't
> > be purely an IQ setting, but it would also (slightly) change the
> > behavior of the processing, but that seems unavoidable as long as we
> > haven't solve READ_UNCOMITTED IQ with WriteBatches.
> >
> > Minor: As for Bruno's point 4, I think if we are concerned about this
> > 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).
> >
> > Cheers,
> > Lucas
> >
> > On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <ni...@gmail.com>
> > wrote:
> > >
> > > Hi Guozhang,
> > >
> > > The KIP as it stands introduces a new configuration,
> > > default.state.isolation.level, which is independent of processing.mode.
> > > It's intended that this new configuration be used to configure a global
> > IQ
> > > isolation level in the short term, with a future KIP introducing the
> > > capability to change the isolation level on a per-query basis, falling
> > back
> > > to the "default" defined by this config. That's why I called it
> > "default",
> > > for future-proofing.
> > >
> > > However, it currently includes the caveat that READ_UNCOMMITTED is not
> > > available under EOS. I think this is the coupling you are alluding to?
> > >
> > > This isn't intended to be a restriction of the API, but is currently a
> > > technical limitation. However, after discussing with some users about
> > > use-cases that would require READ_UNCOMMITTED under EOS, I'm inclined to
> > > remove that clause and put in the necessary work to make that combination
> > > possible now.
> > >
> > > I currently see two possible approaches:
> > >
> > >    1. Disable TX StateStores internally when the IsolationLevel is
> > >    READ_UNCOMMITTED and the processing.mode is EOS. This is more
> > difficult
> > >    than it sounds, as there are many assumptions being made throughout
> > the
> > >    internals about the guarantees StateStores provide. It would
> > definitely add
> > >    a lot of extra "if (read_uncommitted && eos)" branches, complicating
> > >    maintenance and testing.
> > >    2. Invest the time *now* to make READ_UNCOMMITTED of EOS StateStores
> > >    possible. I have some ideas on how this could be achieved, but they
> > would
> > >    need testing and could introduce some additional issues. The benefit
> > of
> > >    this approach is that it would make query-time IsolationLevels much
> > simpler
> > >    to implement in the future.
> > >
> > > Unfortunately, both will require considerable work that will further
> > delay
> > > this KIP, which was the reason I placed the restriction in the KIP in the
> > > first place.
> > >
> > > Regards,
> > > Nick
> > >
> > > On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <gu...@gmail.com>
> > > wrote:
> > >
> > > > Hello Nick,
> > > >
> > > > First of all, thanks a lot for the great effort you've put in driving
> > > > this KIP! I really like it coming through finally, as many people in
> > > > the community have raised this. At the same time I honestly feel a bit
> > > > ashamed for not putting enough of my time supporting it and pushing it
> > > > through the finish line (you raised this KIP almost a year ago).
> > > >
> > > > I briefly passed through the DISCUSS thread so far, not sure I've 100
> > > > percent digested all the bullet points. But with the goal of trying to
> > > > help take it through the finish line in mind, I'd want to throw
> > > > thoughts on top of my head only on the point #4 above which I felt may
> > > > be the main hurdle for the current KIP to drive to a consensus now.
> > > >
> > > > The general question I asked myself is, whether we want to couple "IQ
> > > > reading mode" with "processing mode". While technically I tend to
> > > > agree with you that, it's feels like a bug if some single user chose
> > > > "EOS" for processing mode while choosing "read uncommitted" for IQ
> > > > reading mode, at the same time, I'm thinking if it's possible that
> > > > there could be two different persons (or even two teams) that would be
> > > > using the stream API to build the app, and the IQ API to query the
> > > > running state of the app. I know this is less of a technical thing but
> > > > rather a more design stuff, but if it could be ever the case, I'm
> > > > wondering if the personale using the IQ API knows about the risks of
> > > > using read uncommitted but still chose so for the favor of
> > > > performance, no matter if the underlying stream processing mode
> > > > configured by another personale is EOS or not. In that regard, I'm
> > > > leaning towards a "leaving the door open, and close it later if we
> > > > found it's a bad idea" aspect with a configuration 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 can still enforce
> > > > coupling them.
> > > >
> > > > Again, just my 2c here. Thanks again for the great patience and
> > > > diligence on this KIP.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > >
> > > > On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <ni...@gmail.com>
> > > > wrote:
> > > > >
> > > > > Hi Bruno,
> > > > >
> > > > > 4.
> > > > > I'll hold off on making that change until we have a consensus as to
> > what
> > > > > configuration to use to control all of this, as it'll be affected by
> > the
> > > > > decision on EOS isolation levels.
> > > > >
> > > > > 5.
> > > > > Done. I've chosen "committedOffsets".
> > > > >
> > > > > Regards,
> > > > > Nick
> > > > >
> > > > > On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <ca...@apache.org>
> > wrote:
> > > > >
> > > > > > Hi Nick,
> > > > > >
> > > > > > 1.
> > > > > > Yeah, you are probably right that it does not make too much sense.
> > > > > > Thanks for the clarification!
> > > > > >
> > > > > >
> > > > > > 4.
> > > > > > Yes, sorry for the back and forth, but I think for the sake of the
> > KIP
> > > > > > it is better to let the ALOS behavior as it is for now due to the
> > > > > > possible issues you would run into. Maybe we can find a solution
> > in the
> > > > > > future. Now the question returns to whether we really need
> > > > > > default.state.isolation.level. Maybe the config could be the
> > feature
> > > > > > flag Sophie requested.
> > > > > >
> > > > > >
> > > > > > 5.
> > > > > > There 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 committedOffsetFor().
> > > > > >
> > > > > >
> > > > > > Best,
> > > > > > Bruno
> > > > > >
> > > > > >
> > > > > > On 10/13/23 3:21 PM, Nick Telford wrote:
> > > > > > > Hi Bruno,
> > > > > > >
> > > > > > > Thanks for getting back to me.
> > > > > > >
> > > > > > > 1.
> > > > > > > I think this should be possible. Are you thinking of the
> > situation
> > > > where
> > > > > > a
> > > > > > > user may downgrade to a previous version of Kafka Streams? In
> > that
> > > > case,
> > > > > > > sadly, the RocksDBStore would get wiped by the older version of
> > Kafka
> > > > > > > Streams anyway, because that version wouldn't understand the
> > extra
> > > > column
> > > > > > > family (that holds offsets), so the missing Position file would
> > > > > > > automatically get rebuilt when the store is rebuilt from the
> > > > changelog.
> > > > > > > Are there other situations than downgrade where a transactional
> > store
> > > > > > could
> > > > > > > be replaced by a non-transactional one? I can't think of any.
> > > > > > >
> > > > > > > 2.
> > > > > > > Ahh yes, the Test Plan - my Kryptonite! This section definitely
> > > > needs to
> > > > > > be
> > > > > > > fleshed out. I'll work on that. How much detail do you need?
> > > > > > >
> > > > > > > 3.
> > > > > > > See my previous email discussing this.
> > > > > > >
> > > > > > > 4.
> > > > > > > Hmm, this is an interesting point. Are you suggesting that under
> > ALOS
> > > > > > > READ_COMMITTED should not be supported?
> > > > > > >
> > > > > > > Regards,
> > > > > > > Nick
> > > > > > >
> > > > > > > On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <ca...@apache.org>
> > > > wrote:
> > > > > > >
> > > > > > >> Hi Nick,
> > > > > > >>
> > > > > > >> I think the KIP is converging!
> > > > > > >>
> > > > > > >>
> > > > > > >> 1.
> > > > > > >> I am wondering whether it makes sense to write the position file
> > > > during
> > > > > > >> close as we do for the checkpoint file, so that in case the
> > state
> > > > store
> > > > > > >> is replaced with a non-transactional state store the
> > > > non-transactional
> > > > > > >> state store finds the position file. I think, this is not
> > strictly
> > > > > > >> needed, but would be a nice behavior instead of just deleting
> > the
> > > > > > >> position file.
> > > > > > >>
> > > > > > >>
> > > > > > >> 2.
> > > > > > >> The test plan does not mention integration tests. Do you not
> > need to
> > > > > > >> extend existing ones and add new ones. Also for upgrading and
> > > > > > >> downgrading you might need integration and/or system tests.
> > > > > > >>
> > > > > > >>
> > > > > > >> 3.
> > > > > > >> I think Sophie made a point. Although, IQ reading from
> > uncommitted
> > > > data
> > > > > > >> under EOS might be considered a bug by some people. Thus, your
> > KIP
> > > > would
> > > > > > >> fix a bug rather than changing the intended behavior. However, I
> > > > also
> > > > > > >> see that a feature flag would help users that rely on this buggy
> > > > > > >> behavior (at least until AK 4.0).
> > > > > > >>
> > > > > > >>
> > > > > > >> 4.
> > > > > > >> This is related to the previous point. I assume that the
> > difference
> > > > > > >> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is that in
> > the
> > > > > > >> former you enable transactions on the state store and in the
> > latter
> > > > you
> > > > > > >> disable them. If my assumption is correct, I think that is an
> > issue.
> > > > > > >> Let's assume under ALOS Streams fails over a couple of times
> > more or
> > > > > > >> less at the same step in processing after value 3 is added to an
> > > > > > >> aggregation but the offset of the corresponding input record
> > was not
> > > > > > >> committed. Without transactions disabled, the aggregation value
> > > > would
> > > > > > >> increase by 3 for each failover. With transactions enabled,
> > value 3
> > > > > > >> would only be added to the aggregation once when the offset of
> > the
> > > > input
> > > > > > >> record is committed and the transaction finally completes. So
> > the
> > > > > > >> content of the state store would change depending on the
> > > > configuration
> > > > > > >> for IQ. IMO, the content of the state store should be
> > independent
> > > > from
> > > > > > >> IQ. Given this issue, I propose to not use transactions with
> > ALOS at
> > > > > > >> all. I was a big proponent of using transactions with ALOS, but
> > I
> > > > > > >> realized that transactions with ALOS is not as easy as enabling
> > > > > > >> transactions on state stores. Another aspect that is
> > problematic is
> > > > that
> > > > > > >> the changelog topic which actually replicates the state store
> > is not
> > > > > > >> transactional under ALOS. Thus, it might happen that the state
> > > > store and
> > > > > > >> the changelog differ in their content. All of this is maybe
> > solvable
> > > > > > >> somehow, but for the sake of this KIP, I would leave it for the
> > > > future.
> > > > > > >>
> > > > > > >>
> > > > > > >> Best,
> > > > > > >> Bruno
> > > > > > >>
> > > > > > >>
> > > > > > >>
> > > > > > >> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > > > > > >>> Hey Nick! First of all thanks for taking up this awesome
> > feature,
> > > > I'm
> > > > > > >> sure
> > > > > > >>> every single
> > > > > > >>> Kafka Streams user and dev would agree that it is sorely
> > needed.
> > > > > > >>>
> > > > > > >>> I've just been catching up on the KIP and surrounding
> > discussion,
> > > > so
> > > > > > >> please
> > > > > > >>> forgive me
> > > > > > >>> for any misunderstandings or misinterpretations of the current
> > > > plan and
> > > > > > >>> don't hesitate to
> > > > > > >>> correct me.
> > > > > > >>>
> > > > > > >>> Before I jump in, I just want to say that having seen this
> > drag on
> > > > for
> > > > > > so
> > > > > > >>> long, my singular
> > > > > > >>> goal in responding is to help this KIP past a perceived
> > impasse so
> > > > we
> > > > > > can
> > > > > > >>> finally move on
> > > > > > >>> to voting and implementing it. Long discussions are to be
> > expected
> > > > for
> > > > > > >>> major features like
> > > > > > >>> this but it's completely on us as the Streams devs to make sure
> > > > there
> > > > > > is
> > > > > > >> an
> > > > > > >>> end in sight
> > > > > > >>> for any ongoing discussion.
> > > > > > >>>
> > > > > > >>> With that said, it's my understanding that the KIP as currently
> > > > > > proposed
> > > > > > >> is
> > > > > > >>> just not tenable
> > > > > > >>> for Kafka Streams, and would prevent some EOS users from
> > upgrading
> > > > to
> > > > > > the
> > > > > > >>> version it
> > > > > > >>> first appears in. Given that we can't predict or guarantee
> > whether
> > > > any
> > > > > > of
> > > > > > >>> the followup KIPs
> > > > > > >>> would be completed in the same release cycle as this one, we
> > need
> > > > to
> > > > > > make
> > > > > > >>> sure that the
> > > > > > >>> feature is either compatible with all current users or else
> > > > > > >> feature-flagged
> > > > > > >>> so that they may
> > > > > > >>> opt in/out.
> > > > > > >>>
> > > > > > >>> Therefore, IIUC we need to have either (or both) of these as
> > > > > > >>> fully-implemented config options:
> > > > > > >>> 1. default.state.isolation.level
> > > > > > >>> 2. enable.transactional.state.stores
> > > > > > >>>
> > > > > > >>> This way EOS users for whom read_committed semantics are not
> > > > viable can
> > > > > > >>> still upgrade,
> > > > > > >>> and either use the isolation.level config to leverage the new
> > txn
> > > > state
> > > > > > >>> stores without sacrificing
> > > > > > >>> their application semantics, or else simply keep the
> > transactional
> > > > > > state
> > > > > > >>> stores disabled until we
> > > > > > >>> are able to fully implement the isolation level configuration
> > at
> > > > either
> > > > > > >> an
> > > > > > >>> application or query level.
> > > > > > >>>
> > > > > > >>> Frankly you are the expert here and know much more about the
> > > > tradeoffs
> > > > > > in
> > > > > > >>> both semantics and
> > > > > > >>> effort level of implementing one of these configs vs the
> > other. In
> > > > my
> > > > > > >>> opinion, either option would
> > > > > > >>> be fine and I would leave the decision of which one to include
> > in
> > > > this
> > > > > > >> KIP
> > > > > > >>> completely up to you.
> > > > > > >>> I just don't see a way for the KIP to proceed without some
> > > > variation of
> > > > > > >> the
> > > > > > >>> above that would allow
> > > > > > >>> EOS users to opt-out of read_committed.
> > > > > > >>>
> > > > > > >>> (If it's all the same to you, I would recommend 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 that gets released.
> > > > > > >>> So personally, my choice would be to add the feature flag and
> > > > leave it
> > > > > > >> off
> > > > > > >>> by default. If all 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
> > > > > > >>>
> > > > > > >>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > > > nick.telford@gmail.com>
> > > > > > >> wrote:
> > > > > > >>>
> > > > > > >>>> Hi Bruno,
> > > > > > >>>>
> > > > > > >>>> Agreed, I can live with that for now.
> > > > > > >>>>
> > > > > > >>>> In an effort to keep the scope of this KIP from expanding, I'm
> > > > leaning
> > > > > > >>>> towards just providing a configurable
> > > > default.state.isolation.level
> > > > > > and
> > > > > > >>>> removing IsolationLevel from the StateStoreContext. This
> > would be
> > > > > > >>>> compatible with adding support for query-time IsolationLevels
> > in
> > > > the
> > > > > > >>>> future, whilst providing a way for users to select an
> > isolation
> > > > level
> > > > > > >> now.
> > > > > > >>>>
> > > > > > >>>> The big problem with this, however, is that if a user selects
> > > > > > >>>> processing.mode
> > > > > > >>>> = "exactly-once(-v2|-beta)", and
> > default.state.isolation.level =
> > > > > > >>>> "READ_UNCOMMITTED", we need to guarantee that the data isn't
> > > > written
> > > > > > to
> > > > > > >>>> disk until commit() is called, but we also need to permit IQ
> > > > threads
> > > > > > to
> > > > > > >>>> read from the ongoing transaction.
> > > > > > >>>>
> > > > > > >>>> A simple solution would be to (temporarily) forbid this
> > > > combination of
> > > > > > >>>> configuration, and have default.state.isolation.level
> > > > automatically
> > > > > > >> switch
> > > > > > >>>> to READ_COMMITTED when processing.mode is anything other than
> > > > > > >>>> at-least-once. Do you think this would be acceptable?
> > > > > > >>>>
> > > > > > >>>> In a later KIP, we can add support for query-time isolation
> > > > levels and
> > > > > > >>>> solve this particular problem there, which would relax this
> > > > > > restriction.
> > > > > > >>>>
> > > > > > >>>> Regards,
> > > > > > >>>> Nick
> > > > > > >>>>
> > > > > > >>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> > cadonna@apache.org>
> > > > > > >> 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,
> > > > > > >>>>> Bruno
> > > > > > >>>>>
> > > > > > >>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > > > > > >>>>>> 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 <
> > > > nick.telford@gmail.com
> > > > > > >
> > > > > > >>>>> wrote:
> > > > > > >>>>>>
> > > > > > >>>>>>> Hi everyone,
> > > > > > >>>>>>>
> > > > > > >>>>>>> I agree that having IsolationLevel be determined at
> > query-time
> > > > is
> > > > > > the
> > > > > > >>>>>>> ideal design, but there are a few sticking points:
> > > > > > >>>>>>>
> > > > > > >>>>>>> 1.
> > > > > > >>>>>>> There needs to be some way to communicate the
> > IsolationLevel
> > > > down
> > > > > > to
> > > > > > >>>> the
> > > > > > >>>>>>> RocksDBStore itself, so that the query can respect it.
> > Since
> > > > stores
> > > > > > >>>> are
> > > > > > >>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
> > > > MeteredStore,
> > > > > > >>>>> etc.),
> > > > > > >>>>>>> we need some way to deliver that information to the bottom
> > > > layer.
> > > > > > For
> > > > > > >>>>> IQv2,
> > > > > > >>>>>>> we can use the existing State#query() method, but IQv1 has
> > no
> > > > way
> > > > > > to
> > > > > > >>>> do
> > > > > > >>>>>>> this.
> > > > > > >>>>>>>
> > > > > > >>>>>>> A simple approach, which would potentially open up other
> > > > options,
> > > > > > >>>> would
> > > > > > >>>>> be
> > > > > > >>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> > > > > > >>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > > > > > ReadOnlyKeyValueStore
> > > > > > >>>>> (and
> > > > > > >>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore,
> > etc.).
> > > > > > >>>>>>>
> > > > > > >>>>>>> 2.
> > > > > > >>>>>>> As mentioned above, RocksDB WriteBatches are not
> > thread-safe,
> > > > which
> > > > > > >>>>> causes
> > > > > > >>>>>>> a problem if we want to provide READ_UNCOMMITTED
> > Iterators. I
> > > > also
> > > > > > >>>> had a
> > > > > > >>>>>>> look at RocksDB Transactions[1], but they solve a very
> > > > different
> > > > > > >>>>> problem,
> > > > > > >>>>>>> and have the same thread-safety issue.
> > > > > > >>>>>>>
> > > > > > >>>>>>> One possible approach that I mentioned is chaining
> > > > WriteBatches:
> > > > > > >> every
> > > > > > >>>>>>> time a new Interactive Query is received (i.e.
> > readOnlyView,
> > > > see
> > > > > > >>>> above,
> > > > > > >>>>>>> is called) we "freeze" the existing WriteBatch, and start a
> > > > new one
> > > > > > >>>> for
> > > > > > >>>>> new
> > > > > > >>>>>>> writes. The Interactive Query queries the "chain" of
> > previous
> > > > > > >>>>> WriteBatches
> > > > > > >>>>>>> + the underlying database; while the StreamThread starts
> > > > writing to
> > > > > > >>>> the
> > > > > > >>>>>>> *new* WriteBatch. On-commit, the StreamThread would write
> > *all*
> > > > > > >>>>>>> WriteBatches in the chain to the database (that have not
> > yet
> > > > been
> > > > > > >>>>> written).
> > > > > > >>>>>>>
> > > > > > >>>>>>> WriteBatches would be closed/freed only when they have been
> > > > both
> > > > > > >>>>>>> committed, and all open Interactive Queries on them have
> > been
> > > > > > closed.
> > > > > > >>>>> This
> > > > > > >>>>>>> would require some reference counting.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Obviously a drawback of this approach is the potential for
> > > > > > increased
> > > > > > >>>>>>> memory usage: if an Interactive Query is long-lived, for
> > > > example by
> > > > > > >>>>> doing a
> > > > > > >>>>>>> full scan over a large database, or even just pausing in
> > the
> > > > middle
> > > > > > >> of
> > > > > > >>>>> an
> > > > > > >>>>>>> iteration, then the existing chain of WriteBatches could be
> > > > kept
> > > > > > >>>> around
> > > > > > >>>>> for
> > > > > > >>>>>>> a long time, potentially forever.
> > > > > > >>>>>>>
> > > > > > >>>>>>> --
> > > > > > >>>>>>>
> > > > > > >>>>>>> A.
> > > > > > >>>>>>> Going off on a tangent, it looks like in addition to
> > supporting
> > > > > > >>>>>>> READ_COMMITTED queries, we could go further and support
> > > > > > >>>> REPEATABLE_READ
> > > > > > >>>>>>> queries (i.e. where subsequent reads to the same key in the
> > > > same
> > > > > > >>>>>>> Interactive Query are guaranteed to yield the same value)
> > by
> > > > making
> > > > > > >>>> use
> > > > > > >>>>> of
> > > > > > >>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so the
> > > > > > >> performance
> > > > > > >>>>>>> impact is likely to be negligible, but they do require
> > that the
> > > > > > >>>>> Interactive
> > > > > > >>>>>>> Query session can be explicitly closed.
> > > > > > >>>>>>>
> > > > > > >>>>>>> This could be achieved if we made the above readOnlyView
> > > > interface
> > > > > > >>>> look
> > > > > > >>>>>>> more like:
> > > > > > >>>>>>>
> > > > > > >>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> > > > > > >>>> ReadOnlyKeyValueStore<K,
> > > > > > >>>>>>> V>, AutoCloseable {}
> > > > > > >>>>>>>
> > > > > > >>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > > > > > >>>>>>>        ...
> > > > > > >>>>>>>        ReadOnlyKeyValueView<K, V>
> > readOnlyView(IsolationLevel
> > > > > > >>>>> isolationLevel);
> > > > > > >>>>>>> }
> > > > > > >>>>>>>
> > > > > > >>>>>>> But this would be a breaking change, as existing IQv1
> > queries
> > > > are
> > > > > > >>>>>>> guaranteed to never call store.close(), and therefore these
> > > > would
> > > > > > >> leak
> > > > > > >>>>>>> memory under REPEATABLE_READ.
> > > > > > >>>>>>>
> > > > > > >>>>>>> B.
> > > > > > >>>>>>> One thing that's notable: MyRocks states that they support
> > > > > > >>>>> READ_COMMITTED
> > > > > > >>>>>>> and REPEATABLE_READ, but they make no mention of
> > > > > > >>>> READ_UNCOMMITTED[3][4].
> > > > > > >>>>>>> This could be because doing so is technically
> > > > difficult/impossible
> > > > > > >>>> using
> > > > > > >>>>>>> the primitives available in RocksDB.
> > > > > > >>>>>>>
> > > > > > >>>>>>> --
> > > > > > >>>>>>>
> > > > > > >>>>>>> Lucas, to address your points:
> > > > > > >>>>>>>
> > > > > > >>>>>>> U1.
> > > > > > >>>>>>> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
> > > > > > >>>>>>> implementations of StateStore that do not support atomic
> > > > writes.
> > > > > > >>>>> Obviously
> > > > > > >>>>>>> in those cases, the guarantees Kafka Streams
> > provides/expects
> > > > would
> > > > > > >> be
> > > > > > >>>>>>> relaxed. Do you think we should require all
> > implementations to
> > > > > > >> support
> > > > > > >>>>>>> atomic writes?
> > > > > > >>>>>>>
> > > > > > >>>>>>> U2.
> > > > > > >>>>>>> Stores can support multiple IsolationLevels. As we've
> > discussed
> > > > > > >> above,
> > > > > > >>>>> the
> > > > > > >>>>>>> ideal scenario would be to specify the IsolationLevel at
> > > > > > query-time.
> > > > > > >>>>>>> Failing that, I think the second-best approach is to
> > define the
> > > > > > >>>>>>> IsolationLevel for *all* queries based on the
> > processing.mode,
> > > > > > which
> > > > > > >>>> is
> > > > > > >>>>>>> what the default StateStoreContext#isolationLevel()
> > achieves.
> > > > Would
> > > > > > >>>> you
> > > > > > >>>>>>> prefer an alternative?
> > > > > > >>>>>>>
> > > > > > >>>>>>> While the existing implementation is equivalent to
> > > > > > READ_UNCOMMITTED,
> > > > > > >>>>> this
> > > > > > >>>>>>> can yield unexpected results/errors under EOS, if a
> > > > transaction is
> > > > > > >>>>> rolled
> > > > > > >>>>>>> back. While this would be a change in behaviour for users,
> > it
> > > > would
> > > > > > >>>> look
> > > > > > >>>>>>> more like a bug fix than a breaking change. That said, we
> > > > *could*
> > > > > > >> make
> > > > > > >>>>> it
> > > > > > >>>>>>> configurable, and default to the existing behaviour
> > > > > > >> (READ_UNCOMMITTED)
> > > > > > >>>>>>> instead of inferring it from the processing.mode?
> > > > > > >>>>>>>
> > > > > > >>>>>>> N1, N2.
> > > > > > >>>>>>> These were only primitives to avoid boxing costs, but since
> > > > this is
> > > > > > >>>> not
> > > > > > >>>>> a
> > > > > > >>>>>>> performance sensitive area, it should be fine to change if
> > > > that's
> > > > > > >>>>> desirable.
> > > > > > >>>>>>>
> > > > > > >>>>>>> N3.
> > > > > > >>>>>>> It's because the store "manages its own offsets", which
> > > > includes
> > > > > > both
> > > > > > >>>>>>> committing the offset, *and providing it* via
> > > > getCommittedOffset().
> > > > > > >>>>>>> Personally, I think "managesOffsets" conveys this best,
> > but I
> > > > don't
> > > > > > >>>> mind
> > > > > > >>>>>>> changing it if the nomenclature is unclear.
> > > > > > >>>>>>>
> > > > > > >>>>>>> Sorry for the massive emails/essays!
> > > > > > >>>>>>> --
> > > > > > >>>>>>> Nick
> > > > > > >>>>>>>
> > > > > > >>>>>>> 1: https://github.com/facebook/rocksdb/wiki/Transactions
> > > > > > >>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > > > > > >>>>>>> 3:
> > > > > > https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > > > > >>>>>>> 4:
> > https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > > > > >>>>>>>
> > > > > > >>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > > > > > >>>>>>> <lb...@confluent.io.invalid> wrote:
> > > > > > >>>>>>>
> > > > > > >>>>>>>> Hi Nick,
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> since I last read it in April, the KIP has become much
> > > > cleaner and
> > > > > > >>>>>>>> easier to read. Great work!
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> It feels to me the last big open point is whether we can
> > > > implement
> > > > > > >>>>>>>> isolation level as a query parameter. I understand that
> > there
> > > > are
> > > > > > >>>>>>>> implementation concerns, but as Colt says, it would be a
> > great
> > > > > > >>>>>>>> addition, and would also simplify the migration path for
> > this
> > > > > > >> change.
> > > > > > >>>>>>>> Is the implementation problem you mentioned caused by the
> > > > > > WriteBatch
> > > > > > >>>>>>>> not having a notion of a snapshot, as the underlying DB
> > > > iterator
> > > > > > >>>> does?
> > > > > > >>>>>>>> In that case, I am not sure a chain of WriteBatches as you
> > > > propose
> > > > > > >>>>>>>> would fully solve the problem, but maybe I didn't dig
> > enough
> > > > into
> > > > > > >> the
> > > > > > >>>>>>>> details to fully understand it.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> If it's not possible to implement it now, would it be an
> > > > option to
> > > > > > >>>>>>>> make sure in this KIP that we do not fully close the door
> > on
> > > > > > >>>> per-query
> > > > > > >>>>>>>> isolation levels in the interface, as it may be possible
> > to
> > > > > > >> implement
> > > > > > >>>>>>>> the missing primitives in RocksDB or Speedb in the future.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Understanding:
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to be
> > > > persisted
> > > > > > >>>>>>>> atomically with the records?
> > > > > > >>>>>>>> * U2) Don't understand the default implementation of
> > > > > > >>>> `isolationLevel`.
> > > > > > >>>>>>>> The isolation level should be a property of the underlying
> > > > store,
> > > > > > >> and
> > > > > > >>>>>>>> not be defined by the default config? Existing stores
> > probably
> > > > > > don't
> > > > > > >>>>>>>> guarantee READ_COMMITTED, so the default should be to
> > return
> > > > > > >>>>>>>> READ_UNCOMMITTED.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Nits:
> > > > > > >>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong`
> > return
> > > > type,
> > > > > > >> to
> > > > > > >>>>>>>> avoid the `null`?
> > > > > > >>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
> > > > `OptionalLong`
> > > > > > >>>>>>>> return type, to avoid the `-1`?
> > > > > > >>>>>>>> * N3) I don't understand why `managesOffsets` uses the
> > > > 'manage'
> > > > > > >> verb,
> > > > > > >>>>>>>> whereas all other methods use the "commits" verb. I'd
> > suggest
> > > > > > >>>>>>>> `commitsOffsets`.
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Either way, it feels this KIP is very close to the finish
> > > > line,
> > > > > > I'm
> > > > > > >>>>>>>> looking forward to seeing this in production!
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> Cheers,
> > > > > > >>>>>>>> Lucas
> > > > > > >>>>>>>>
> > > > > > >>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> > > > colt@littlehorse.io
> > > > > > >
> > > > > > >>>>> wrote:
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>> Making IsolationLevel a query-time constraint, rather
> > than
> > > > > > linking
> > > > > > >>>> it
> > > > > > >>>>>>>> to
> > > > > > >>>>>>>>> the processing.guarantee.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> As I understand it, would this allow even a user of EOS
> > to
> > > > > > control
> > > > > > >>>>>>>> whether
> > > > > > >>>>>>>>> reading committed or uncommitted records? If so, I am
> > highly
> > > > in
> > > > > > >>>> favor
> > > > > > >>>>> of
> > > > > > >>>>>>>>> this.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> I know that I was one of the early people to point out
> > the
> > > > > > current
> > > > > > >>>>>>>>> shortcoming that IQ reads uncommitted records, but just
> > this
> > > > > > >>>> morning I
> > > > > > >>>>>>>>> realized a pattern we use which means that (for certain
> > > > queries)
> > > > > > >> our
> > > > > > >>>>>>>> system
> > > > > > >>>>>>>>> needs to be able to read uncommitted records, which is
> > the
> > > > > > current
> > > > > > >>>>>>>> behavior
> > > > > > >>>>>>>>> of Kafka Streams in EOS.***
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> If IsolationLevel being a query-time decision allows for
> > > > this,
> > > > > > then
> > > > > > >>>>> that
> > > > > > >>>>>>>>> would be amazing. I would also vote that the default
> > behavior
> > > > > > >> should
> > > > > > >>>>> be
> > > > > > >>>>>>>> for
> > > > > > >>>>>>>>> reading uncommitted records, because it is totally
> > possible
> > > > for a
> > > > > > >>>>> valid
> > > > > > >>>>>>>>> application to depend on that behavior, and breaking it
> > in a
> > > > > > minor
> > > > > > >>>>>>>> release
> > > > > > >>>>>>>>> might be a bit strong.
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>> *** (Note, for the curious reader....) Our use-case/query
> > > > pattern
> > > > > > >>>> is a
> > > > > > >>>>>>>> bit
> > > > > > >>>>>>>>> complex, but reading "uncommitted" records is actually
> > safe
> > > > in
> > > > > > our
> > > > > > >>>>> case
> > > > > > >>>>>>>>> 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 soon as the processor processes the
> > > > result.
> > > > > > If
> > > > > > >>>>> they
> > > > > > >>>>>>>>> can't read uncommitted records, they may receive a "201 -
> > > > > > Created"
> > > > > > >>>>>>>>> response, immediately followed by a "404 - 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 <
> > > > > > >>>> nick.telford@gmail.com>
> > > > > > >>>>>>>> wrote:
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>> Addendum:
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> I think we would also face the same problem with the
> > > > approach
> > > > > > John
> > > > > > >>>>>>>> outlined
> > > > > > >>>>>>>>>> earlier (using the record cache as a transaction buffer
> > and
> > > > > > >>>> flushing
> > > > > > >>>>>>>> it
> > > > > > >>>>>>>>>> straight to SST files). This is because the record cache
> > > > (the
> > > > > > >>>>>>>> ThreadCache
> > > > > > >>>>>>>>>> class) is not thread-safe, so every commit would
> > invalidate
> > > > open
> > > > > > >> IQ
> > > > > > >>>>>>>>>> Iterators in the same way that RocksDB WriteBatches do.
> > > > > > >>>>>>>>>> --
> > > > > > >>>>>>>>>> Nick
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > > > > > >>>> nick.telford@gmail.com>
> > > > > > >>>>>>>>>> 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.
> > > > > > >>>>>>>>>>> 2. Making IsolationLevel a query-time constraint,
> > rather
> > > > than
> > > > > > >>>>>>>> linking it
> > > > > > >>>>>>>>>>> to the processing.guarantee.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> There's a wrinkle that makes this a challenge:
> > Interactive
> > > > > > >> Queries
> > > > > > >>>>>>>> that
> > > > > > >>>>>>>>>>> open an Iterator, when using transactions and
> > > > READ_UNCOMMITTED.
> > > > > > >>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries
> > need
> > > > to be
> > > > > > >>>> able
> > > > > > >>>>>>>> to
> > > > > > >>>>>>>>>>> read records from the currently uncommitted transaction
> > > > buffer
> > > > > > >>>>>>>>>>> (WriteBatch). This includes for Iterators, which should
> > > > iterate
> > > > > > >>>>>>>> both the
> > > > > > >>>>>>>>>>> transaction buffer and underlying database (using
> > > > > > >>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> The issue is that when the StreamThread commits, it
> > writes
> > > > the
> > > > > > >>>>>>>> current
> > > > > > >>>>>>>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*.
> > > > > > Clearing
> > > > > > >>>> the
> > > > > > >>>>>>>>>>> WriteBatch while an Interactive Query holds an open
> > > > Iterator on
> > > > > > >> it
> > > > > > >>>>>>>> will
> > > > > > >>>>>>>>>>> invalidate the Iterator. Worse, it turns out that
> > Iterators
> > > > > > over
> > > > > > >> a
> > > > > > >>>>>>>>>>> WriteBatch become invalidated not just when the
> > WriteBatch
> > > > is
> > > > > > >>>>>>>> cleared,
> > > > > > >>>>>>>>>> but
> > > > > > >>>>>>>>>>> also when the Iterators' current key receives a new
> > write.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> Now that I'm writing this, I remember that this is the
> > > > major
> > > > > > >>>> reason
> > > > > > >>>>>>>> that
> > > > > > >>>>>>>>>> I
> > > > > > >>>>>>>>>>> switched the original design from having a query-time
> > > > > > >>>>>>>> IsolationLevel to
> > > > > > >>>>>>>>>>> having the IsolationLevel linked to the
> > transactionality
> > > > of the
> > > > > > >>>>>>>> stores
> > > > > > >>>>>>>>>>> themselves.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> It *might* be possible to resolve this, by having a
> > > > "chain" of
> > > > > > >>>>>>>>>>> WriteBatches, with the StreamThread switching to a new
> > > > > > WriteBatch
> > > > > > >>>>>>>>>> whenever
> > > > > > >>>>>>>>>>> a new Interactive Query attempts to read from the
> > > > database, but
> > > > > > >>>> that
> > > > > > >>>>>>>>>> could
> > > > > > >>>>>>>>>>> cause some performance problems/memory pressure when
> > > > subjected
> > > > > > to
> > > > > > >>>> a
> > > > > > >>>>>>>> high
> > > > > > >>>>>>>>>>> Interactive Query load. It would also reduce the
> > > > efficiency of
> > > > > > >>>>>>>>>> WriteBatches
> > > > > > >>>>>>>>>>> on-commit, as we'd have to write N WriteBatches, where
> > N
> > > > is the
> > > > > > >>>>>>>> number of
> > > > > > >>>>>>>>>>> Interactive Queries since the last commit.
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> I realise this is getting into the weeds of the
> > > > implementation,
> > > > > > >>>> and
> > > > > > >>>>>>>> you'd
> > > > > > >>>>>>>>>>> rather we focus on the API for now, but I think it's
> > > > important
> > > > > > to
> > > > > > >>>>>>>>>> consider
> > > > > > >>>>>>>>>>> how to implement the desired API, in case we come up
> > with
> > > > an
> > > > > > API
> > > > > > >>>>>>>> that
> > > > > > >>>>>>>>>>> cannot be implemented efficiently, or even at all!
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> Thoughts?
> > > > > > >>>>>>>>>>> --
> > > > > > >>>>>>>>>>> Nick
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> > > > > > cadonna@apache.org
> > > > > > >>>
> > > > > > >>>>>>>> wrote:
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>>> Hi Nick,
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> 6.
> > > > > > >>>>>>>>>>>> Of course, you are right! My bad!
> > > > > > >>>>>>>>>>>> Wiping out the state in the downgrading case is fine.
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> 3a.
> > > > > > >>>>>>>>>>>> Focus on the public facing changes for the KIP. We
> > will
> > > > manage
> > > > > > >> 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.
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> Best,
> > > > > > >>>>>>>>>>>> Bruno
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > > > > > >>>>>>>>>>>>> Hi Bruno,
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> Thanks for getting back to me!
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> 2.
> > > > > > >>>>>>>>>>>>> The fact that implementations can always track
> > estimated
> > > > > > memory
> > > > > > >>>>>>>> usage
> > > > > > >>>>>>>>>> in
> > > > > > >>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as an
> > > > option,
> > > > > > and
> > > > > > >>>>>>>> I'll
> > > > > > >>>>>>>>>>>> clarify
> > > > > > >>>>>>>>>>>>> the JavaDoc that 0 is not just for non-transactional
> > > > stores,
> > > > > > >>>>>>>> which is
> > > > > > >>>>>>>>>>>>> currently misleading.
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> 6.
> > > > > > >>>>>>>>>>>>> The problem with catching the exception in the
> > downgrade
> > > > > > >> process
> > > > > > >>>>>>>> is
> > > > > > >>>>>>>>>> that
> > > > > > >>>>>>>>>>>>> would require new code in the Kafka version being
> > > > downgraded
> > > > > > >> to.
> > > > > > >>>>>>>> Since
> > > > > > >>>>>>>>>>>>> users could conceivably downgrade to almost *any*
> > older
> > > > > > version
> > > > > > >>>>>>>> of
> > > > > > >>>>>>>>>> Kafka
> > > > > > >>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
> > > > > > >>>>>>>>>>>>> The only way I can think of doing it would be to
> > provide
> > > > a
> > > > > > >>>>>>>> dedicated
> > > > > > >>>>>>>>>>>>> downgrade tool, that goes through every local store
> > and
> > > > > > removes
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>>> offsets column families. But that seems like an
> > > > unnecessary
> > > > > > >>>>>>>> amount of
> > > > > > >>>>>>>>>>>> extra
> > > > > > >>>>>>>>>>>>> code to maintain just to handle a somewhat niche
> > > > situation,
> > > > > > >> when
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>>> alternative (automatically wipe and restore stores)
> > > > should be
> > > > > > >>>>>>>>>>>> acceptable.
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've
> > requested.
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> 3a.
> > > > > > >>>>>>>>>>>>> I agree that IsolationLevel makes more sense at
> > > > query-time,
> > > > > > and
> > > > > > >>>> I
> > > > > > >>>>>>>>>>>> actually
> > > > > > >>>>>>>>>>>>> initially attempted to place the IsolationLevel at
> > > > > > query-time,
> > > > > > >>>>>>>> but I
> > > > > > >>>>>>>>>> ran
> > > > > > >>>>>>>>>>>>> into some problems:
> > > > > > >>>>>>>>>>>>> - The key issue is that, under ALOS we're not staging
> > > > writes
> > > > > > in
> > > > > > >>>>>>>>>>>>> transactions, so can't perform writes at the
> > > > READ_COMMITTED
> > > > > > >>>>>>>> isolation
> > > > > > >>>>>>>>>>>>> level. However, this may be addressed if we decide to
> > > > > > *always*
> > > > > > >>>>>>>> use
> > > > > > >>>>>>>>>>>>> transactions as discussed under 3b.
> > > > > > >>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> > implementations. I
> > > > > > >> remember
> > > > > > >>>>>>>>>> having
> > > > > > >>>>>>>>>>>>> some difficulty understanding the IQv1 internals,
> > which
> > > > made
> > > > > > it
> > > > > > >>>>>>>>>>>> difficult
> > > > > > >>>>>>>>>>>>> to determine what needed to be changed. However, I
> > > > *think*
> > > > > > this
> > > > > > >>>>>>>> can be
> > > > > > >>>>>>>>>>>>> addressed for both implementations by wrapping the
> > > > > > RocksDBStore
> > > > > > >>>>>>>> in an
> > > > > > >>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides read
> > > > methods
> > > > > > >>>>>>>> (get,
> > > > > > >>>>>>>>>>>> etc.)
> > > > > > >>>>>>>>>>>>> to either read directly from the database or from the
> > > > ongoing
> > > > > > >>>>>>>>>>>> transaction.
> > > > > > >>>>>>>>>>>>> But IQv1 might still be difficult.
> > > > > > >>>>>>>>>>>>> - If IsolationLevel becomes a query constraint, then
> > all
> > > > > > other
> > > > > > >>>>>>>>>>>> StateStores
> > > > > > >>>>>>>>>>>>> will need to respect it, including the in-memory
> > stores.
> > > > This
> > > > > > >>>>>>>> would
> > > > > > >>>>>>>>>>>> require
> > > > > > >>>>>>>>>>>>> us to adapt in-memory stores to stage their writes so
> > > > they
> > > > > > can
> > > > > > >>>> be
> > > > > > >>>>>>>>>>>> isolated
> > > > > > >>>>>>>>>>>>> from READ_COMMITTTED queries. It would also become an
> > > > > > important
> > > > > > >>>>>>>>>>>>> consideration for third-party stores on upgrade, as
> > > > without
> > > > > > >>>>>>>> changes,
> > > > > > >>>>>>>>>>>> they
> > > > > > >>>>>>>>>>>>> would not support READ_COMMITTED queries correctly.
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> Ultimately, I may need some help making the necessary
> > > > change
> > > > > > to
> > > > > > >>>>>>>> IQv1
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>>> support this, but I don't think it's fundamentally
> > > > > > impossible,
> > > > > > >>>>>>>> if we
> > > > > > >>>>>>>>>>>> want
> > > > > > >>>>>>>>>>>>> to pursue this route.
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> 3b.
> > > > > > >>>>>>>>>>>>> The main reason I chose to keep ALOS un-transactional
> > > > was to
> > > > > > >>>>>>>> minimize
> > > > > > >>>>>>>>>>>>> behavioural change for most users (I believe most
> > Streams
> > > > > > users
> > > > > > >>>>>>>> use
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>> default configuration, which is ALOS). That said,
> > it's
> > > > clear
> > > > > > >>>>>>>> that if
> > > > > > >>>>>>>>>>>> ALOS
> > > > > > >>>>>>>>>>>>> also used transactional stores, the only change in
> > > > behaviour
> > > > > > >>>>>>>> would be
> > > > > > >>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>> it would become *more correct*, which could be
> > > > considered a
> > > > > > >> "bug
> > > > > > >>>>>>>> fix"
> > > > > > >>>>>>>>>> by
> > > > > > >>>>>>>>>>>>> users, rather than a change they need to handle.
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> I believe that performance using transactions (aka.
> > > > RocksDB
> > > > > > >>>>>>>>>>>> WriteBatches)
> > > > > > >>>>>>>>>>>>> should actually be *better* than the un-batched
> > > > write-path
> > > > > > that
> > > > > > >>>>>>>> is
> > > > > > >>>>>>>>>>>>> currently used[1]. The only "performance"
> > consideration
> > > > will
> > > > > > be
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>>> increased memory usage that transactions require.
> > Given
> > > > the
> > > > > > >>>>>>>>>> mitigations
> > > > > > >>>>>>>>>>>> for
> > > > > > >>>>>>>>>>>>> this memory that we have in place, I would expect
> > that
> > > > this
> > > > > > is
> > > > > > >>>>>>>> not a
> > > > > > >>>>>>>>>>>>> problem for most users.
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> If we're happy to do so, we can make ALOS also use
> > > > > > >> transactions.
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> Regards,
> > > > > > >>>>>>>>>>>>> Nick
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> Link 1:
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>
> > > > > > >>
> > > >
> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> > > > > > >>>> cadonna@apache.org
> > > > > > >>>>>>>>>
> > > > > > >>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> Hi Nick,
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> Thanks for the updates and sorry for the delay on my
> > > > side!
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> 1.
> > > > > > >>>>>>>>>>>>>> Making the default implementation for flush() a
> > no-op
> > > > sounds
> > > > > > >>>>>>>> good to
> > > > > > >>>>>>>>>>>> me.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> 2.
> > > > > > >>>>>>>>>>>>>> I think what was bugging me here is that a
> > third-party
> > > > state
> > > > > > >>>>>>>> store
> > > > > > >>>>>>>>>>>> needs
> > > > > > >>>>>>>>>>>>>> to implement the state store interface. That means
> > they
> > > > need
> > > > > > >> to
> > > > > > >>>>>>>>>>>>>> implement a wrapper around the actual state store
> > as we
> > > > do
> > > > > > for
> > > > > > >>>>>>>>>> RocksDB
> > > > > > >>>>>>>>>>>>>> with RocksDBStore. So, a third-party state store can
> > > > always
> > > > > > >>>>>>>> estimate
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>> uncommitted bytes, if it wants, because the wrapper
> > can
> > > > > > record
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>> added
> > > > > > >>>>>>>>>>>>>> bytes.
> > > > > > >>>>>>>>>>>>>> One case I can think of where returning -1 makes
> > sense
> > > > is
> > > > > > when
> > > > > > >>>>>>>>>> Streams
> > > > > > >>>>>>>>>>>>>> does not need to estimate the size of the write
> > batch
> > > > and
> > > > > > >>>>>>>> trigger
> > > > > > >>>>>>>>>>>>>> extraordinary commits, because the third-party state
> > > > store
> > > > > > >>>>>>>> takes care
> > > > > > >>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>> memory. But in that case the method could also just
> > > > return
> > > > > > 0.
> > > > > > >>>>>>>> Even
> > > > > > >>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>> case would be better solved with a method that
> > returns
> > > > > > whether
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>> state
> > > > > > >>>>>>>>>>>>>> store manages itself the memory used for uncommitted
> > > > bytes
> > > > > > or
> > > > > > >>>>>>>> not.
> > > > > > >>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return
> > value,
> > > > I was
> > > > > > >>>>>>>> just
> > > > > > >>>>>>>>>>>>>> wondering when and if it will be used.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> Regarding returning 0 for transactional state stores
> > > > when
> > > > > > the
> > > > > > >>>>>>>> batch
> > > > > > >>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>> empty, I was just wondering because you explicitly
> > > > stated
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> "or {@code 0} if this StateStore does not support
> > > > > > >>>> transactions."
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> So it seemed to me returning 0 could only happen for
> > > > > > >>>>>>>>>> non-transactional
> > > > > > >>>>>>>>>>>>>> state stores.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> 3.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> a) What do you think if we move the isolation level
> > to
> > > > IQ
> > > > > > (v1
> > > > > > >>>>>>>> and
> > > > > > >>>>>>>>>> v2)?
> > > > > > >>>>>>>>>>>>>> In the end this is the only component that really
> > needs
> > > > to
> > > > > > >>>>>>>> specify
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>> isolation level. It is similar to the Kafka consumer
> > > > that
> > > > > > can
> > > > > > >>>>>>>> choose
> > > > > > >>>>>>>>>>>>>> with what isolation level to read the input topic.
> > > > > > >>>>>>>>>>>>>> For IQv1 the isolation level should go into
> > > > > > >>>>>>>> StoreQueryParameters. For
> > > > > > >>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> b) Point a) raises the question what should happen
> > > > during
> > > > > > >>>>>>>>>> at-least-once
> > > > > > >>>>>>>>>>>>>> processing when the state store does not use
> > > > transactions?
> > > > > > >> John
> > > > > > >>>>>>>> in
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>> past proposed to also use transactions on state
> > stores
> > > > for
> > > > > > >>>>>>>>>>>>>> at-least-once. I like that idea, because it avoids
> > > > > > aggregating
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>> same
> > > > > > >>>>>>>>>>>>>> records over and over again in the case of a
> > failure. We
> > > > > > had a
> > > > > > >>>>>>>> case
> > > > > > >>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>> the past where a Streams applications in
> > at-least-once
> > > > mode
> > > > > > >> was
> > > > > > >>>>>>>>>> failing
> > > > > > >>>>>>>>>>>>>> continuously for some reasons I do not remember
> > before
> > > > > > >>>>>>>> committing the
> > > > > > >>>>>>>>>>>>>> offsets. After each failover, the app aggregated
> > again
> > > > and
> > > > > > >>>>>>>> again the
> > > > > > >>>>>>>>>>>>>> same records. Of course the aggregate increased to
> > very
> > > > > > wrong
> > > > > > >>>>>>>> values
> > > > > > >>>>>>>>>>>>>> just because of the failover. With transactions on
> > the
> > > > state
> > > > > > >>>>>>>> stores
> > > > > > >>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>> could have avoided this. The app would have output
> > the
> > > > same
> > > > > > >>>>>>>> aggregate
> > > > > > >>>>>>>>>>>>>> multiple times (i.e., after each failover) but at
> > least
> > > > the
> > > > > > >>>>>>>> value of
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>> aggregate would not depend on the number of
> > failovers.
> > > > > > >>>>>>>> Outputting the
> > > > > > >>>>>>>>>>>>>> same aggregate multiple times would be incorrect
> > under
> > > > > > >>>>>>>> exactly-once
> > > > > > >>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>> it is OK for at-least-once.
> > > > > > >>>>>>>>>>>>>> If it makes sense to add a config to turn on and off
> > > > > > >>>>>>>> transactions on
> > > > > > >>>>>>>>>>>>>> state stores under at-least-once or just use
> > > > transactions in
> > > > > > >>>>>>>> any case
> > > > > > >>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>> a question we should also discuss in this KIP. It
> > > > depends a
> > > > > > >> bit
> > > > > > >>>>>>>> on
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would
> > add a
> > > > > > config.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> 4.
> > > > > > >>>>>>>>>>>>>> Your points are all valid. I tend to say to keep the
> > > > metrics
> > > > > > >>>>>>>> around
> > > > > > >>>>>>>>>>>>>> flush() until we remove flush() completely from the
> > > > > > interface.
> > > > > > >>>>>>>> Calls
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>> flush() might still exist since existing processors
> > > > might
> > > > > > >> still
> > > > > > >>>>>>>> call
> > > > > > >>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For
> > sure, we
> > > > need
> > > > > > >> to
> > > > > > >>>>>>>>>>>> document
> > > > > > >>>>>>>>>>>>>> how the metrics change due to the transactions in
> > the
> > > > > > upgrade
> > > > > > >>>>>>>> notes.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> 5.
> > > > > > >>>>>>>>>>>>>> I see. Then you should describe how the .position
> > files
> > > > are
> > > > > > >>>>>>>> handled
> > > > > > >>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>> a dedicated section of the KIP or incorporate the
> > > > > > description
> > > > > > >>>>>>>> in the
> > > > > > >>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only
> > > > mentioning it
> > > > > > >> in
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>>>> "Compatibility, 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:
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > >
> > https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> It is a bit brittle, but it works.
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> Best,
> > > > > > >>>>>>>>>>>>>> Bruno
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > > > > > >>>>>>>>>>>>>>> Hi Bruno,
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Thanks for taking the time to review the KIP. I'm
> > back
> > > > from
> > > > > > >>>>>>>> leave
> > > > > > >>>>>>>>>> now
> > > > > > >>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>> intend to move this forwards as quickly as I can.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Addressing your points:
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> 1.
> > > > > > >>>>>>>>>>>>>>> Because flush() is part of the StateStore API, it's
> > > > exposed
> > > > > > >> to
> > > > > > >>>>>>>>>> custom
> > > > > > >>>>>>>>>>>>>>> Processors, which might be making calls to flush().
> > > > This
> > > > > > was
> > > > > > >>>>>>>>>> actually
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> case in a few integration tests.
> > > > > > >>>>>>>>>>>>>>> To maintain as much compatibility as possible, I'd
> > > > prefer
> > > > > > not
> > > > > > >>>>>>>> to
> > > > > > >>>>>>>>>> make
> > > > > > >>>>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>> an UnsupportedOperationException, as it will cause
> > > > > > previously
> > > > > > >>>>>>>>>> working
> > > > > > >>>>>>>>>>>>>>> Processors to start throwing exceptions at runtime.
> > > > > > >>>>>>>>>>>>>>> I agree that it doesn't make sense for it to proxy
> > > > > > commit(),
> > > > > > >>>>>>>> though,
> > > > > > >>>>>>>>>>>> as
> > > > > > >>>>>>>>>>>>>>> that would cause it to violate the "StateStores
> > commit
> > > > only
> > > > > > >>>>>>>> when the
> > > > > > >>>>>>>>>>>> Task
> > > > > > >>>>>>>>>>>>>>> commits" rule.
> > > > > > >>>>>>>>>>>>>>> Instead, I think we should make this a no-op. That
> > way,
> > > > > > >>>>>>>> existing
> > > > > > >>>>>>>>>> user
> > > > > > >>>>>>>>>>>>>>> Processors will continue to work as-before, without
> > > > > > violation
> > > > > > >>>>>>>> of
> > > > > > >>>>>>>>>> store
> > > > > > >>>>>>>>>>>>>>> consistency that would be caused by premature
> > > > flush/commit
> > > > > > of
> > > > > > >>>>>>>>>>>> StateStore
> > > > > > >>>>>>>>>>>>>>> data to disk.
> > > > > > >>>>>>>>>>>>>>> What do you think?
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> 2.
> > > > > > >>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
> > > > implementation
> > > > > > is
> > > > > > >>>>>>>>>>>>>>> transactional, but is unable to estimate the
> > > > uncommitted
> > > > > > >>>> memory
> > > > > > >>>>>>>>>> usage,
> > > > > > >>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> method will return -1.
> > > > > > >>>>>>>>>>>>>>> The intention here is to permit third-party
> > > > implementations
> > > > > > >>>>>>>> that may
> > > > > > >>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>> able to estimate memory usage.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been written to
> > the
> > > > > > store
> > > > > > >>>>>>>> yet. I
> > > > > > >>>>>>>>>>>>>> thought
> > > > > > >>>>>>>>>>>>>>> that was implied by "This method will return an
> > > > > > approximation
> > > > > > >>>>>>>> of the
> > > > > > >>>>>>>>>>>>>> memory
> > > > > > >>>>>>>>>>>>>>> would be freed by the next call to {@link
> > > > #commit(Map)}"
> > > > > > and
> > > > > > >>>>>>>>>> "@return
> > > > > > >>>>>>>>>>>> The
> > > > > > >>>>>>>>>>>>>>> approximate size of all records awaiting {@link
> > > > > > >>>> #commit(Map)}",
> > > > > > >>>>>>>>>>>> however,
> > > > > > >>>>>>>>>>>>>> I
> > > > > > >>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you think
> > this
> > > > is
> > > > > > >>>>>>>> unclear?
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> 3.
> > > > > > >>>>>>>>>>>>>>> I realise this is probably the most contentious
> > point
> > > > in my
> > > > > > >>>>>>>> design,
> > > > > > >>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>> I'm
> > > > > > >>>>>>>>>>>>>>> open to changing it if I'm unable to convince you
> > of
> > > > the
> > > > > > >>>>>>>> benefits.
> > > > > > >>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > > > > > >>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly
> > provided
> > > > > > >>>>>>>> StateStores
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>> query, and it may be important for users to
> > > > > > programmatically
> > > > > > >>>>>>>> know
> > > > > > >>>>>>>>>>>> which
> > > > > > >>>>>>>>>>>>>>> mode the StateStore is operating under. If we
> > simply
> > > > > > provide
> > > > > > >>>> an
> > > > > > >>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the
> > internal
> > > > > > streams
> > > > > > >>>>>>>>>>>> engine), or
> > > > > > >>>>>>>>>>>>>>> similar, then users will need to understand the
> > > > operation
> > > > > > and
> > > > > > >>>>>>>>>>>>>> consequences
> > > > > > >>>>>>>>>>>>>>> of each available processing mode and how it
> > pertains
> > > > to
> > > > > > >> their
> > > > > > >>>>>>>>>>>>>> StateStore.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Interactive Query users aren't the only people that
> > > > care
> > > > > > >> about
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
> > > > > > implementers
> > > > > > >>>> of
> > > > > > >>>>>>>>>> custom
> > > > > > >>>>>>>>>>>>>>> StateStores also need to understand the behaviour
> > > > expected
> > > > > > of
> > > > > > >>>>>>>> their
> > > > > > >>>>>>>>>>>>>>> implementation. KIP-892 introduces some assumptions
> > > > into
> > > > > > the
> > > > > > >>>>>>>> Streams
> > > > > > >>>>>>>>>>>>>> Engine
> > > > > > >>>>>>>>>>>>>>> about how StateStores operate under each processing
> > > > mode,
> > > > > > and
> > > > > > >>>>>>>> it's
> > > > > > >>>>>>>>>>>>>>> important that custom implementations adhere to
> > those
> > > > > > >>>>>>>> assumptions in
> > > > > > >>>>>>>>>>>>>> order
> > > > > > >>>>>>>>>>>>>>> to maintain the consistency guarantees.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> IsolationLevels provide a high-level contract on
> > the
> > > > > > >> behaviour
> > > > > > >>>>>>>> of
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>> StateStore: a user knows that under READ_COMMITTED,
> > > > they
> > > > > > will
> > > > > > >>>>>>>> see
> > > > > > >>>>>>>>>>>> writes
> > > > > > >>>>>>>>>>>>>>> only after the Task has committed, and under
> > > > > > READ_UNCOMMITTED
> > > > > > >>>>>>>> they
> > > > > > >>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>> see
> > > > > > >>>>>>>>>>>>>>> writes immediately. No understanding of the
> > details of
> > > > each
> > > > > > >>>>>>>>>>>>>> processing.mode
> > > > > > >>>>>>>>>>>>>>> is required, either for IQ users or StateStore
> > > > > > implementers.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> An argument can be made that these contractual
> > > > guarantees
> > > > > > can
> > > > > > >>>>>>>> simply
> > > > > > >>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>> documented for the processing.mode (i.e. that
> > > > exactly-once
> > > > > > >> and
> > > > > > >>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
> > > > > > at-least-once
> > > > > > >>>>>>>> behaves
> > > > > > >>>>>>>>>>>> like
> > > > > > >>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small
> > issues
> > > > with
> > > > > > >>>>>>>> this I'd
> > > > > > >>>>>>>>>>>>>> prefer
> > > > > > >>>>>>>>>>>>>>> to avoid:
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>         - Where would we document these contracts,
> > in
> > > > a way
> > > > > > >> that
> > > > > > >>>>>>>> is
> > > > > > >>>>>>>>>>>> difficult
> > > > > > >>>>>>>>>>>>>>>         for users/implementers to miss/ignore?
> > > > > > >>>>>>>>>>>>>>>         - It's not clear to users that the
> > processing
> > > > mode
> > > > > > is
> > > > > > >>>>>>>>>>>> communicating
> > > > > > >>>>>>>>>>>>>>>         an expectation of read isolation, unless
> > they
> > > > read
> > > > > > the
> > > > > > >>>>>>>>>>>>>> documentation. Users
> > > > > > >>>>>>>>>>>>>>>         rarely consult documentation unless they
> > feel
> > > > they
> > > > > > >> need
> > > > > > >>>>>>>> to, so
> > > > > > >>>>>>>>>>>> it's
> > > > > > >>>>>>>>>>>>>> likely
> > > > > > >>>>>>>>>>>>>>>         this detail would get missed by many users.
> > > > > > >>>>>>>>>>>>>>>         - It tightly couples processing modes to
> > read
> > > > > > >> isolation.
> > > > > > >>>>>>>> Adding
> > > > > > >>>>>>>>>>>> new
> > > > > > >>>>>>>>>>>>>>>         processing modes, or changing the read
> > > > isolation of
> > > > > > >>>>>>>> existing
> > > > > > >>>>>>>>>>>>>> processing
> > > > > > >>>>>>>>>>>>>>>         modes would be difficult/impossible.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Ultimately, the cost of introducing
> > IsolationLevels is
> > > > > > just a
> > > > > > >>>>>>>> single
> > > > > > >>>>>>>>>>>>>>> method, since we re-use the existing IsolationLevel
> > > > enum
> > > > > > from
> > > > > > >>>>>>>> Kafka.
> > > > > > >>>>>>>>>>>> This
> > > > > > >>>>>>>>>>>>>>> gives us a clear place to document the contractual
> > > > > > guarantees
> > > > > > >>>>>>>>>> expected
> > > > > > >>>>>>>>>>>>>>> of/provided by StateStores, that is accessible
> > both by
> > > > the
> > > > > > >>>>>>>>>> StateStore
> > > > > > >>>>>>>>>>>>>>> itself, and by IQ users.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> (Writing this I've just realised that the
> > StateStore
> > > > and IQ
> > > > > > >>>>>>>> APIs
> > > > > > >>>>>>>>>>>> actually
> > > > > > >>>>>>>>>>>>>>> don't provide access to StateStoreContext that IQ
> > users
> > > > > > would
> > > > > > >>>>>>>> have
> > > > > > >>>>>>>>>>>> direct
> > > > > > >>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> > > > > > >> isolationLevel()
> > > > > > >>>>>>>>>> itself
> > > > > > >>>>>>>>>>>>>> too?)
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> 4.
> > > > > > >>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics
> > in-place
> > > > > > >>>>>>>> either, as
> > > > > > >>>>>>>>>>>> it's a
> > > > > > >>>>>>>>>>>>>>> backwards incompatible change. My concern is that,
> > if
> > > > we
> > > > > > >> leave
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>>>> existing
> > > > > > >>>>>>>>>>>>>>> "flush" metrics in place, they will be confusing to
> > > > users.
> > > > > > >>>>>>>> Right
> > > > > > >>>>>>>>>> now,
> > > > > > >>>>>>>>>>>>>>> "flush" metrics record explicit flushes to disk,
> > but
> > > > under
> > > > > > >>>>>>>> KIP-892,
> > > > > > >>>>>>>>>>>> even
> > > > > > >>>>>>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>> commit() will not explicitly flush data to disk -
> > > > RocksDB
> > > > > > >> will
> > > > > > >>>>>>>>>> decide
> > > > > > >>>>>>>>>>>> on
> > > > > > >>>>>>>>>>>>>>> when to flush memtables to disk itself.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd have
> > two
> > > > > > >>>> options,
> > > > > > >>>>>>>>>> which
> > > > > > >>>>>>>>>>>>>> both
> > > > > > >>>>>>>>>>>>>>> seem pretty bad to me:
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>         1. Have them record calls to commit(),
> > which
> > > > would
> > > > > > be
> > > > > > >>>>>>>>>>>> misleading, as
> > > > > > >>>>>>>>>>>>>>>         data is no longer explicitly "flushed" to
> > disk
> > > > by
> > > > > > this
> > > > > > >>>>>>>> call.
> > > > > > >>>>>>>>>>>>>>>         2. Have them record nothing at all, which
> > is
> > > > > > >> equivalent
> > > > > > >>>> to
> > > > > > >>>>>>>>>>>> removing
> > > > > > >>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>         metrics, except that users will see the
> > metric
> > > > > > still
> > > > > > >>>>>>>> exists and
> > > > > > >>>>>>>>>>>> so
> > > > > > >>>>>>>>>>>>>> assume
> > > > > > >>>>>>>>>>>>>>>         that the metric is correct, and that
> > there's a
> > > > > > problem
> > > > > > >>>>>>>> with
> > > > > > >>>>>>>>>> their
> > > > > > >>>>>>>>>>>>>> system
> > > > > > >>>>>>>>>>>>>>>         when there isn't.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> I agree that removing them is also a bad solution,
> > and
> > > > I'd
> > > > > > >>>>>>>> like some
> > > > > > >>>>>>>>>>>>>>> guidance on the best path forward here.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> 5.
> > > > > > >>>>>>>>>>>>>>> Position files are updated on every write to a
> > > > StateStore.
> > > > > > >>>>>>>> Since our
> > > > > > >>>>>>>>>>>>>> writes
> > > > > > >>>>>>>>>>>>>>> are now buffered until commit(), we can't update
> > the
> > > > > > Position
> > > > > > >>>>>>>> file
> > > > > > >>>>>>>>>>>> until
> > > > > > >>>>>>>>>>>>>>> commit() has been called, otherwise it would be
> > > > > > inconsistent
> > > > > > >>>>>>>> with
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>> data
> > > > > > >>>>>>>>>>>>>>> in the event of a rollback. Consequently, we need
> > to
> > > > manage
> > > > > > >>>>>>>> these
> > > > > > >>>>>>>>>>>> offsets
> > > > > > >>>>>>>>>>>>>>> the same way we manage the checkpoint offsets, and
> > > > ensure
> > > > > > >>>>>>>> they're
> > > > > > >>>>>>>>>> only
> > > > > > >>>>>>>>>>>>>>> written on commit().
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> 6.
> > > > > > >>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what
> > tests to
> > > > > > >> write.
> > > > > > >>>>>>>> How
> > > > > > >>>>>>>>>>>>>> explicit
> > > > > > >>>>>>>>>>>>>>> do we need to be here in the KIP?
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be
> > > > > > seamless,
> > > > > > >>>>>>>> and we
> > > > > > >>>>>>>>>>>>>> should
> > > > > > >>>>>>>>>>>>>>> definitely add some tests around that. Downgrade,
> > it
> > > > > > >>>>>>>> transpires,
> > > > > > >>>>>>>>>> isn't
> > > > > > >>>>>>>>>>>>>>> currently possible, as the extra column family for
> > > > offset
> > > > > > >>>>>>>> storage is
> > > > > > >>>>>>>>>>>>>>> incompatible with the pre-KIP-892 implementation:
> > when
> > > > you
> > > > > > >>>>>>>> open a
> > > > > > >>>>>>>>>>>> RocksDB
> > > > > > >>>>>>>>>>>>>>> database, you must open all available column
> > families
> > > > or
> > > > > > >>>>>>>> receive an
> > > > > > >>>>>>>>>>>>>> error.
> > > > > > >>>>>>>>>>>>>>> What currently happens on downgrade is that it
> > > > attempts to
> > > > > > >>>>>>>> open the
> > > > > > >>>>>>>>>>>>>> store,
> > > > > > >>>>>>>>>>>>>>> throws an error about the offsets column family not
> > > > being
> > > > > > >>>>>>>> opened,
> > > > > > >>>>>>>>>>>> which
> > > > > > >>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given that
> > > > > > >> downgrades
> > > > > > >>>>>>>>>> should
> > > > > > >>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>> uncommon, I think this is acceptable behaviour, as
> > the
> > > > > > >>>>>>>> end-state is
> > > > > > >>>>>>>>>>>>>>> consistent, even if it results in an undesirable
> > state
> > > > > > >>>> restore.
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Should I document the upgrade/downgrade behaviour
> > > > > > explicitly
> > > > > > >>>>>>>> in the
> > > > > > >>>>>>>>>>>> KIP?
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> --
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> Regards,
> > > > > > >>>>>>>>>>>>>>> Nick
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > > > > > >>>>>>>> cadonna@apache.org>
> > > > > > >>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> Hi Nick!
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> Thanks for the updates!
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> 1.
> > > > > > >>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> > > > > > >>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > > > > > >>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore
> > after
> > > > this
> > > > > > KIP
> > > > > > >>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>> released, I would rather throw an unsupported
> > > > operation
> > > > > > >>>>>>>> exception
> > > > > > >>>>>>>>>> by
> > > > > > >>>>>>>>>>>>>>>> default.
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> 2.
> > > > > > >>>>>>>>>>>>>>>> When would a state store return -1 from
> > > > > > >>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while
> > > > being
> > > > > > >>>>>>>>>>>> transactional?
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> Wouldn't
> > StateStore#approximateNumUncommittedBytes()
> > > > also
> > > > > > >>>>>>>> return 0
> > > > > > >>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>> the state store is transactional but nothing has
> > been
> > > > > > >> written
> > > > > > >>>>>>>> to
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>> state store yet?
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> 3.
> > > > > > >>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this KIP
> > really
> > > > > > need
> > > > > > >>>> to
> > > > > > >>>>>>>>>>>> introduce
> > > > > > >>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> > StateStoreContext
> > > > has
> > > > > > >>>>>>>> already
> > > > > > >>>>>>>>>>>>>>>> appConfigs() which basically exposes the same
> > > > information,
> > > > > > >>>>>>>> i.e., if
> > > > > > >>>>>>>>>>>> EOS
> > > > > > >>>>>>>>>>>>>>>> is enabled or not.
> > > > > > >>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore
> > interface
> > > > as
> > > > > > >>>>>>>> loosely
> > > > > > >>>>>>>>>>>> coupled
> > > > > > >>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > > > implementers
> > > > > > >>>> more
> > > > > > >>>>>>>>>>>> freedom,
> > > > > > >>>>>>>>>>>>>>>> and reduce the amount of internal knowledge
> > required."
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> While I understand the intent, I doubt that it
> > > > decreases
> > > > > > the
> > > > > > >>>>>>>>>>>> coupling of
> > > > > > >>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> > > > > > >> READ_COMMITTED
> > > > > > >>>>>>>> only
> > > > > > >>>>>>>>>>>>>>>> applies to IQ but not to reads by processors.
> > Thus,
> > > > > > >>>>>>>> implementers
> > > > > > >>>>>>>>>>>> need to
> > > > > > >>>>>>>>>>>>>>>> understand how Streams accesses the state stores.
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> I would like to hear what others think about this.
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> 4.
> > > > > > >>>>>>>>>>>>>>>> Great exposing new metrics for transactional state
> > > > stores!
> > > > > > >>>>>>>>>> However, I
> > > > > > >>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate (in
> > the
> > > > > > docs)
> > > > > > >>>>>>>> the 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 also need to adapt/add integration
> > > > tests
> > > > > > >>>>>>>> besides
> > > > > > >>>>>>>>>> unit
> > > > > > >>>>>>>>>>>>>>>> tests. Additionally, we probably need integration
> > or
> > > > > > system
> > > > > > >>>>>>>> tests
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>> verify that upgrades and downgrades between
> > > > transactional
> > > > > > >> and
> > > > > > >>>>>>>>>>>>>>>> non-transactional state stores work as expected.
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> Best,
> > > > > > >>>>>>>>>>>>>>>> Bruno
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > > > > > >>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in the
> > KIP,
> > > > > > under
> > > > > > >>>>>>>>>>>> "Rejected
> > > > > > >>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea worth
> > > > pursuing,
> > > > > > >>>>>>>> but I
> > > > > > >>>>>>>>>>>> believe
> > > > > > >>>>>>>>>>>>>>>>> that it's out of the scope of this KIP, because
> > it
> > > > > > solves a
> > > > > > >>>>>>>>>>>> different
> > > > > > >>>>>>>>>>>>>> set
> > > > > > >>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of this
> > one
> > > > has
> > > > > > >>>>>>>> already
> > > > > > >>>>>>>>>> grown
> > > > > > >>>>>>>>>>>>>>>> quite
> > > > > > >>>>>>>>>>>>>>>>> large!
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > > > > > >>>>>>>>>> nick.telford@gmail.com>
> > > > > > >>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> Hi everyone,
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> I've updated the KIP (
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > > > >>>>>>>>>>>>>>>> )
> > > > > > >>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing back
> > > > "Atomic
> > > > > > >>>>>>>>>>>> Checkpointing"
> > > > > > >>>>>>>>>>>>>>>> (for
> > > > > > >>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think the one
> > > > thing
> > > > > > >>>>>>>> missing is
> > > > > > >>>>>>>>>>>> some
> > > > > > >>>>>>>>>>>>>>>>>> changes to metrics (notably the store "flush"
> > > > metrics
> > > > > > will
> > > > > > >>>>>>>> need
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>> renamed to "commit").
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> The reason I brought back Atomic Checkpointing
> > was
> > > > to
> > > > > > >>>>>>>> decouple
> > > > > > >>>>>>>>>>>> store
> > > > > > >>>>>>>>>>>>>>>> flush
> > > > > > >>>>>>>>>>>>>>>>>> from store commit. This is important, because
> > with
> > > > > > >>>>>>>> Transactional
> > > > > > >>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on
> > *every*
> > > > Task
> > > > > > >>>>>>>> commit,
> > > > > > >>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>> just when the StateStore is closing, otherwise
> > our
> > > > > > >>>>>>>> transaction
> > > > > > >>>>>>>>>>>> buffer
> > > > > > >>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>> never be written and persisted, instead growing
> > > > > > unbounded!
> > > > > > >>>> I
> > > > > > >>>>>>>>>>>>>>>> experimented
> > > > > > >>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a store
> > > > flush
> > > > > > >>>>>>>> whenever
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its
> > > > configured
> > > > > > >>>>>>>> size, but
> > > > > > >>>>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>> was
> > > > > > >>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer
> > from
> > > > being
> > > > > > >>>>>>>>>> configured
> > > > > > >>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>> unbounded, and it still would have required
> > explicit
> > > > > > >>>>>>>> flushes of
> > > > > > >>>>>>>>>>>>>> RocksDB,
> > > > > > >>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory
> > > > utilization.
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right"
> > way
> > > > to
> > > > > > >>>>>>>> resolve
> > > > > > >>>>>>>>>> this
> > > > > > >>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog offsets
> > that
> > > > > > >>>>>>>> correspond
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>> most
> > > > > > >>>>>>>>>>>>>>>>>> recently written records are always atomically
> > > > written
> > > > > > to
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>>>> StateStore
> > > > > > >>>>>>>>>>>>>>>>>> (by writing them to the same transaction
> > buffer),
> > > > we can
> > > > > > >>>>>>>> avoid
> > > > > > >>>>>>>>>>>>>> forcibly
> > > > > > >>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting
> > > > RocksDB
> > > > > > >>>>>>>> flush
> > > > > > >>>>>>>>>> them
> > > > > > >>>>>>>>>>>>>> only
> > > > > > >>>>>>>>>>>>>>>>>> when necessary, without losing any of our
> > > > consistency
> > > > > > >>>>>>>> guarantees.
> > > > > > >>>>>>>>>>>> See
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>> updated KIP for more info.
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> I have fully implemented these changes,
> > although I'm
> > > > > > still
> > > > > > >>>>>>>> not
> > > > > > >>>>>>>>>>>>>> entirely
> > > > > > >>>>>>>>>>>>>>>>>> happy with the implementation for segmented
> > > > StateStores,
> > > > > > >> so
> > > > > > >>>>>>>> I
> > > > > > >>>>>>>>>> plan
> > > > > > >>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests pass. If
> > > > you'd
> > > > > > like
> > > > > > >>>>>>>> to try
> > > > > > >>>>>>>>>>>> out
> > > > > > >>>>>>>>>>>>>> or
> > > > > > >>>>>>>>>>>>>>>>>> review this highly experimental and incomplete
> > > > branch,
> > > > > > >> it's
> > > > > > >>>>>>>>>>>> available
> > > > > > >>>>>>>>>>>>>>>> here:
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > > > > > .
> > > > > > >>>>>>>> Note:
> > > > > > >>>>>>>>>>>> it's
> > > > > > >>>>>>>>>>>>>>>> built
> > > > > > >>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base
> > to
> > > > build
> > > > > > >>>>>>>> and test
> > > > > > >>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>> on,
> > > > > > >>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples comparisons
> > in a
> > > > > > live
> > > > > > >>>>>>>>>>>>>> environment. I
> > > > > > >>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's nearer
> > > > > > >> completion
> > > > > > >>>>>>>> and
> > > > > > >>>>>>>>>> has
> > > > > > >>>>>>>>>>>>>> been
> > > > > > >>>>>>>>>>>>>>>>>> proven on our main application.
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing and
> > > > testing:
> > > > > > >>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window)
> > stores
> > > > > > >>>>>>>>>>>>>>>>>> - Global stores
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> As I do not currently use either of these, so my
> > > > primary
> > > > > > >>>>>>>> test
> > > > > > >>>>>>>>>>>>>>>> environment
> > > > > > >>>>>>>>>>>>>>>>>> doesn't test these areas.
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next week
> > for
> > > > a few
> > > > > > >>>>>>>> weeks,
> > > > > > >>>>>>>>>> so
> > > > > > >>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>> not have time to move this forward until late
> > > > August.
> > > > > > That
> > > > > > >>>>>>>> said,
> > > > > > >>>>>>>>>>>> your
> > > > > > >>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just
> > won't be
> > > > > > able
> > > > > > >>>> to
> > > > > > >>>>>>>>>>>> respond
> > > > > > >>>>>>>>>>>>>> as
> > > > > > >>>>>>>>>>>>>>>>>> quickly as usual.
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> Regards,
> > > > > > >>>>>>>>>>>>>>>>>> Nick
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > > > > > >>>>>>>>>> nick.telford@gmail.com>
> > > > > > >>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> Hi Bruno
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ
> > is
> > > > not
> > > > > > >>>>>>>> something
> > > > > > >>>>>>>>>> I
> > > > > > >>>>>>>>>>>> had
> > > > > > >>>>>>>>>>>>>>>>>>> considered.
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> What about atomically updating the state store
> > > > from the
> > > > > > >>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> > > > > > checkpoint
> > > > > > >>>>>>>> (thus,
> > > > > > >>>>>>>>>>>>>>>> flushing
> > > > > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data
> > > > and/or
> > > > > > >>>>>>>> number of
> > > > > > >>>>>>>>>>>>>> commit
> > > > > > >>>>>>>>>>>>>>>>>>>> intervals?
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting
> > > > that we
> > > > > > >>>>>>>> add an
> > > > > > >>>>>>>>>>>>>>>> additional
> > > > > > >>>>>>>>>>>>>>>>>>> config for the max number of commit intervals
> > > > between
> > > > > > >>>>>>>>>> checkpoints?
> > > > > > >>>>>>>>>>>>>> That
> > > > > > >>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the
> > > > transaction
> > > > > > >>>>>>>> buffers
> > > > > > >>>>>>>>>> are
> > > > > > >>>>>>>>>>>>>>>> nearly
> > > > > > >>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of commit
> > > > > > intervals
> > > > > > >>>>>>>> have
> > > > > > >>>>>>>>>>>>>> elapsed,
> > > > > > >>>>>>>>>>>>>>>>>>> whichever comes first?
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although this
> > > > > > re-ignites
> > > > > > >>>>>>>> an
> > > > > > >>>>>>>>>>>> earlier
> > > > > > >>>>>>>>>>>>>>>>>>> debate about whether a config should be
> > measured in
> > > > > > >>>>>>>> "number of
> > > > > > >>>>>>>>>>>> commit
> > > > > > >>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute time.
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the reason
> > I
> > > > was
> > > > > > >>>>>>>> pursuing
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>> Atomic
> > > > > > >>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush
> > from
> > > > > > >>>>>>>> checkpointing,
> > > > > > >>>>>>>>>>>>>> which
> > > > > > >>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every commit
> > > > without
> > > > > > any
> > > > > > >>>>>>>>>>>> performance
> > > > > > >>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the
> > > > "best"
> > > > > > >>>>>>>> solution,
> > > > > > >>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>> I'm not
> > > > > > >>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back into
> > this
> > > > KIP.
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
> > > > transactional
> > > > > > >>>> logic
> > > > > > >>>>>>>>>>>> directly
> > > > > > >>>>>>>>>>>>>>>> into
> > > > > > >>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with the
> > > > > > >>>>>>>>>>>>>> StateStore#newTransaction
> > > > > > >>>>>>>>>>>>>>>>>>> method, and reduces the number of new classes
> > > > > > introduced,
> > > > > > >>>>>>>>>>>>>> significantly
> > > > > > >>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and the
> > > > > > complexity
> > > > > > >>>> is
> > > > > > >>>>>>>>>>>>>> drastically
> > > > > > >>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic
> > Checkpoints
> > > > > > into
> > > > > > >>>>>>>> this
> > > > > > >>>>>>>>>> KIP.
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> Regards,
> > > > > > >>>>>>>>>>>>>>>>>>> Nick
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
> > > > > > >>>>>>>> cadonna@apache.org>
> > > > > > >>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> As far as I understand, you want to atomically
> > > > update
> > > > > > >> the
> > > > > > >>>>>>>> state
> > > > > > >>>>>>>>>>>>>> store
> > > > > > >>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the
> > memtable
> > > > of a
> > > > > > >>>> state
> > > > > > >>>>>>>>>> store
> > > > > > >>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit time
> > > > elapsed
> > > > > > >>>> but
> > > > > > >>>>>>>>>> after
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that would
> > lead
> > > > to
> > > > > > >>>>>>>> exceeding
> > > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > before the
> > > > > > next
> > > > > > >>>>>>>> commit
> > > > > > >>>>>>>>>>>>>>>> interval
> > > > > > >>>>>>>>>>>>>>>>>>>> ends.
> > > > > > >>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would commit
> > > > every
> > > > > > >>>>>>>> commit
> > > > > > >>>>>>>>>>>> interval
> > > > > > >>>>>>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>>>>>> the state store will only be atomically
> > updated
> > > > > > roughly
> > > > > > >>>>>>>> every
> > > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of
> > data.
> > > > Also
> > > > > > IQ
> > > > > > >>>>>>>> would
> > > > > > >>>>>>>>>>>> then
> > > > > > >>>>>>>>>>>>>>>> only
> > > > > > >>>>>>>>>>>>>>>>>>>> see new data roughly every
> > > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > > > >>>>>>>>>>>>>>>>>>>> After a failure the state store needs to
> > restore
> > > > up to
> > > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> Is this correct?
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> What about atomically updating the state store
> > > > from
> > > > > > the
> > > > > > >>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> > > > > > checkpoint
> > > > > > >>>>>>>> (thus,
> > > > > > >>>>>>>>>>>>>>>> flushing
> > > > > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data
> > > > and/or
> > > > > > >>>>>>>> number of
> > > > > > >>>>>>>>>>>>>> commit
> > > > > > >>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have the
> > same
> > > > delay
> > > > > > >>>> for
> > > > > > >>>>>>>>>>>> records
> > > > > > >>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because both
> > > > would
> > > > > > >>>>>>>> appear
> > > > > > >>>>>>>>>> when
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However,
> > after a
> > > > > > failure
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>> state
> > > > > > >>>>>>>>>>>>>>>> store
> > > > > > >>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > > > >>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>> it might restore data that is already in the
> > state
> > > > > > store
> > > > > > >>>>>>>>>> because
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable offset
> > > > (i.e.
> > > > > > the
> > > > > > >>>>>>>> last
> > > > > > >>>>>>>>>>>>>> committed
> > > > > > >>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring
> > data
> > > > that
> > > > > > is
> > > > > > >>>>>>>> already
> > > > > > >>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should not
> > > > violated.
> > > > > > >>>>>>>>>>>>>>>>>>>> This solution needs at least one new config to
> > > > specify
> > > > > > >>>>>>>> when a
> > > > > > >>>>>>>>>>>>>>>> checkpoint
> > > > > > >>>>>>>>>>>>>>>>>>>> should be written.
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail
> > that
> > > > does
> > > > > > not
> > > > > > >>>>>>>> change
> > > > > > >>>>>>>>>>>>>>>> anything
> > > > > > >>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit
> > interval
> > > > is 30
> > > > > > >>>>>>>> seconds,
> > > > > > >>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>> five
> > > > > > >>>>>>>>>>>>>>>>>>>> seconds.
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> Best,
> > > > > > >>>>>>>>>>>>>>>>>>>> Bruno
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch on
> > our
> > > > > > staging
> > > > > > >>>>>>>>>>>>>> environment,
> > > > > > >>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
> > non-trivial
> > > > > > >>>>>>>> application.
> > > > > > >>>>>>>>>> I'm
> > > > > > >>>>>>>>>>>>>>>>>>>> already
> > > > > > >>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate that
> > we
> > > > saw
> > > > > > the
> > > > > > >>>>>>>> last
> > > > > > >>>>>>>>>>>> time
> > > > > > >>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP, but
> > this
> > > > > > time,
> > > > > > >> I
> > > > > > >>>>>>>>>> think I
> > > > > > >>>>>>>>>>>>>> know
> > > > > > >>>>>>>>>>>>>>>>>>>> why.
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is
> > > > called
> > > > > > at
> > > > > > >>>>>>>> the end
> > > > > > >>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> Task
> > > > > > >>>>>>>>>>>>>>>>>>>>> commit process, has the following behaviour:
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>           - Under ALOS: checkpoint the state
> > > > stores.
> > > > > > >> This
> > > > > > >>>>>>>>>> includes
> > > > > > >>>>>>>>>>>>>>>>>>>>>           flushing memtables in RocksDB.
> > This is
> > > > > > >>>> acceptable
> > > > > > >>>>>>>>>>>> because the
> > > > > > >>>>>>>>>>>>>>>>>>>> default
> > > > > > >>>>>>>>>>>>>>>>>>>>>           commit.interval.ms is 5 seconds,
> > so
> > > > > > forcibly
> > > > > > >>>>>>>> flushing
> > > > > > >>>>>>>>>>>>>> memtables
> > > > > > >>>>>>>>>>>>>>>>>>>> every 5
> > > > > > >>>>>>>>>>>>>>>>>>>>>           seconds is acceptable for most
> > > > > > applications.
> > > > > > >>>>>>>>>>>>>>>>>>>>>           - Under EOS: checkpointing is not
> > done,
> > > > > > >> *unless*
> > > > > > >>>>>>>> it's
> > > > > > >>>>>>>>>>>> being
> > > > > > >>>>>>>>>>>>>>>>>>>> forced, due
> > > > > > >>>>>>>>>>>>>>>>>>>>>           to e.g. the Task closing or being
> > > > revoked.
> > > > > > >> This
> > > > > > >>>>>>>> means
> > > > > > >>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>> under
> > > > > > >>>>>>>>>>>>>>>>>>>> normal
> > > > > > >>>>>>>>>>>>>>>>>>>>>           processing conditions, the state
> > stores
> > > > > > will
> > > > > > >> not
> > > > > > >>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>> checkpointed,
> > > > > > >>>>>>>>>>>>>>>>>>>> and will
> > > > > > >>>>>>>>>>>>>>>>>>>>>           not have memtables flushed at all ,
> > > > unless
> > > > > > >>>> RocksDB
> > > > > > >>>>>>>>>>>> decides to
> > > > > > >>>>>>>>>>>>>>>>>>>> flush them on
> > > > > > >>>>>>>>>>>>>>>>>>>>>           its own. Checkpointing stores and
> > > > > > >> force-flushing
> > > > > > >>>>>>>> their
> > > > > > >>>>>>>>>>>>>> memtables
> > > > > > >>>>>>>>>>>>>>>>>>>> is only
> > > > > > >>>>>>>>>>>>>>>>>>>>>           done when a Task is being closed.
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint
> > stores on
> > > > at
> > > > > > >>>> least
> > > > > > >>>>>>>>>> *some*
> > > > > > >>>>>>>>>>>>>>>> normal
> > > > > > >>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB
> > > > > > transaction
> > > > > > >>>>>>>>>> buffers
> > > > > > >>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets are
> > > > synced to
> > > > > > >>>>>>>> disk to
> > > > > > >>>>>>>>>>>>>> prevent
> > > > > > >>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
> > Consequently,
> > > > my
> > > > > > >>>>>>>> current
> > > > > > >>>>>>>>>>>>>>>>>>>> implementation
> > > > > > >>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit,
> > > > which
> > > > > > is
> > > > > > >>>>>>>> far too
> > > > > > >>>>>>>>>>>>>>>>>>>> frequent.
> > > > > > >>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records,
> > > > which
> > > > > > is
> > > > > > >> a
> > > > > > >>>>>>>>>> change
> > > > > > >>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>> flush
> > > > > > >>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing performance
> > > > problems
> > > > > > for
> > > > > > >>>>>>>> some
> > > > > > >>>>>>>>>>>>>>>>>>>> applications.
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm
> > > > > > currently
> > > > > > >>>>>>>> leaning
> > > > > > >>>>>>>>>>>>>>>> towards
> > > > > > >>>>>>>>>>>>>>>>>>>>> using the
> > statestore.transaction.buffer.max.bytes
> > > > > > >>>>>>>>>> configuration
> > > > > > >>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to
> > exceed it.
> > > > > > This
> > > > > > >>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>> complement the
> > > > > > >>>>>>>>>>>>>>>>>>>>> existing "early Task commit" functionality
> > that
> > > > this
> > > > > > >>>>>>>>>>>> configuration
> > > > > > >>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>           - Currently, we use
> > > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > > > >>>>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>> force an
> > > > > > >>>>>>>>>>>>>>>>>>>>>           early Task commit if processing
> > more
> > > > > > records
> > > > > > >>>> would
> > > > > > >>>>>>>>>> cause
> > > > > > >>>>>>>>>>>> our
> > > > > > >>>>>>>>>>>>>>>> state
> > > > > > >>>>>>>>>>>>>>>>>>>> store
> > > > > > >>>>>>>>>>>>>>>>>>>>>           transactions to exceed the memory
> > > > assigned
> > > > > > to
> > > > > > >>>>>>>> them.
> > > > > > >>>>>>>>>>>>>>>>>>>>>           - New functionality: when a Task
> > *does*
> > > > > > >> commit,
> > > > > > >>>>>>>> we will
> > > > > > >>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>> checkpoint
> > > > > > >>>>>>>>>>>>>>>>>>>>>           the stores (and hence flush the
> > > > transaction
> > > > > > >>>>>>>> buffers)
> > > > > > >>>>>>>>>>>> unless
> > > > > > >>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>> expect to
> > > > > > >>>>>>>>>>>>>>>>>>>>>           cross the
> > > > > > >>>> statestore.transaction.buffer.max.bytes
> > > > > > >>>>>>>>>>>> threshold
> > > > > > >>>>>>>>>>>>>>>> before
> > > > > > >>>>>>>>>>>>>>>>>>>> the next
> > > > > > >>>>>>>>>>>>>>>>>>>>>           commit
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > > >>>>>>>>>>>>>>>>>>>>> Nick
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> > > > > > >>>>>>>>>>>> nick.telford@gmail.com
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > >>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I meant in
> > > > terms of
> > > > > > >>>>>>>>>>>> understanding
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>> performance profile under various
> > > > circumstances. The
> > > > > > >>>>>>>> more
> > > > > > >>>>>>>>>>>> complex
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would be for
> > > > users
> > > > > > to
> > > > > > >>>>>>>>>>>> understand
> > > > > > >>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>> performance they see. For example, spilling
> > > > records
> > > > > > to
> > > > > > >>>>>>>> disk
> > > > > > >>>>>>>>>>>> when
> > > > > > >>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold
> > would, I
> > > > > > >> expect,
> > > > > > >>>>>>>>>> reduce
> > > > > > >>>>>>>>>>>>>> write
> > > > > > >>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
> > throughput
> > > > could
> > > > > > >> be
> > > > > > >>>>>>>>>>>>>> unexpected,
> > > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> > diagnose/understand for
> > > > > > >> users.
> > > > > > >>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early commit"
> > > > concept is
> > > > > > >>>>>>>>>> relatively
> > > > > > >>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document, and
> > > > > > >>>> conceptually
> > > > > > >>>>>>>>>> fairly
> > > > > > >>>>>>>>>>>>>>>>>>>> obvious to
> > > > > > >>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to
> > make it
> > > > > > >> easier
> > > > > > >>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>> understand
> > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > >>>>>>>>>>>>>>>>>>>>>> it happens though.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > > > > > >>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an
> > indirect
> > > > way
> > > > > > of
> > > > > > >>>>>>>>>> telling
> > > > > > >>>>>>>>>>>>>>>>>>>> StateStores
> > > > > > >>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
> > > > READ_COMMITTED
> > > > > > >>>>>>>>>>>> essentially
> > > > > > >>>>>>>>>>>>>>>>>>>> requires
> > > > > > >>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that two
> > > > threads
> > > > > > >>>>>>>> calling
> > > > > > >>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes
> > from
> > > > the
> > > > > > >> other
> > > > > > >>>>>>>>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>>>>>>>> until
> > > > > > >>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> > > > READ_UNCOMMITTED, all
> > > > > > >>>>>>>> bets are
> > > > > > >>>>>>>>>>>> off,
> > > > > > >>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe written
> > > > records
> > > > > > at
> > > > > > >>>>>>>> any
> > > > > > >>>>>>>>>>>> time,
> > > > > > >>>>>>>>>>>>>>>>>>>> which is
> > > > > > >>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That said,
> > > > > > StateStores
> > > > > > >>>>>>>> are
> > > > > > >>>>>>>>>> free
> > > > > > >>>>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>> implement
> > > > > > >>>>>>>>>>>>>>>>>>>>>> these guarantees however they can, which is
> > a
> > > > bit
> > > > > > more
> > > > > > >>>>>>>>>> relaxed
> > > > > > >>>>>>>>>>>>>> than
> > > > > > >>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions". For
> > > > example,
> > > > > > >>>> with
> > > > > > >>>>>>>>>>>> RocksDB
> > > > > > >>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>> would
> > > > > > >>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED ==
> > WBWI-based
> > > > > > >>>>>>>>>> "transactions",
> > > > > > >>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the
> > > > database.
> > > > > > But
> > > > > > >>>>>>>> with
> > > > > > >>>>>>>>>>>> other
> > > > > > >>>>>>>>>>>>>>>>>>>> storage
> > > > > > >>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to *always*
> > use
> > > > > > >>>>>>>> transactions,
> > > > > > >>>>>>>>>>>> even
> > > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > > >>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines
> > > > that
> > > > > > >> don't
> > > > > > >>>>>>>>>> provide
> > > > > > >>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation guarantees
> > can
> > > > be
> > > > > > met
> > > > > > >>>>>>>> using a
> > > > > > >>>>>>>>>>>>>>>>>>>> different
> > > > > > >>>>>>>>>>>>>>>>>>>>>> technique.
> > > > > > >>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore
> > > > interface
> > > > > > as
> > > > > > >>>>>>>>>> loosely
> > > > > > >>>>>>>>>>>>>>>> coupled
> > > > > > >>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > > > > > >>>>>>>> implementers
> > > > > > >>>>>>>>>> more
> > > > > > >>>>>>>>>>>>>>>>>>>> freedom, and
> > > > > > >>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge
> > > > required.
> > > > > > >>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> > "IsolationLevel"
> > > > might
> > > > > > >> not
> > > > > > >>>>>>>> be
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>> right
> > > > > > >>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it much
> > more
> > > > > > >>>>>>>> explicit if
> > > > > > >>>>>>>>>>>>>>>>>>>> required, e.g.
> > > > > > >>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> 7-8.
> > > > > > >>>>>>>>>>>>>>>>>>>>>> I can make these changes either later today
> > or
> > > > > > >>>> tomorrow.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> Small update:
> > > > > > >>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a
> > > > bunch of
> > > > > > >>>>>>>> issues
> > > > > > >>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>> needed
> > > > > > >>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass,
> > > > which is
> > > > > > >>>>>>>>>> promising,
> > > > > > >>>>>>>>>>>> but
> > > > > > >>>>>>>>>>>>>>>> it
> > > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > > >>>>>>>>>>>>>>>>>>>>>> need to undergo some performance testing. I
> > > > haven't
> > > > > > >>>>>>>> (yet)
> > > > > > >>>>>>>>>>>> worked
> > > > > > >>>>>>>>>>>>>> on
> > > > > > >>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I
> > > > would
> > > > > > >>>>>>>> expect
> > > > > > >>>>>>>>>> that,
> > > > > > >>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
> > difference. The
> > > > > > >> branch
> > > > > > >>>>>>>> is
> > > > > > >>>>>>>>>>>>>> available
> > > > > > >>>>>>>>>>>>>>>>>>>> at
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > https://github.com/nicktelford/kafka/tree/KIP-892-c
> > > > > > >> if
> > > > > > >>>>>>>>>> anyone
> > > > > > >>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > > >>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna
> > <
> > > > > > >>>>>>>>>>>> cadonna@apache.org>
> > > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually
> > also
> > > > my
> > > > > > >>>>>>>> point. I
> > > > > > >>>>>>>>>>>>>>>> understood
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion path
> > as
> > > > a way
> > > > > > >> to
> > > > > > >>>>>>>> avoid
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> early
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the
> > intent.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually it is
> > > > public
> > > > > > >>>>>>>> API. My
> > > > > > >>>>>>>>>>>>>> question
> > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal
> > > > processing.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the size
> > of the
> > > > > > >>>>>>>> transaction
> > > > > > >>>>>>>>>>>>>> buffer
> > > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit affects
> > the
> > > > whole
> > > > > > >>>>>>>>>> processing
> > > > > > >>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a state
> > store
> > > > is
> > > > > > not
> > > > > > >>>>>>>>>>>> confined to
> > > > > > >>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and
> > > > changes the
> > > > > > >>>>>>>> behavior
> > > > > > >>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>> other
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you that
> > it
> > > > is a
> > > > > > >>>>>>>> decent
> > > > > > >>>>>>>>>>>>>>>>>>>> compromise. I
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and
> > list
> > > > the
> > > > > > >>>>>>>> options to
> > > > > > >>>>>>>>>>>>>>>> overcome
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all
> > options
> > > > seem
> > > > > > >>>> quite
> > > > > > >>>>>>>>>> heavy
> > > > > > >>>>>>>>>>>>>>>>>>>> compared
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you
> > > > mean
> > > > > > with
> > > > > > >>>>>>>> "less
> > > > > > >>>>>>>>>>>>>>>>>>>> predictable
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
> > alternatives
> > > > > > really
> > > > > > >>>>>>>>>>>>>> interesting.
> > > > > > >>>>>>>>>>>>>>>>>>>> But I
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good and
> > we
> > > > should
> > > > > > >>>>>>>> continue
> > > > > > >>>>>>>>>>>> with
> > > > > > >>>>>>>>>>>>>>>> it!
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on
> > > > June
> > > > > > >> 20th:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind
> > > > putting
> > > > > > >>>>>>>> isolation
> > > > > > >>>>>>>>>>>>>> level
> > > > > > >>>>>>>>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should
> > that
> > > > also
> > > > > > be
> > > > > > >> a
> > > > > > >>>>>>>> way
> > > > > > >>>>>>>>>> to
> > > > > > >>>>>>>>>>>>>> give
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to decide
> > > > whether
> > > > > > to
> > > > > > >>>>>>>> turn on
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned about
> > > > how do
> > > > > > >> you
> > > > > > >>>>>>>> know
> > > > > > >>>>>>>>>>>> if a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written under
> > EOS,
> > > > if
> > > > > > you
> > > > > > >>>>>>>> do not
> > > > > > >>>>>>>>>>>>>> have a
> > > > > > >>>>>>>>>>>>>>>>>>>> way
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
> > transactional or
> > > > not.
> > > > > > >> If
> > > > > > >>>>>>>> a
> > > > > > >>>>>>>>>> state
> > > > > > >>>>>>>>>>>>>>>> store
> > > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be
> > > > written
> > > > > > >>>>>>>> during
> > > > > > >>>>>>>>>>>> normal
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state store
> > is not
> > > > > > >>>>>>>>>> transactional,
> > > > > > >>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written under
> > EOS.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> 7.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> My point was about not only considering the
> > > > bytes
> > > > > > in
> > > > > > >>>>>>>> memory
> > > > > > >>>>>>>>>> in
> > > > > > >>>>>>>>>>>>>>>> config
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also
> > > > bytes
> > > > > > that
> > > > > > >>>>>>>> might
> > > > > > >>>>>>>>>> be
> > > > > > >>>>>>>>>>>>>>>>>>>> spilled
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering
> > whether you
> > > > > > >> should
> > > > > > >>>>>>>>>> remove
> > > > > > >>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory
> > bytes to
> > > > be
> > > > > > >> used
> > > > > > >>>>>>>> to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My
> > > > > > thinking
> > > > > > >>>>>>>> was
> > > > > > >>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>> even
> > > > > > >>>>>>>>>>>>>>>>>>>> if a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to
> > disk,
> > > > > > >> limiting
> > > > > > >>>>>>>> the
> > > > > > >>>>>>>>>>>>>> overall
> > > > > > >>>>>>>>>>>>>>>>>>>> bytes
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it again
> > and
> > > > > > >>>>>>>> considering
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>> recent
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much
> > sense
> > > > > > anymore.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> I like the name
> > > > > > >>>>>>>> statestore.transaction.buffer.max.bytes that
> > > > > > >>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>>>>>>>> proposed.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> 8.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> > > > implementation
> > > > > > >>>>>>>> details) of
> > > > > > >>>>>>>>>>>> how
> > > > > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
> > > > > > >>>>>>>> transactions,
> > > > > > >>>>>>>>>>>> state
> > > > > > >>>>>>>>>>>>>>>> store
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be
> > great.
> > > > > > Would
> > > > > > >>>> be
> > > > > > >>>>>>>>>> great
> > > > > > >>>>>>>>>>>> if
> > > > > > >>>>>>>>>>>>>>>> you
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the
> > > > behavior in
> > > > > > >>>>>>>> case of
> > > > > > >>>>>>>>>> a
> > > > > > >>>>>>>>>>>>>>>>>>>> failure.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional state
> > > > store
> > > > > > >>>>>>>> recover
> > > > > > >>>>>>>>>>>> after a
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
> > transaction
> > > > > > buffer,
> > > > > > >>>>>>>> etc.
> > > > > > >>>>>>>>>>>> (that
> > > > > > >>>>>>>>>>>>>> is
> > > > > > >>>>>>>>>>>>>>>>>>>> what
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> > > > > > >>>>>>>> WriteBatchWithIndex
> > > > > > >>>>>>>>>>>>>>>>>>>> transactions
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to
> > exceed)
> > > > > > >>>>>>>> configured
> > > > > > >>>>>>>>>>>> memory
> > > > > > >>>>>>>>>>>>>>>>>>>> needs to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
> > Ultimately,
> > > > any
> > > > > > >>>>>>>> approach
> > > > > > >>>>>>>>>>>> based
> > > > > > >>>>>>>>>>>>>> on
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> cracking
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in
> > ways
> > > > it's
> > > > > > not
> > > > > > >>>>>>>> really
> > > > > > >>>>>>>>>>>>>>>> designed
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> for is
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen performance
> > or
> > > > > > >>>>>>>> consistency
> > > > > > >>>>>>>>>>>> issues.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing these
> > > > early
> > > > > > >>>>>>>> commits?
> > > > > > >>>>>>>>>>>> While
> > > > > > >>>>>>>>>>>>>> not
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to
> > ensure
> > > > > > >>>>>>>> consistency
> > > > > > >>>>>>>>>>>> whilst
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> maintaining
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
> > > > > > >>>>>>>> complicated, and
> > > > > > >>>>>>>>>>>> might
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for users
> > as a
> > > > > > >>>>>>>> consequence.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this
> > > > KIP is
> > > > > > >>>>>>>> growing a
> > > > > > >>>>>>>>>>>> bit
> > > > > > >>>>>>>>>>>>>>>> out
> > > > > > >>>>>>>>>>>>>>>>>>>> of
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas
> > for
> > > > > > future
> > > > > > >>>>>>>>>>>>>>>> improvements, I
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> think
> >
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope down
> > to a
> > > > > > design
> > > > > > >>>>>>>> that
> > > > > > >>>>>>>>>>>>>> achieves
> > > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> most
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
> > > > restorations
> > > > > > >>>>>>>> during
> > > > > > >>>>>>>>>> dirty
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that
> > > > this KIP
> > > > > > >>>>>>>> produces
> > > > > > >>>>>>>>>>>> can
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> ultimately
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially if
> > the
> > > > bulk
> > > > > > of
> > > > > > >>>>>>>> it is
> > > > > > >>>>>>>>>>>>>> internal
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week
> > trying
> > > > to
> > > > > > >>>>>>>> re-work
> > > > > > >>>>>>>>>> the
> > > > > > >>>>>>>>>>>>>>>>>>>> original
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
> > > > > > >>>>>>>> newTransaction()
> > > > > > >>>>>>>>>>>>>> method,
> > > > > > >>>>>>>>>>>>>>>>>>>> such
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
> > > > > > RocksDBStore.
> > > > > > >>>>>>>> That
> > > > > > >>>>>>>>>>>> way, if
> > > > > > >>>>>>>>>>>>>>>> we
> > > > > > >>>>>>>>>>>>>>>>>>>>>>> want to
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the future,
> > > > like
> > > > > > the
> > > > > > >>>>>>>> SST
> > > > > > >>>>>>>>>> file
> > > > > > >>>>>>>>>>>>>>>>>>>> management
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with
> > > > little/no
> > > > > > >>>> API
> > > > > > >>>>>>>>>>>> changes.
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>>
> > > > > > >>>>>>>>>>>>
> > > > > > >>>>>>>>>>>
> > > > > > >>>>>>>>>>
> > > > > > >>>>>>>>
> > > > > > >>>>>>>
> > > > > > >>>>>>
> > > > > > >>>>>
> > > > > > >>>>
> > > > > > >>>
> > > > > > >>
> > > > > > >
> > > > > >
> > > >
> > > >
> >

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Lucas,

Yeah, this is pretty much the direction I'm thinking of going in now. You
make an interesting point about committing on-error under
ALOS/READ_COMMITTED, although I haven't had a chance to think through the
implications yet.

Something that I ran into earlier this week is an issue with the new
handling of TimeoutException. Without TX stores, TimeoutException under EOS
throws a TaskCorruptedException, which wipes the stores. However, with TX
stores, TimeoutException is now just bubbled up and dealt with as it is
under ALOS. The problem arises when the Producer#commitTransaction call
times out: Streams attempts to ignore the error and continue producing,
which causes the next call to Producer#send to throw
"IllegalStateException: Cannot attempt operation `send` because the
previous call to `commitTransaction` timed out and must be retried".

I'm not sure what we should do here: retrying the commitTransaction seems
logical, but what if it times out again? Where do we draw the line and
shutdown the instance?

Regards,
Nick

On Mon, 16 Oct 2023 at 13:19, Lucas Brutschy <lb...@confluent.io.invalid>
wrote:

> Hi all,
>
> I think I liked your suggestion of allowing EOS with READ_UNCOMMITTED,
> but keep wiping the state on error, and I'd vote for this solution
> when introducing `default.state.isolation.level`. This way, we'd have
> the most low-risk roll-out of this feature (no behavior change without
> reconfiguration), with the possibility of switching to the most sane /
> battle-tested default settings in 4.0. Essentially, we'd have a
> feature flag but call it `default.state.isolation.level` and don't
> have to deprecate it later.
>
> So the possible configurations would then be this:
>
> 1. ALOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
> reads from DB.
> 2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> WriteBatch/DB. Flush on error (see note below).
> 3. EOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
> reads from DB. Wipe state on error.
> 4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
> WriteBatch/DB.
>
> I believe the feature is important enough that we will see good
> adoption even without changing the default. In 4.0, when we have seen
> this being adopted and is battle-tested, we make READ_COMMITTED the
> default for EOS, or even READ_COMITTED always the default, depending
> on our experiences. And we could add a clever implementation of
> READ_UNCOMITTED with WriteBatches later.
>
> The only smell here is that `default.state.isolation.level` wouldn't
> be purely an IQ setting, but it would also (slightly) change the
> behavior of the processing, but that seems unavoidable as long as we
> haven't solve READ_UNCOMITTED IQ with WriteBatches.
>
> Minor: As for Bruno's point 4, I think if we are concerned about this
> 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).
>
> Cheers,
> Lucas
>
> On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <ni...@gmail.com>
> wrote:
> >
> > Hi Guozhang,
> >
> > The KIP as it stands introduces a new configuration,
> > default.state.isolation.level, which is independent of processing.mode.
> > It's intended that this new configuration be used to configure a global
> IQ
> > isolation level in the short term, with a future KIP introducing the
> > capability to change the isolation level on a per-query basis, falling
> back
> > to the "default" defined by this config. That's why I called it
> "default",
> > for future-proofing.
> >
> > However, it currently includes the caveat that READ_UNCOMMITTED is not
> > available under EOS. I think this is the coupling you are alluding to?
> >
> > This isn't intended to be a restriction of the API, but is currently a
> > technical limitation. However, after discussing with some users about
> > use-cases that would require READ_UNCOMMITTED under EOS, I'm inclined to
> > remove that clause and put in the necessary work to make that combination
> > possible now.
> >
> > I currently see two possible approaches:
> >
> >    1. Disable TX StateStores internally when the IsolationLevel is
> >    READ_UNCOMMITTED and the processing.mode is EOS. This is more
> difficult
> >    than it sounds, as there are many assumptions being made throughout
> the
> >    internals about the guarantees StateStores provide. It would
> definitely add
> >    a lot of extra "if (read_uncommitted && eos)" branches, complicating
> >    maintenance and testing.
> >    2. Invest the time *now* to make READ_UNCOMMITTED of EOS StateStores
> >    possible. I have some ideas on how this could be achieved, but they
> would
> >    need testing and could introduce some additional issues. The benefit
> of
> >    this approach is that it would make query-time IsolationLevels much
> simpler
> >    to implement in the future.
> >
> > Unfortunately, both will require considerable work that will further
> delay
> > this KIP, which was the reason I placed the restriction in the KIP in the
> > first place.
> >
> > Regards,
> > Nick
> >
> > On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <gu...@gmail.com>
> > wrote:
> >
> > > Hello Nick,
> > >
> > > First of all, thanks a lot for the great effort you've put in driving
> > > this KIP! I really like it coming through finally, as many people in
> > > the community have raised this. At the same time I honestly feel a bit
> > > ashamed for not putting enough of my time supporting it and pushing it
> > > through the finish line (you raised this KIP almost a year ago).
> > >
> > > I briefly passed through the DISCUSS thread so far, not sure I've 100
> > > percent digested all the bullet points. But with the goal of trying to
> > > help take it through the finish line in mind, I'd want to throw
> > > thoughts on top of my head only on the point #4 above which I felt may
> > > be the main hurdle for the current KIP to drive to a consensus now.
> > >
> > > The general question I asked myself is, whether we want to couple "IQ
> > > reading mode" with "processing mode". While technically I tend to
> > > agree with you that, it's feels like a bug if some single user chose
> > > "EOS" for processing mode while choosing "read uncommitted" for IQ
> > > reading mode, at the same time, I'm thinking if it's possible that
> > > there could be two different persons (or even two teams) that would be
> > > using the stream API to build the app, and the IQ API to query the
> > > running state of the app. I know this is less of a technical thing but
> > > rather a more design stuff, but if it could be ever the case, I'm
> > > wondering if the personale using the IQ API knows about the risks of
> > > using read uncommitted but still chose so for the favor of
> > > performance, no matter if the underlying stream processing mode
> > > configured by another personale is EOS or not. In that regard, I'm
> > > leaning towards a "leaving the door open, and close it later if we
> > > found it's a bad idea" aspect with a configuration 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 can still enforce
> > > coupling them.
> > >
> > > Again, just my 2c here. Thanks again for the great patience and
> > > diligence on this KIP.
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > > On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <ni...@gmail.com>
> > > wrote:
> > > >
> > > > Hi Bruno,
> > > >
> > > > 4.
> > > > I'll hold off on making that change until we have a consensus as to
> what
> > > > configuration to use to control all of this, as it'll be affected by
> the
> > > > decision on EOS isolation levels.
> > > >
> > > > 5.
> > > > Done. I've chosen "committedOffsets".
> > > >
> > > > Regards,
> > > > Nick
> > > >
> > > > On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <ca...@apache.org>
> wrote:
> > > >
> > > > > Hi Nick,
> > > > >
> > > > > 1.
> > > > > Yeah, you are probably right that it does not make too much sense.
> > > > > Thanks for the clarification!
> > > > >
> > > > >
> > > > > 4.
> > > > > Yes, sorry for the back and forth, but I think for the sake of the
> KIP
> > > > > it is better to let the ALOS behavior as it is for now due to the
> > > > > possible issues you would run into. Maybe we can find a solution
> in the
> > > > > future. Now the question returns to whether we really need
> > > > > default.state.isolation.level. Maybe the config could be the
> feature
> > > > > flag Sophie requested.
> > > > >
> > > > >
> > > > > 5.
> > > > > There 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 committedOffsetFor().
> > > > >
> > > > >
> > > > > Best,
> > > > > Bruno
> > > > >
> > > > >
> > > > > On 10/13/23 3:21 PM, Nick Telford wrote:
> > > > > > Hi Bruno,
> > > > > >
> > > > > > Thanks for getting back to me.
> > > > > >
> > > > > > 1.
> > > > > > I think this should be possible. Are you thinking of the
> situation
> > > where
> > > > > a
> > > > > > user may downgrade to a previous version of Kafka Streams? In
> that
> > > case,
> > > > > > sadly, the RocksDBStore would get wiped by the older version of
> Kafka
> > > > > > Streams anyway, because that version wouldn't understand the
> extra
> > > column
> > > > > > family (that holds offsets), so the missing Position file would
> > > > > > automatically get rebuilt when the store is rebuilt from the
> > > changelog.
> > > > > > Are there other situations than downgrade where a transactional
> store
> > > > > could
> > > > > > be replaced by a non-transactional one? I can't think of any.
> > > > > >
> > > > > > 2.
> > > > > > Ahh yes, the Test Plan - my Kryptonite! This section definitely
> > > needs to
> > > > > be
> > > > > > fleshed out. I'll work on that. How much detail do you need?
> > > > > >
> > > > > > 3.
> > > > > > See my previous email discussing this.
> > > > > >
> > > > > > 4.
> > > > > > Hmm, this is an interesting point. Are you suggesting that under
> ALOS
> > > > > > READ_COMMITTED should not be supported?
> > > > > >
> > > > > > Regards,
> > > > > > Nick
> > > > > >
> > > > > > On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <ca...@apache.org>
> > > wrote:
> > > > > >
> > > > > >> Hi Nick,
> > > > > >>
> > > > > >> I think the KIP is converging!
> > > > > >>
> > > > > >>
> > > > > >> 1.
> > > > > >> I am wondering whether it makes sense to write the position file
> > > during
> > > > > >> close as we do for the checkpoint file, so that in case the
> state
> > > store
> > > > > >> is replaced with a non-transactional state store the
> > > non-transactional
> > > > > >> state store finds the position file. I think, this is not
> strictly
> > > > > >> needed, but would be a nice behavior instead of just deleting
> the
> > > > > >> position file.
> > > > > >>
> > > > > >>
> > > > > >> 2.
> > > > > >> The test plan does not mention integration tests. Do you not
> need to
> > > > > >> extend existing ones and add new ones. Also for upgrading and
> > > > > >> downgrading you might need integration and/or system tests.
> > > > > >>
> > > > > >>
> > > > > >> 3.
> > > > > >> I think Sophie made a point. Although, IQ reading from
> uncommitted
> > > data
> > > > > >> under EOS might be considered a bug by some people. Thus, your
> KIP
> > > would
> > > > > >> fix a bug rather than changing the intended behavior. However, I
> > > also
> > > > > >> see that a feature flag would help users that rely on this buggy
> > > > > >> behavior (at least until AK 4.0).
> > > > > >>
> > > > > >>
> > > > > >> 4.
> > > > > >> This is related to the previous point. I assume that the
> difference
> > > > > >> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is that in
> the
> > > > > >> former you enable transactions on the state store and in the
> latter
> > > you
> > > > > >> disable them. If my assumption is correct, I think that is an
> issue.
> > > > > >> Let's assume under ALOS Streams fails over a couple of times
> more or
> > > > > >> less at the same step in processing after value 3 is added to an
> > > > > >> aggregation but the offset of the corresponding input record
> was not
> > > > > >> committed. Without transactions disabled, the aggregation value
> > > would
> > > > > >> increase by 3 for each failover. With transactions enabled,
> value 3
> > > > > >> would only be added to the aggregation once when the offset of
> the
> > > input
> > > > > >> record is committed and the transaction finally completes. So
> the
> > > > > >> content of the state store would change depending on the
> > > configuration
> > > > > >> for IQ. IMO, the content of the state store should be
> independent
> > > from
> > > > > >> IQ. Given this issue, I propose to not use transactions with
> ALOS at
> > > > > >> all. I was a big proponent of using transactions with ALOS, but
> I
> > > > > >> realized that transactions with ALOS is not as easy as enabling
> > > > > >> transactions on state stores. Another aspect that is
> problematic is
> > > that
> > > > > >> the changelog topic which actually replicates the state store
> is not
> > > > > >> transactional under ALOS. Thus, it might happen that the state
> > > store and
> > > > > >> the changelog differ in their content. All of this is maybe
> solvable
> > > > > >> somehow, but for the sake of this KIP, I would leave it for the
> > > future.
> > > > > >>
> > > > > >>
> > > > > >> Best,
> > > > > >> Bruno
> > > > > >>
> > > > > >>
> > > > > >>
> > > > > >> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > > > > >>> Hey Nick! First of all thanks for taking up this awesome
> feature,
> > > I'm
> > > > > >> sure
> > > > > >>> every single
> > > > > >>> Kafka Streams user and dev would agree that it is sorely
> needed.
> > > > > >>>
> > > > > >>> I've just been catching up on the KIP and surrounding
> discussion,
> > > so
> > > > > >> please
> > > > > >>> forgive me
> > > > > >>> for any misunderstandings or misinterpretations of the current
> > > plan and
> > > > > >>> don't hesitate to
> > > > > >>> correct me.
> > > > > >>>
> > > > > >>> Before I jump in, I just want to say that having seen this
> drag on
> > > for
> > > > > so
> > > > > >>> long, my singular
> > > > > >>> goal in responding is to help this KIP past a perceived
> impasse so
> > > we
> > > > > can
> > > > > >>> finally move on
> > > > > >>> to voting and implementing it. Long discussions are to be
> expected
> > > for
> > > > > >>> major features like
> > > > > >>> this but it's completely on us as the Streams devs to make sure
> > > there
> > > > > is
> > > > > >> an
> > > > > >>> end in sight
> > > > > >>> for any ongoing discussion.
> > > > > >>>
> > > > > >>> With that said, it's my understanding that the KIP as currently
> > > > > proposed
> > > > > >> is
> > > > > >>> just not tenable
> > > > > >>> for Kafka Streams, and would prevent some EOS users from
> upgrading
> > > to
> > > > > the
> > > > > >>> version it
> > > > > >>> first appears in. Given that we can't predict or guarantee
> whether
> > > any
> > > > > of
> > > > > >>> the followup KIPs
> > > > > >>> would be completed in the same release cycle as this one, we
> need
> > > to
> > > > > make
> > > > > >>> sure that the
> > > > > >>> feature is either compatible with all current users or else
> > > > > >> feature-flagged
> > > > > >>> so that they may
> > > > > >>> opt in/out.
> > > > > >>>
> > > > > >>> Therefore, IIUC we need to have either (or both) of these as
> > > > > >>> fully-implemented config options:
> > > > > >>> 1. default.state.isolation.level
> > > > > >>> 2. enable.transactional.state.stores
> > > > > >>>
> > > > > >>> This way EOS users for whom read_committed semantics are not
> > > viable can
> > > > > >>> still upgrade,
> > > > > >>> and either use the isolation.level config to leverage the new
> txn
> > > state
> > > > > >>> stores without sacrificing
> > > > > >>> their application semantics, or else simply keep the
> transactional
> > > > > state
> > > > > >>> stores disabled until we
> > > > > >>> are able to fully implement the isolation level configuration
> at
> > > either
> > > > > >> an
> > > > > >>> application or query level.
> > > > > >>>
> > > > > >>> Frankly you are the expert here and know much more about the
> > > tradeoffs
> > > > > in
> > > > > >>> both semantics and
> > > > > >>> effort level of implementing one of these configs vs the
> other. In
> > > my
> > > > > >>> opinion, either option would
> > > > > >>> be fine and I would leave the decision of which one to include
> in
> > > this
> > > > > >> KIP
> > > > > >>> completely up to you.
> > > > > >>> I just don't see a way for the KIP to proceed without some
> > > variation of
> > > > > >> the
> > > > > >>> above that would allow
> > > > > >>> EOS users to opt-out of read_committed.
> > > > > >>>
> > > > > >>> (If it's all the same to you, I would recommend 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 that gets released.
> > > > > >>> So personally, my choice would be to add the feature flag and
> > > leave it
> > > > > >> off
> > > > > >>> by default. If all 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
> > > > > >>>
> > > > > >>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > > nick.telford@gmail.com>
> > > > > >> wrote:
> > > > > >>>
> > > > > >>>> Hi Bruno,
> > > > > >>>>
> > > > > >>>> Agreed, I can live with that for now.
> > > > > >>>>
> > > > > >>>> In an effort to keep the scope of this KIP from expanding, I'm
> > > leaning
> > > > > >>>> towards just providing a configurable
> > > default.state.isolation.level
> > > > > and
> > > > > >>>> removing IsolationLevel from the StateStoreContext. This
> would be
> > > > > >>>> compatible with adding support for query-time IsolationLevels
> in
> > > the
> > > > > >>>> future, whilst providing a way for users to select an
> isolation
> > > level
> > > > > >> now.
> > > > > >>>>
> > > > > >>>> The big problem with this, however, is that if a user selects
> > > > > >>>> processing.mode
> > > > > >>>> = "exactly-once(-v2|-beta)", and
> default.state.isolation.level =
> > > > > >>>> "READ_UNCOMMITTED", we need to guarantee that the data isn't
> > > written
> > > > > to
> > > > > >>>> disk until commit() is called, but we also need to permit IQ
> > > threads
> > > > > to
> > > > > >>>> read from the ongoing transaction.
> > > > > >>>>
> > > > > >>>> A simple solution would be to (temporarily) forbid this
> > > combination of
> > > > > >>>> configuration, and have default.state.isolation.level
> > > automatically
> > > > > >> switch
> > > > > >>>> to READ_COMMITTED when processing.mode is anything other than
> > > > > >>>> at-least-once. Do you think this would be acceptable?
> > > > > >>>>
> > > > > >>>> In a later KIP, we can add support for query-time isolation
> > > levels and
> > > > > >>>> solve this particular problem there, which would relax this
> > > > > restriction.
> > > > > >>>>
> > > > > >>>> Regards,
> > > > > >>>> Nick
> > > > > >>>>
> > > > > >>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <
> cadonna@apache.org>
> > > > > >> 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,
> > > > > >>>>> Bruno
> > > > > >>>>>
> > > > > >>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > > > > >>>>>> 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 <
> > > nick.telford@gmail.com
> > > > > >
> > > > > >>>>> wrote:
> > > > > >>>>>>
> > > > > >>>>>>> Hi everyone,
> > > > > >>>>>>>
> > > > > >>>>>>> I agree that having IsolationLevel be determined at
> query-time
> > > is
> > > > > the
> > > > > >>>>>>> ideal design, but there are a few sticking points:
> > > > > >>>>>>>
> > > > > >>>>>>> 1.
> > > > > >>>>>>> There needs to be some way to communicate the
> IsolationLevel
> > > down
> > > > > to
> > > > > >>>> the
> > > > > >>>>>>> RocksDBStore itself, so that the query can respect it.
> Since
> > > stores
> > > > > >>>> are
> > > > > >>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
> > > MeteredStore,
> > > > > >>>>> etc.),
> > > > > >>>>>>> we need some way to deliver that information to the bottom
> > > layer.
> > > > > For
> > > > > >>>>> IQv2,
> > > > > >>>>>>> we can use the existing State#query() method, but IQv1 has
> no
> > > way
> > > > > to
> > > > > >>>> do
> > > > > >>>>>>> this.
> > > > > >>>>>>>
> > > > > >>>>>>> A simple approach, which would potentially open up other
> > > options,
> > > > > >>>> would
> > > > > >>>>> be
> > > > > >>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> > > > > >>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > > > > ReadOnlyKeyValueStore
> > > > > >>>>> (and
> > > > > >>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore,
> etc.).
> > > > > >>>>>>>
> > > > > >>>>>>> 2.
> > > > > >>>>>>> As mentioned above, RocksDB WriteBatches are not
> thread-safe,
> > > which
> > > > > >>>>> causes
> > > > > >>>>>>> a problem if we want to provide READ_UNCOMMITTED
> Iterators. I
> > > also
> > > > > >>>> had a
> > > > > >>>>>>> look at RocksDB Transactions[1], but they solve a very
> > > different
> > > > > >>>>> problem,
> > > > > >>>>>>> and have the same thread-safety issue.
> > > > > >>>>>>>
> > > > > >>>>>>> One possible approach that I mentioned is chaining
> > > WriteBatches:
> > > > > >> every
> > > > > >>>>>>> time a new Interactive Query is received (i.e.
> readOnlyView,
> > > see
> > > > > >>>> above,
> > > > > >>>>>>> is called) we "freeze" the existing WriteBatch, and start a
> > > new one
> > > > > >>>> for
> > > > > >>>>> new
> > > > > >>>>>>> writes. The Interactive Query queries the "chain" of
> previous
> > > > > >>>>> WriteBatches
> > > > > >>>>>>> + the underlying database; while the StreamThread starts
> > > writing to
> > > > > >>>> the
> > > > > >>>>>>> *new* WriteBatch. On-commit, the StreamThread would write
> *all*
> > > > > >>>>>>> WriteBatches in the chain to the database (that have not
> yet
> > > been
> > > > > >>>>> written).
> > > > > >>>>>>>
> > > > > >>>>>>> WriteBatches would be closed/freed only when they have been
> > > both
> > > > > >>>>>>> committed, and all open Interactive Queries on them have
> been
> > > > > closed.
> > > > > >>>>> This
> > > > > >>>>>>> would require some reference counting.
> > > > > >>>>>>>
> > > > > >>>>>>> Obviously a drawback of this approach is the potential for
> > > > > increased
> > > > > >>>>>>> memory usage: if an Interactive Query is long-lived, for
> > > example by
> > > > > >>>>> doing a
> > > > > >>>>>>> full scan over a large database, or even just pausing in
> the
> > > middle
> > > > > >> of
> > > > > >>>>> an
> > > > > >>>>>>> iteration, then the existing chain of WriteBatches could be
> > > kept
> > > > > >>>> around
> > > > > >>>>> for
> > > > > >>>>>>> a long time, potentially forever.
> > > > > >>>>>>>
> > > > > >>>>>>> --
> > > > > >>>>>>>
> > > > > >>>>>>> A.
> > > > > >>>>>>> Going off on a tangent, it looks like in addition to
> supporting
> > > > > >>>>>>> READ_COMMITTED queries, we could go further and support
> > > > > >>>> REPEATABLE_READ
> > > > > >>>>>>> queries (i.e. where subsequent reads to the same key in the
> > > same
> > > > > >>>>>>> Interactive Query are guaranteed to yield the same value)
> by
> > > making
> > > > > >>>> use
> > > > > >>>>> of
> > > > > >>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so the
> > > > > >> performance
> > > > > >>>>>>> impact is likely to be negligible, but they do require
> that the
> > > > > >>>>> Interactive
> > > > > >>>>>>> Query session can be explicitly closed.
> > > > > >>>>>>>
> > > > > >>>>>>> This could be achieved if we made the above readOnlyView
> > > interface
> > > > > >>>> look
> > > > > >>>>>>> more like:
> > > > > >>>>>>>
> > > > > >>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> > > > > >>>> ReadOnlyKeyValueStore<K,
> > > > > >>>>>>> V>, AutoCloseable {}
> > > > > >>>>>>>
> > > > > >>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > > > > >>>>>>>        ...
> > > > > >>>>>>>        ReadOnlyKeyValueView<K, V>
> readOnlyView(IsolationLevel
> > > > > >>>>> isolationLevel);
> > > > > >>>>>>> }
> > > > > >>>>>>>
> > > > > >>>>>>> But this would be a breaking change, as existing IQv1
> queries
> > > are
> > > > > >>>>>>> guaranteed to never call store.close(), and therefore these
> > > would
> > > > > >> leak
> > > > > >>>>>>> memory under REPEATABLE_READ.
> > > > > >>>>>>>
> > > > > >>>>>>> B.
> > > > > >>>>>>> One thing that's notable: MyRocks states that they support
> > > > > >>>>> READ_COMMITTED
> > > > > >>>>>>> and REPEATABLE_READ, but they make no mention of
> > > > > >>>> READ_UNCOMMITTED[3][4].
> > > > > >>>>>>> This could be because doing so is technically
> > > difficult/impossible
> > > > > >>>> using
> > > > > >>>>>>> the primitives available in RocksDB.
> > > > > >>>>>>>
> > > > > >>>>>>> --
> > > > > >>>>>>>
> > > > > >>>>>>> Lucas, to address your points:
> > > > > >>>>>>>
> > > > > >>>>>>> U1.
> > > > > >>>>>>> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
> > > > > >>>>>>> implementations of StateStore that do not support atomic
> > > writes.
> > > > > >>>>> Obviously
> > > > > >>>>>>> in those cases, the guarantees Kafka Streams
> provides/expects
> > > would
> > > > > >> be
> > > > > >>>>>>> relaxed. Do you think we should require all
> implementations to
> > > > > >> support
> > > > > >>>>>>> atomic writes?
> > > > > >>>>>>>
> > > > > >>>>>>> U2.
> > > > > >>>>>>> Stores can support multiple IsolationLevels. As we've
> discussed
> > > > > >> above,
> > > > > >>>>> the
> > > > > >>>>>>> ideal scenario would be to specify the IsolationLevel at
> > > > > query-time.
> > > > > >>>>>>> Failing that, I think the second-best approach is to
> define the
> > > > > >>>>>>> IsolationLevel for *all* queries based on the
> processing.mode,
> > > > > which
> > > > > >>>> is
> > > > > >>>>>>> what the default StateStoreContext#isolationLevel()
> achieves.
> > > Would
> > > > > >>>> you
> > > > > >>>>>>> prefer an alternative?
> > > > > >>>>>>>
> > > > > >>>>>>> While the existing implementation is equivalent to
> > > > > READ_UNCOMMITTED,
> > > > > >>>>> this
> > > > > >>>>>>> can yield unexpected results/errors under EOS, if a
> > > transaction is
> > > > > >>>>> rolled
> > > > > >>>>>>> back. While this would be a change in behaviour for users,
> it
> > > would
> > > > > >>>> look
> > > > > >>>>>>> more like a bug fix than a breaking change. That said, we
> > > *could*
> > > > > >> make
> > > > > >>>>> it
> > > > > >>>>>>> configurable, and default to the existing behaviour
> > > > > >> (READ_UNCOMMITTED)
> > > > > >>>>>>> instead of inferring it from the processing.mode?
> > > > > >>>>>>>
> > > > > >>>>>>> N1, N2.
> > > > > >>>>>>> These were only primitives to avoid boxing costs, but since
> > > this is
> > > > > >>>> not
> > > > > >>>>> a
> > > > > >>>>>>> performance sensitive area, it should be fine to change if
> > > that's
> > > > > >>>>> desirable.
> > > > > >>>>>>>
> > > > > >>>>>>> N3.
> > > > > >>>>>>> It's because the store "manages its own offsets", which
> > > includes
> > > > > both
> > > > > >>>>>>> committing the offset, *and providing it* via
> > > getCommittedOffset().
> > > > > >>>>>>> Personally, I think "managesOffsets" conveys this best,
> but I
> > > don't
> > > > > >>>> mind
> > > > > >>>>>>> changing it if the nomenclature is unclear.
> > > > > >>>>>>>
> > > > > >>>>>>> Sorry for the massive emails/essays!
> > > > > >>>>>>> --
> > > > > >>>>>>> Nick
> > > > > >>>>>>>
> > > > > >>>>>>> 1: https://github.com/facebook/rocksdb/wiki/Transactions
> > > > > >>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > > > > >>>>>>> 3:
> > > > > https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > > > >>>>>>> 4:
> https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > > > >>>>>>>
> > > > > >>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > > > > >>>>>>> <lb...@confluent.io.invalid> wrote:
> > > > > >>>>>>>
> > > > > >>>>>>>> Hi Nick,
> > > > > >>>>>>>>
> > > > > >>>>>>>> since I last read it in April, the KIP has become much
> > > cleaner and
> > > > > >>>>>>>> easier to read. Great work!
> > > > > >>>>>>>>
> > > > > >>>>>>>> It feels to me the last big open point is whether we can
> > > implement
> > > > > >>>>>>>> isolation level as a query parameter. I understand that
> there
> > > are
> > > > > >>>>>>>> implementation concerns, but as Colt says, it would be a
> great
> > > > > >>>>>>>> addition, and would also simplify the migration path for
> this
> > > > > >> change.
> > > > > >>>>>>>> Is the implementation problem you mentioned caused by the
> > > > > WriteBatch
> > > > > >>>>>>>> not having a notion of a snapshot, as the underlying DB
> > > iterator
> > > > > >>>> does?
> > > > > >>>>>>>> In that case, I am not sure a chain of WriteBatches as you
> > > propose
> > > > > >>>>>>>> would fully solve the problem, but maybe I didn't dig
> enough
> > > into
> > > > > >> the
> > > > > >>>>>>>> details to fully understand it.
> > > > > >>>>>>>>
> > > > > >>>>>>>> If it's not possible to implement it now, would it be an
> > > option to
> > > > > >>>>>>>> make sure in this KIP that we do not fully close the door
> on
> > > > > >>>> per-query
> > > > > >>>>>>>> isolation levels in the interface, as it may be possible
> to
> > > > > >> implement
> > > > > >>>>>>>> the missing primitives in RocksDB or Speedb in the future.
> > > > > >>>>>>>>
> > > > > >>>>>>>> Understanding:
> > > > > >>>>>>>>
> > > > > >>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to be
> > > persisted
> > > > > >>>>>>>> atomically with the records?
> > > > > >>>>>>>> * U2) Don't understand the default implementation of
> > > > > >>>> `isolationLevel`.
> > > > > >>>>>>>> The isolation level should be a property of the underlying
> > > store,
> > > > > >> and
> > > > > >>>>>>>> not be defined by the default config? Existing stores
> probably
> > > > > don't
> > > > > >>>>>>>> guarantee READ_COMMITTED, so the default should be to
> return
> > > > > >>>>>>>> READ_UNCOMMITTED.
> > > > > >>>>>>>>
> > > > > >>>>>>>> Nits:
> > > > > >>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong`
> return
> > > type,
> > > > > >> to
> > > > > >>>>>>>> avoid the `null`?
> > > > > >>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
> > > `OptionalLong`
> > > > > >>>>>>>> return type, to avoid the `-1`?
> > > > > >>>>>>>> * N3) I don't understand why `managesOffsets` uses the
> > > 'manage'
> > > > > >> verb,
> > > > > >>>>>>>> whereas all other methods use the "commits" verb. I'd
> suggest
> > > > > >>>>>>>> `commitsOffsets`.
> > > > > >>>>>>>>
> > > > > >>>>>>>> Either way, it feels this KIP is very close to the finish
> > > line,
> > > > > I'm
> > > > > >>>>>>>> looking forward to seeing this in production!
> > > > > >>>>>>>>
> > > > > >>>>>>>> Cheers,
> > > > > >>>>>>>> Lucas
> > > > > >>>>>>>>
> > > > > >>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> > > colt@littlehorse.io
> > > > > >
> > > > > >>>>> wrote:
> > > > > >>>>>>>>>
> > > > > >>>>>>>>>> Making IsolationLevel a query-time constraint, rather
> than
> > > > > linking
> > > > > >>>> it
> > > > > >>>>>>>> to
> > > > > >>>>>>>>> the processing.guarantee.
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> As I understand it, would this allow even a user of EOS
> to
> > > > > control
> > > > > >>>>>>>> whether
> > > > > >>>>>>>>> reading committed or uncommitted records? If so, I am
> highly
> > > in
> > > > > >>>> favor
> > > > > >>>>> of
> > > > > >>>>>>>>> this.
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> I know that I was one of the early people to point out
> the
> > > > > current
> > > > > >>>>>>>>> shortcoming that IQ reads uncommitted records, but just
> this
> > > > > >>>> morning I
> > > > > >>>>>>>>> realized a pattern we use which means that (for certain
> > > queries)
> > > > > >> our
> > > > > >>>>>>>> system
> > > > > >>>>>>>>> needs to be able to read uncommitted records, which is
> the
> > > > > current
> > > > > >>>>>>>> behavior
> > > > > >>>>>>>>> of Kafka Streams in EOS.***
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> If IsolationLevel being a query-time decision allows for
> > > this,
> > > > > then
> > > > > >>>>> that
> > > > > >>>>>>>>> would be amazing. I would also vote that the default
> behavior
> > > > > >> should
> > > > > >>>>> be
> > > > > >>>>>>>> for
> > > > > >>>>>>>>> reading uncommitted records, because it is totally
> possible
> > > for a
> > > > > >>>>> valid
> > > > > >>>>>>>>> application to depend on that behavior, and breaking it
> in a
> > > > > minor
> > > > > >>>>>>>> release
> > > > > >>>>>>>>> might be a bit strong.
> > > > > >>>>>>>>>
> > > > > >>>>>>>>> *** (Note, for the curious reader....) Our use-case/query
> > > pattern
> > > > > >>>> is a
> > > > > >>>>>>>> bit
> > > > > >>>>>>>>> complex, but reading "uncommitted" records is actually
> safe
> > > in
> > > > > our
> > > > > >>>>> case
> > > > > >>>>>>>>> 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 soon as the processor processes the
> > > result.
> > > > > If
> > > > > >>>>> they
> > > > > >>>>>>>>> can't read uncommitted records, they may receive a "201 -
> > > > > Created"
> > > > > >>>>>>>>> response, immediately followed by a "404 - 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 <
> > > > > >>>> nick.telford@gmail.com>
> > > > > >>>>>>>> wrote:
> > > > > >>>>>>>>>
> > > > > >>>>>>>>>> Addendum:
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> I think we would also face the same problem with the
> > > approach
> > > > > John
> > > > > >>>>>>>> outlined
> > > > > >>>>>>>>>> earlier (using the record cache as a transaction buffer
> and
> > > > > >>>> flushing
> > > > > >>>>>>>> it
> > > > > >>>>>>>>>> straight to SST files). This is because the record cache
> > > (the
> > > > > >>>>>>>> ThreadCache
> > > > > >>>>>>>>>> class) is not thread-safe, so every commit would
> invalidate
> > > open
> > > > > >> IQ
> > > > > >>>>>>>>>> Iterators in the same way that RocksDB WriteBatches do.
> > > > > >>>>>>>>>> --
> > > > > >>>>>>>>>> Nick
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > > > > >>>> nick.telford@gmail.com>
> > > > > >>>>>>>>>> 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.
> > > > > >>>>>>>>>>> 2. Making IsolationLevel a query-time constraint,
> rather
> > > than
> > > > > >>>>>>>> linking it
> > > > > >>>>>>>>>>> to the processing.guarantee.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> There's a wrinkle that makes this a challenge:
> Interactive
> > > > > >> Queries
> > > > > >>>>>>>> that
> > > > > >>>>>>>>>>> open an Iterator, when using transactions and
> > > READ_UNCOMMITTED.
> > > > > >>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries
> need
> > > to be
> > > > > >>>> able
> > > > > >>>>>>>> to
> > > > > >>>>>>>>>>> read records from the currently uncommitted transaction
> > > buffer
> > > > > >>>>>>>>>>> (WriteBatch). This includes for Iterators, which should
> > > iterate
> > > > > >>>>>>>> both the
> > > > > >>>>>>>>>>> transaction buffer and underlying database (using
> > > > > >>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> The issue is that when the StreamThread commits, it
> writes
> > > the
> > > > > >>>>>>>> current
> > > > > >>>>>>>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*.
> > > > > Clearing
> > > > > >>>> the
> > > > > >>>>>>>>>>> WriteBatch while an Interactive Query holds an open
> > > Iterator on
> > > > > >> it
> > > > > >>>>>>>> will
> > > > > >>>>>>>>>>> invalidate the Iterator. Worse, it turns out that
> Iterators
> > > > > over
> > > > > >> a
> > > > > >>>>>>>>>>> WriteBatch become invalidated not just when the
> WriteBatch
> > > is
> > > > > >>>>>>>> cleared,
> > > > > >>>>>>>>>> but
> > > > > >>>>>>>>>>> also when the Iterators' current key receives a new
> write.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> Now that I'm writing this, I remember that this is the
> > > major
> > > > > >>>> reason
> > > > > >>>>>>>> that
> > > > > >>>>>>>>>> I
> > > > > >>>>>>>>>>> switched the original design from having a query-time
> > > > > >>>>>>>> IsolationLevel to
> > > > > >>>>>>>>>>> having the IsolationLevel linked to the
> transactionality
> > > of the
> > > > > >>>>>>>> stores
> > > > > >>>>>>>>>>> themselves.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> It *might* be possible to resolve this, by having a
> > > "chain" of
> > > > > >>>>>>>>>>> WriteBatches, with the StreamThread switching to a new
> > > > > WriteBatch
> > > > > >>>>>>>>>> whenever
> > > > > >>>>>>>>>>> a new Interactive Query attempts to read from the
> > > database, but
> > > > > >>>> that
> > > > > >>>>>>>>>> could
> > > > > >>>>>>>>>>> cause some performance problems/memory pressure when
> > > subjected
> > > > > to
> > > > > >>>> a
> > > > > >>>>>>>> high
> > > > > >>>>>>>>>>> Interactive Query load. It would also reduce the
> > > efficiency of
> > > > > >>>>>>>>>> WriteBatches
> > > > > >>>>>>>>>>> on-commit, as we'd have to write N WriteBatches, where
> N
> > > is the
> > > > > >>>>>>>> number of
> > > > > >>>>>>>>>>> Interactive Queries since the last commit.
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> I realise this is getting into the weeds of the
> > > implementation,
> > > > > >>>> and
> > > > > >>>>>>>> you'd
> > > > > >>>>>>>>>>> rather we focus on the API for now, but I think it's
> > > important
> > > > > to
> > > > > >>>>>>>>>> consider
> > > > > >>>>>>>>>>> how to implement the desired API, in case we come up
> with
> > > an
> > > > > API
> > > > > >>>>>>>> that
> > > > > >>>>>>>>>>> cannot be implemented efficiently, or even at all!
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> Thoughts?
> > > > > >>>>>>>>>>> --
> > > > > >>>>>>>>>>> Nick
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> > > > > cadonna@apache.org
> > > > > >>>
> > > > > >>>>>>>> wrote:
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>>> Hi Nick,
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> 6.
> > > > > >>>>>>>>>>>> Of course, you are right! My bad!
> > > > > >>>>>>>>>>>> Wiping out the state in the downgrading case is fine.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> 3a.
> > > > > >>>>>>>>>>>> Focus on the public facing changes for the KIP. We
> will
> > > manage
> > > > > >> 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.
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> Best,
> > > > > >>>>>>>>>>>> Bruno
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > > > > >>>>>>>>>>>>> Hi Bruno,
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Thanks for getting back to me!
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> 2.
> > > > > >>>>>>>>>>>>> The fact that implementations can always track
> estimated
> > > > > memory
> > > > > >>>>>>>> usage
> > > > > >>>>>>>>>> in
> > > > > >>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as an
> > > option,
> > > > > and
> > > > > >>>>>>>> I'll
> > > > > >>>>>>>>>>>> clarify
> > > > > >>>>>>>>>>>>> the JavaDoc that 0 is not just for non-transactional
> > > stores,
> > > > > >>>>>>>> which is
> > > > > >>>>>>>>>>>>> currently misleading.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> 6.
> > > > > >>>>>>>>>>>>> The problem with catching the exception in the
> downgrade
> > > > > >> process
> > > > > >>>>>>>> is
> > > > > >>>>>>>>>> that
> > > > > >>>>>>>>>>>>> would require new code in the Kafka version being
> > > downgraded
> > > > > >> to.
> > > > > >>>>>>>> Since
> > > > > >>>>>>>>>>>>> users could conceivably downgrade to almost *any*
> older
> > > > > version
> > > > > >>>>>>>> of
> > > > > >>>>>>>>>> Kafka
> > > > > >>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
> > > > > >>>>>>>>>>>>> The only way I can think of doing it would be to
> provide
> > > a
> > > > > >>>>>>>> dedicated
> > > > > >>>>>>>>>>>>> downgrade tool, that goes through every local store
> and
> > > > > removes
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>>> offsets column families. But that seems like an
> > > unnecessary
> > > > > >>>>>>>> amount of
> > > > > >>>>>>>>>>>> extra
> > > > > >>>>>>>>>>>>> code to maintain just to handle a somewhat niche
> > > situation,
> > > > > >> when
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>>> alternative (automatically wipe and restore stores)
> > > should be
> > > > > >>>>>>>>>>>> acceptable.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've
> requested.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> 3a.
> > > > > >>>>>>>>>>>>> I agree that IsolationLevel makes more sense at
> > > query-time,
> > > > > and
> > > > > >>>> I
> > > > > >>>>>>>>>>>> actually
> > > > > >>>>>>>>>>>>> initially attempted to place the IsolationLevel at
> > > > > query-time,
> > > > > >>>>>>>> but I
> > > > > >>>>>>>>>> ran
> > > > > >>>>>>>>>>>>> into some problems:
> > > > > >>>>>>>>>>>>> - The key issue is that, under ALOS we're not staging
> > > writes
> > > > > in
> > > > > >>>>>>>>>>>>> transactions, so can't perform writes at the
> > > READ_COMMITTED
> > > > > >>>>>>>> isolation
> > > > > >>>>>>>>>>>>> level. However, this may be addressed if we decide to
> > > > > *always*
> > > > > >>>>>>>> use
> > > > > >>>>>>>>>>>>> transactions as discussed under 3b.
> > > > > >>>>>>>>>>>>> - IQv1 and IQv2 have quite different
> implementations. I
> > > > > >> remember
> > > > > >>>>>>>>>> having
> > > > > >>>>>>>>>>>>> some difficulty understanding the IQv1 internals,
> which
> > > made
> > > > > it
> > > > > >>>>>>>>>>>> difficult
> > > > > >>>>>>>>>>>>> to determine what needed to be changed. However, I
> > > *think*
> > > > > this
> > > > > >>>>>>>> can be
> > > > > >>>>>>>>>>>>> addressed for both implementations by wrapping the
> > > > > RocksDBStore
> > > > > >>>>>>>> in an
> > > > > >>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides read
> > > methods
> > > > > >>>>>>>> (get,
> > > > > >>>>>>>>>>>> etc.)
> > > > > >>>>>>>>>>>>> to either read directly from the database or from the
> > > ongoing
> > > > > >>>>>>>>>>>> transaction.
> > > > > >>>>>>>>>>>>> But IQv1 might still be difficult.
> > > > > >>>>>>>>>>>>> - If IsolationLevel becomes a query constraint, then
> all
> > > > > other
> > > > > >>>>>>>>>>>> StateStores
> > > > > >>>>>>>>>>>>> will need to respect it, including the in-memory
> stores.
> > > This
> > > > > >>>>>>>> would
> > > > > >>>>>>>>>>>> require
> > > > > >>>>>>>>>>>>> us to adapt in-memory stores to stage their writes so
> > > they
> > > > > can
> > > > > >>>> be
> > > > > >>>>>>>>>>>> isolated
> > > > > >>>>>>>>>>>>> from READ_COMMITTTED queries. It would also become an
> > > > > important
> > > > > >>>>>>>>>>>>> consideration for third-party stores on upgrade, as
> > > without
> > > > > >>>>>>>> changes,
> > > > > >>>>>>>>>>>> they
> > > > > >>>>>>>>>>>>> would not support READ_COMMITTED queries correctly.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Ultimately, I may need some help making the necessary
> > > change
> > > > > to
> > > > > >>>>>>>> IQv1
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>>> support this, but I don't think it's fundamentally
> > > > > impossible,
> > > > > >>>>>>>> if we
> > > > > >>>>>>>>>>>> want
> > > > > >>>>>>>>>>>>> to pursue this route.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> 3b.
> > > > > >>>>>>>>>>>>> The main reason I chose to keep ALOS un-transactional
> > > was to
> > > > > >>>>>>>> minimize
> > > > > >>>>>>>>>>>>> behavioural change for most users (I believe most
> Streams
> > > > > users
> > > > > >>>>>>>> use
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>> default configuration, which is ALOS). That said,
> it's
> > > clear
> > > > > >>>>>>>> that if
> > > > > >>>>>>>>>>>> ALOS
> > > > > >>>>>>>>>>>>> also used transactional stores, the only change in
> > > behaviour
> > > > > >>>>>>>> would be
> > > > > >>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>> it would become *more correct*, which could be
> > > considered a
> > > > > >> "bug
> > > > > >>>>>>>> fix"
> > > > > >>>>>>>>>> by
> > > > > >>>>>>>>>>>>> users, rather than a change they need to handle.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> I believe that performance using transactions (aka.
> > > RocksDB
> > > > > >>>>>>>>>>>> WriteBatches)
> > > > > >>>>>>>>>>>>> should actually be *better* than the un-batched
> > > write-path
> > > > > that
> > > > > >>>>>>>> is
> > > > > >>>>>>>>>>>>> currently used[1]. The only "performance"
> consideration
> > > will
> > > > > be
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>>> increased memory usage that transactions require.
> Given
> > > the
> > > > > >>>>>>>>>> mitigations
> > > > > >>>>>>>>>>>> for
> > > > > >>>>>>>>>>>>> this memory that we have in place, I would expect
> that
> > > this
> > > > > is
> > > > > >>>>>>>> not a
> > > > > >>>>>>>>>>>>> problem for most users.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> If we're happy to do so, we can make ALOS also use
> > > > > >> transactions.
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Regards,
> > > > > >>>>>>>>>>>>> Nick
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> Link 1:
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>
> > > > > >>
> > >
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> > > > > >>>> cadonna@apache.org
> > > > > >>>>>>>>>
> > > > > >>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Hi Nick,
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Thanks for the updates and sorry for the delay on my
> > > side!
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> 1.
> > > > > >>>>>>>>>>>>>> Making the default implementation for flush() a
> no-op
> > > sounds
> > > > > >>>>>>>> good to
> > > > > >>>>>>>>>>>> me.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> 2.
> > > > > >>>>>>>>>>>>>> I think what was bugging me here is that a
> third-party
> > > state
> > > > > >>>>>>>> store
> > > > > >>>>>>>>>>>> needs
> > > > > >>>>>>>>>>>>>> to implement the state store interface. That means
> they
> > > need
> > > > > >> to
> > > > > >>>>>>>>>>>>>> implement a wrapper around the actual state store
> as we
> > > do
> > > > > for
> > > > > >>>>>>>>>> RocksDB
> > > > > >>>>>>>>>>>>>> with RocksDBStore. So, a third-party state store can
> > > always
> > > > > >>>>>>>> estimate
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>> uncommitted bytes, if it wants, because the wrapper
> can
> > > > > record
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>> added
> > > > > >>>>>>>>>>>>>> bytes.
> > > > > >>>>>>>>>>>>>> One case I can think of where returning -1 makes
> sense
> > > is
> > > > > when
> > > > > >>>>>>>>>> Streams
> > > > > >>>>>>>>>>>>>> does not need to estimate the size of the write
> batch
> > > and
> > > > > >>>>>>>> trigger
> > > > > >>>>>>>>>>>>>> extraordinary commits, because the third-party state
> > > store
> > > > > >>>>>>>> takes care
> > > > > >>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>> memory. But in that case the method could also just
> > > return
> > > > > 0.
> > > > > >>>>>>>> Even
> > > > > >>>>>>>>>> that
> > > > > >>>>>>>>>>>>>> case would be better solved with a method that
> returns
> > > > > whether
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>> state
> > > > > >>>>>>>>>>>>>> store manages itself the memory used for uncommitted
> > > bytes
> > > > > or
> > > > > >>>>>>>> not.
> > > > > >>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return
> value,
> > > I was
> > > > > >>>>>>>> just
> > > > > >>>>>>>>>>>>>> wondering when and if it will be used.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Regarding returning 0 for transactional state stores
> > > when
> > > > > the
> > > > > >>>>>>>> batch
> > > > > >>>>>>>>>> is
> > > > > >>>>>>>>>>>>>> empty, I was just wondering because you explicitly
> > > stated
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> "or {@code 0} if this StateStore does not support
> > > > > >>>> transactions."
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> So it seemed to me returning 0 could only happen for
> > > > > >>>>>>>>>> non-transactional
> > > > > >>>>>>>>>>>>>> state stores.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> 3.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> a) What do you think if we move the isolation level
> to
> > > IQ
> > > > > (v1
> > > > > >>>>>>>> and
> > > > > >>>>>>>>>> v2)?
> > > > > >>>>>>>>>>>>>> In the end this is the only component that really
> needs
> > > to
> > > > > >>>>>>>> specify
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>> isolation level. It is similar to the Kafka consumer
> > > that
> > > > > can
> > > > > >>>>>>>> choose
> > > > > >>>>>>>>>>>>>> with what isolation level to read the input topic.
> > > > > >>>>>>>>>>>>>> For IQv1 the isolation level should go into
> > > > > >>>>>>>> StoreQueryParameters. For
> > > > > >>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> b) Point a) raises the question what should happen
> > > during
> > > > > >>>>>>>>>> at-least-once
> > > > > >>>>>>>>>>>>>> processing when the state store does not use
> > > transactions?
> > > > > >> John
> > > > > >>>>>>>> in
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>> past proposed to also use transactions on state
> stores
> > > for
> > > > > >>>>>>>>>>>>>> at-least-once. I like that idea, because it avoids
> > > > > aggregating
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>> same
> > > > > >>>>>>>>>>>>>> records over and over again in the case of a
> failure. We
> > > > > had a
> > > > > >>>>>>>> case
> > > > > >>>>>>>>>> in
> > > > > >>>>>>>>>>>>>> the past where a Streams applications in
> at-least-once
> > > mode
> > > > > >> was
> > > > > >>>>>>>>>> failing
> > > > > >>>>>>>>>>>>>> continuously for some reasons I do not remember
> before
> > > > > >>>>>>>> committing the
> > > > > >>>>>>>>>>>>>> offsets. After each failover, the app aggregated
> again
> > > and
> > > > > >>>>>>>> again the
> > > > > >>>>>>>>>>>>>> same records. Of course the aggregate increased to
> very
> > > > > wrong
> > > > > >>>>>>>> values
> > > > > >>>>>>>>>>>>>> just because of the failover. With transactions on
> the
> > > state
> > > > > >>>>>>>> stores
> > > > > >>>>>>>>>> we
> > > > > >>>>>>>>>>>>>> could have avoided this. The app would have output
> the
> > > same
> > > > > >>>>>>>> aggregate
> > > > > >>>>>>>>>>>>>> multiple times (i.e., after each failover) but at
> least
> > > the
> > > > > >>>>>>>> value of
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>> aggregate would not depend on the number of
> failovers.
> > > > > >>>>>>>> Outputting the
> > > > > >>>>>>>>>>>>>> same aggregate multiple times would be incorrect
> under
> > > > > >>>>>>>> exactly-once
> > > > > >>>>>>>>>> but
> > > > > >>>>>>>>>>>>>> it is OK for at-least-once.
> > > > > >>>>>>>>>>>>>> If it makes sense to add a config to turn on and off
> > > > > >>>>>>>> transactions on
> > > > > >>>>>>>>>>>>>> state stores under at-least-once or just use
> > > transactions in
> > > > > >>>>>>>> any case
> > > > > >>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>> a question we should also discuss in this KIP. It
> > > depends a
> > > > > >> bit
> > > > > >>>>>>>> on
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would
> add a
> > > > > config.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> 4.
> > > > > >>>>>>>>>>>>>> Your points are all valid. I tend to say to keep the
> > > metrics
> > > > > >>>>>>>> around
> > > > > >>>>>>>>>>>>>> flush() until we remove flush() completely from the
> > > > > interface.
> > > > > >>>>>>>> Calls
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>>>> flush() might still exist since existing processors
> > > might
> > > > > >> still
> > > > > >>>>>>>> call
> > > > > >>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For
> sure, we
> > > need
> > > > > >> to
> > > > > >>>>>>>>>>>> document
> > > > > >>>>>>>>>>>>>> how the metrics change due to the transactions in
> the
> > > > > upgrade
> > > > > >>>>>>>> notes.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> 5.
> > > > > >>>>>>>>>>>>>> I see. Then you should describe how the .position
> files
> > > are
> > > > > >>>>>>>> handled
> > > > > >>>>>>>>>> in
> > > > > >>>>>>>>>>>>>> a dedicated section of the KIP or incorporate the
> > > > > description
> > > > > >>>>>>>> in the
> > > > > >>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only
> > > mentioning it
> > > > > >> in
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>>>> "Compatibility, 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:
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>
> > > > > >>>>
> > > > > >>
> > > > >
> > >
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> It is a bit brittle, but it works.
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> Best,
> > > > > >>>>>>>>>>>>>> Bruno
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > > > > >>>>>>>>>>>>>>> Hi Bruno,
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Thanks for taking the time to review the KIP. I'm
> back
> > > from
> > > > > >>>>>>>> leave
> > > > > >>>>>>>>>> now
> > > > > >>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>> intend to move this forwards as quickly as I can.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Addressing your points:
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> 1.
> > > > > >>>>>>>>>>>>>>> Because flush() is part of the StateStore API, it's
> > > exposed
> > > > > >> to
> > > > > >>>>>>>>>> custom
> > > > > >>>>>>>>>>>>>>> Processors, which might be making calls to flush().
> > > This
> > > > > was
> > > > > >>>>>>>>>> actually
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> case in a few integration tests.
> > > > > >>>>>>>>>>>>>>> To maintain as much compatibility as possible, I'd
> > > prefer
> > > > > not
> > > > > >>>>>>>> to
> > > > > >>>>>>>>>> make
> > > > > >>>>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>> an UnsupportedOperationException, as it will cause
> > > > > previously
> > > > > >>>>>>>>>> working
> > > > > >>>>>>>>>>>>>>> Processors to start throwing exceptions at runtime.
> > > > > >>>>>>>>>>>>>>> I agree that it doesn't make sense for it to proxy
> > > > > commit(),
> > > > > >>>>>>>> though,
> > > > > >>>>>>>>>>>> as
> > > > > >>>>>>>>>>>>>>> that would cause it to violate the "StateStores
> commit
> > > only
> > > > > >>>>>>>> when the
> > > > > >>>>>>>>>>>> Task
> > > > > >>>>>>>>>>>>>>> commits" rule.
> > > > > >>>>>>>>>>>>>>> Instead, I think we should make this a no-op. That
> way,
> > > > > >>>>>>>> existing
> > > > > >>>>>>>>>> user
> > > > > >>>>>>>>>>>>>>> Processors will continue to work as-before, without
> > > > > violation
> > > > > >>>>>>>> of
> > > > > >>>>>>>>>> store
> > > > > >>>>>>>>>>>>>>> consistency that would be caused by premature
> > > flush/commit
> > > > > of
> > > > > >>>>>>>>>>>> StateStore
> > > > > >>>>>>>>>>>>>>> data to disk.
> > > > > >>>>>>>>>>>>>>> What do you think?
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> 2.
> > > > > >>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
> > > implementation
> > > > > is
> > > > > >>>>>>>>>>>>>>> transactional, but is unable to estimate the
> > > uncommitted
> > > > > >>>> memory
> > > > > >>>>>>>>>> usage,
> > > > > >>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> method will return -1.
> > > > > >>>>>>>>>>>>>>> The intention here is to permit third-party
> > > implementations
> > > > > >>>>>>>> that may
> > > > > >>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>> able to estimate memory usage.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been written to
> the
> > > > > store
> > > > > >>>>>>>> yet. I
> > > > > >>>>>>>>>>>>>> thought
> > > > > >>>>>>>>>>>>>>> that was implied by "This method will return an
> > > > > approximation
> > > > > >>>>>>>> of the
> > > > > >>>>>>>>>>>>>> memory
> > > > > >>>>>>>>>>>>>>> would be freed by the next call to {@link
> > > #commit(Map)}"
> > > > > and
> > > > > >>>>>>>>>> "@return
> > > > > >>>>>>>>>>>> The
> > > > > >>>>>>>>>>>>>>> approximate size of all records awaiting {@link
> > > > > >>>> #commit(Map)}",
> > > > > >>>>>>>>>>>> however,
> > > > > >>>>>>>>>>>>>> I
> > > > > >>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you think
> this
> > > is
> > > > > >>>>>>>> unclear?
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> 3.
> > > > > >>>>>>>>>>>>>>> I realise this is probably the most contentious
> point
> > > in my
> > > > > >>>>>>>> design,
> > > > > >>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>> I'm
> > > > > >>>>>>>>>>>>>>> open to changing it if I'm unable to convince you
> of
> > > the
> > > > > >>>>>>>> benefits.
> > > > > >>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > > > > >>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly
> provided
> > > > > >>>>>>>> StateStores
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>> query, and it may be important for users to
> > > > > programmatically
> > > > > >>>>>>>> know
> > > > > >>>>>>>>>>>> which
> > > > > >>>>>>>>>>>>>>> mode the StateStore is operating under. If we
> simply
> > > > > provide
> > > > > >>>> an
> > > > > >>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the
> internal
> > > > > streams
> > > > > >>>>>>>>>>>> engine), or
> > > > > >>>>>>>>>>>>>>> similar, then users will need to understand the
> > > operation
> > > > > and
> > > > > >>>>>>>>>>>>>> consequences
> > > > > >>>>>>>>>>>>>>> of each available processing mode and how it
> pertains
> > > to
> > > > > >> their
> > > > > >>>>>>>>>>>>>> StateStore.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Interactive Query users aren't the only people that
> > > care
> > > > > >> about
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
> > > > > implementers
> > > > > >>>> of
> > > > > >>>>>>>>>> custom
> > > > > >>>>>>>>>>>>>>> StateStores also need to understand the behaviour
> > > expected
> > > > > of
> > > > > >>>>>>>> their
> > > > > >>>>>>>>>>>>>>> implementation. KIP-892 introduces some assumptions
> > > into
> > > > > the
> > > > > >>>>>>>> Streams
> > > > > >>>>>>>>>>>>>> Engine
> > > > > >>>>>>>>>>>>>>> about how StateStores operate under each processing
> > > mode,
> > > > > and
> > > > > >>>>>>>> it's
> > > > > >>>>>>>>>>>>>>> important that custom implementations adhere to
> those
> > > > > >>>>>>>> assumptions in
> > > > > >>>>>>>>>>>>>> order
> > > > > >>>>>>>>>>>>>>> to maintain the consistency guarantees.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> IsolationLevels provide a high-level contract on
> the
> > > > > >> behaviour
> > > > > >>>>>>>> of
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>> StateStore: a user knows that under READ_COMMITTED,
> > > they
> > > > > will
> > > > > >>>>>>>> see
> > > > > >>>>>>>>>>>> writes
> > > > > >>>>>>>>>>>>>>> only after the Task has committed, and under
> > > > > READ_UNCOMMITTED
> > > > > >>>>>>>> they
> > > > > >>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>> see
> > > > > >>>>>>>>>>>>>>> writes immediately. No understanding of the
> details of
> > > each
> > > > > >>>>>>>>>>>>>> processing.mode
> > > > > >>>>>>>>>>>>>>> is required, either for IQ users or StateStore
> > > > > implementers.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> An argument can be made that these contractual
> > > guarantees
> > > > > can
> > > > > >>>>>>>> simply
> > > > > >>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>> documented for the processing.mode (i.e. that
> > > exactly-once
> > > > > >> and
> > > > > >>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
> > > > > at-least-once
> > > > > >>>>>>>> behaves
> > > > > >>>>>>>>>>>> like
> > > > > >>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small
> issues
> > > with
> > > > > >>>>>>>> this I'd
> > > > > >>>>>>>>>>>>>> prefer
> > > > > >>>>>>>>>>>>>>> to avoid:
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>         - Where would we document these contracts,
> in
> > > a way
> > > > > >> that
> > > > > >>>>>>>> is
> > > > > >>>>>>>>>>>> difficult
> > > > > >>>>>>>>>>>>>>>         for users/implementers to miss/ignore?
> > > > > >>>>>>>>>>>>>>>         - It's not clear to users that the
> processing
> > > mode
> > > > > is
> > > > > >>>>>>>>>>>> communicating
> > > > > >>>>>>>>>>>>>>>         an expectation of read isolation, unless
> they
> > > read
> > > > > the
> > > > > >>>>>>>>>>>>>> documentation. Users
> > > > > >>>>>>>>>>>>>>>         rarely consult documentation unless they
> feel
> > > they
> > > > > >> need
> > > > > >>>>>>>> to, so
> > > > > >>>>>>>>>>>> it's
> > > > > >>>>>>>>>>>>>> likely
> > > > > >>>>>>>>>>>>>>>         this detail would get missed by many users.
> > > > > >>>>>>>>>>>>>>>         - It tightly couples processing modes to
> read
> > > > > >> isolation.
> > > > > >>>>>>>> Adding
> > > > > >>>>>>>>>>>> new
> > > > > >>>>>>>>>>>>>>>         processing modes, or changing the read
> > > isolation of
> > > > > >>>>>>>> existing
> > > > > >>>>>>>>>>>>>> processing
> > > > > >>>>>>>>>>>>>>>         modes would be difficult/impossible.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Ultimately, the cost of introducing
> IsolationLevels is
> > > > > just a
> > > > > >>>>>>>> single
> > > > > >>>>>>>>>>>>>>> method, since we re-use the existing IsolationLevel
> > > enum
> > > > > from
> > > > > >>>>>>>> Kafka.
> > > > > >>>>>>>>>>>> This
> > > > > >>>>>>>>>>>>>>> gives us a clear place to document the contractual
> > > > > guarantees
> > > > > >>>>>>>>>> expected
> > > > > >>>>>>>>>>>>>>> of/provided by StateStores, that is accessible
> both by
> > > the
> > > > > >>>>>>>>>> StateStore
> > > > > >>>>>>>>>>>>>>> itself, and by IQ users.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> (Writing this I've just realised that the
> StateStore
> > > and IQ
> > > > > >>>>>>>> APIs
> > > > > >>>>>>>>>>>> actually
> > > > > >>>>>>>>>>>>>>> don't provide access to StateStoreContext that IQ
> users
> > > > > would
> > > > > >>>>>>>> have
> > > > > >>>>>>>>>>>> direct
> > > > > >>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> > > > > >> isolationLevel()
> > > > > >>>>>>>>>> itself
> > > > > >>>>>>>>>>>>>> too?)
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> 4.
> > > > > >>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics
> in-place
> > > > > >>>>>>>> either, as
> > > > > >>>>>>>>>>>> it's a
> > > > > >>>>>>>>>>>>>>> backwards incompatible change. My concern is that,
> if
> > > we
> > > > > >> leave
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>>>> existing
> > > > > >>>>>>>>>>>>>>> "flush" metrics in place, they will be confusing to
> > > users.
> > > > > >>>>>>>> Right
> > > > > >>>>>>>>>> now,
> > > > > >>>>>>>>>>>>>>> "flush" metrics record explicit flushes to disk,
> but
> > > under
> > > > > >>>>>>>> KIP-892,
> > > > > >>>>>>>>>>>> even
> > > > > >>>>>>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>> commit() will not explicitly flush data to disk -
> > > RocksDB
> > > > > >> will
> > > > > >>>>>>>>>> decide
> > > > > >>>>>>>>>>>> on
> > > > > >>>>>>>>>>>>>>> when to flush memtables to disk itself.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd have
> two
> > > > > >>>> options,
> > > > > >>>>>>>>>> which
> > > > > >>>>>>>>>>>>>> both
> > > > > >>>>>>>>>>>>>>> seem pretty bad to me:
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>         1. Have them record calls to commit(),
> which
> > > would
> > > > > be
> > > > > >>>>>>>>>>>> misleading, as
> > > > > >>>>>>>>>>>>>>>         data is no longer explicitly "flushed" to
> disk
> > > by
> > > > > this
> > > > > >>>>>>>> call.
> > > > > >>>>>>>>>>>>>>>         2. Have them record nothing at all, which
> is
> > > > > >> equivalent
> > > > > >>>> to
> > > > > >>>>>>>>>>>> removing
> > > > > >>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>         metrics, except that users will see the
> metric
> > > > > still
> > > > > >>>>>>>> exists and
> > > > > >>>>>>>>>>>> so
> > > > > >>>>>>>>>>>>>> assume
> > > > > >>>>>>>>>>>>>>>         that the metric is correct, and that
> there's a
> > > > > problem
> > > > > >>>>>>>> with
> > > > > >>>>>>>>>> their
> > > > > >>>>>>>>>>>>>> system
> > > > > >>>>>>>>>>>>>>>         when there isn't.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> I agree that removing them is also a bad solution,
> and
> > > I'd
> > > > > >>>>>>>> like some
> > > > > >>>>>>>>>>>>>>> guidance on the best path forward here.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> 5.
> > > > > >>>>>>>>>>>>>>> Position files are updated on every write to a
> > > StateStore.
> > > > > >>>>>>>> Since our
> > > > > >>>>>>>>>>>>>> writes
> > > > > >>>>>>>>>>>>>>> are now buffered until commit(), we can't update
> the
> > > > > Position
> > > > > >>>>>>>> file
> > > > > >>>>>>>>>>>> until
> > > > > >>>>>>>>>>>>>>> commit() has been called, otherwise it would be
> > > > > inconsistent
> > > > > >>>>>>>> with
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>> data
> > > > > >>>>>>>>>>>>>>> in the event of a rollback. Consequently, we need
> to
> > > manage
> > > > > >>>>>>>> these
> > > > > >>>>>>>>>>>> offsets
> > > > > >>>>>>>>>>>>>>> the same way we manage the checkpoint offsets, and
> > > ensure
> > > > > >>>>>>>> they're
> > > > > >>>>>>>>>> only
> > > > > >>>>>>>>>>>>>>> written on commit().
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> 6.
> > > > > >>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what
> tests to
> > > > > >> write.
> > > > > >>>>>>>> How
> > > > > >>>>>>>>>>>>>> explicit
> > > > > >>>>>>>>>>>>>>> do we need to be here in the KIP?
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be
> > > > > seamless,
> > > > > >>>>>>>> and we
> > > > > >>>>>>>>>>>>>> should
> > > > > >>>>>>>>>>>>>>> definitely add some tests around that. Downgrade,
> it
> > > > > >>>>>>>> transpires,
> > > > > >>>>>>>>>> isn't
> > > > > >>>>>>>>>>>>>>> currently possible, as the extra column family for
> > > offset
> > > > > >>>>>>>> storage is
> > > > > >>>>>>>>>>>>>>> incompatible with the pre-KIP-892 implementation:
> when
> > > you
> > > > > >>>>>>>> open a
> > > > > >>>>>>>>>>>> RocksDB
> > > > > >>>>>>>>>>>>>>> database, you must open all available column
> families
> > > or
> > > > > >>>>>>>> receive an
> > > > > >>>>>>>>>>>>>> error.
> > > > > >>>>>>>>>>>>>>> What currently happens on downgrade is that it
> > > attempts to
> > > > > >>>>>>>> open the
> > > > > >>>>>>>>>>>>>> store,
> > > > > >>>>>>>>>>>>>>> throws an error about the offsets column family not
> > > being
> > > > > >>>>>>>> opened,
> > > > > >>>>>>>>>>>> which
> > > > > >>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given that
> > > > > >> downgrades
> > > > > >>>>>>>>>> should
> > > > > >>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>> uncommon, I think this is acceptable behaviour, as
> the
> > > > > >>>>>>>> end-state is
> > > > > >>>>>>>>>>>>>>> consistent, even if it results in an undesirable
> state
> > > > > >>>> restore.
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Should I document the upgrade/downgrade behaviour
> > > > > explicitly
> > > > > >>>>>>>> in the
> > > > > >>>>>>>>>>>> KIP?
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> --
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> Regards,
> > > > > >>>>>>>>>>>>>>> Nick
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > > > > >>>>>>>> cadonna@apache.org>
> > > > > >>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Hi Nick!
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Thanks for the updates!
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> 1.
> > > > > >>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> > > > > >>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > > > > >>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore
> after
> > > this
> > > > > KIP
> > > > > >>>>>>>> is
> > > > > >>>>>>>>>>>>>>>> released, I would rather throw an unsupported
> > > operation
> > > > > >>>>>>>> exception
> > > > > >>>>>>>>>> by
> > > > > >>>>>>>>>>>>>>>> default.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> 2.
> > > > > >>>>>>>>>>>>>>>> When would a state store return -1 from
> > > > > >>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while
> > > being
> > > > > >>>>>>>>>>>> transactional?
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Wouldn't
> StateStore#approximateNumUncommittedBytes()
> > > also
> > > > > >>>>>>>> return 0
> > > > > >>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>> the state store is transactional but nothing has
> been
> > > > > >> written
> > > > > >>>>>>>> to
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> state store yet?
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> 3.
> > > > > >>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this KIP
> really
> > > > > need
> > > > > >>>> to
> > > > > >>>>>>>>>>>> introduce
> > > > > >>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()?
> StateStoreContext
> > > has
> > > > > >>>>>>>> already
> > > > > >>>>>>>>>>>>>>>> appConfigs() which basically exposes the same
> > > information,
> > > > > >>>>>>>> i.e., if
> > > > > >>>>>>>>>>>> EOS
> > > > > >>>>>>>>>>>>>>>> is enabled or not.
> > > > > >>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore
> interface
> > > as
> > > > > >>>>>>>> loosely
> > > > > >>>>>>>>>>>> coupled
> > > > > >>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > > implementers
> > > > > >>>> more
> > > > > >>>>>>>>>>>> freedom,
> > > > > >>>>>>>>>>>>>>>> and reduce the amount of internal knowledge
> required."
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> While I understand the intent, I doubt that it
> > > decreases
> > > > > the
> > > > > >>>>>>>>>>>> coupling of
> > > > > >>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> > > > > >> READ_COMMITTED
> > > > > >>>>>>>> only
> > > > > >>>>>>>>>>>>>>>> applies to IQ but not to reads by processors.
> Thus,
> > > > > >>>>>>>> implementers
> > > > > >>>>>>>>>>>> need to
> > > > > >>>>>>>>>>>>>>>> understand how Streams accesses the state stores.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> I would like to hear what others think about this.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> 4.
> > > > > >>>>>>>>>>>>>>>> Great exposing new metrics for transactional state
> > > stores!
> > > > > >>>>>>>>>> However, I
> > > > > >>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate (in
> the
> > > > > docs)
> > > > > >>>>>>>> the 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 also need to adapt/add integration
> > > tests
> > > > > >>>>>>>> besides
> > > > > >>>>>>>>>> unit
> > > > > >>>>>>>>>>>>>>>> tests. Additionally, we probably need integration
> or
> > > > > system
> > > > > >>>>>>>> tests
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>> verify that upgrades and downgrades between
> > > transactional
> > > > > >> and
> > > > > >>>>>>>>>>>>>>>> non-transactional state stores work as expected.
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> Best,
> > > > > >>>>>>>>>>>>>>>> Bruno
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > > > > >>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in the
> KIP,
> > > > > under
> > > > > >>>>>>>>>>>> "Rejected
> > > > > >>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea worth
> > > pursuing,
> > > > > >>>>>>>> but I
> > > > > >>>>>>>>>>>> believe
> > > > > >>>>>>>>>>>>>>>>> that it's out of the scope of this KIP, because
> it
> > > > > solves a
> > > > > >>>>>>>>>>>> different
> > > > > >>>>>>>>>>>>>> set
> > > > > >>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of this
> one
> > > has
> > > > > >>>>>>>> already
> > > > > >>>>>>>>>> grown
> > > > > >>>>>>>>>>>>>>>> quite
> > > > > >>>>>>>>>>>>>>>>> large!
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > > > > >>>>>>>>>> nick.telford@gmail.com>
> > > > > >>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> Hi everyone,
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> I've updated the KIP (
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>
> > > > > >>>>
> > > > > >>
> > > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > > >>>>>>>>>>>>>>>> )
> > > > > >>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing back
> > > "Atomic
> > > > > >>>>>>>>>>>> Checkpointing"
> > > > > >>>>>>>>>>>>>>>> (for
> > > > > >>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think the one
> > > thing
> > > > > >>>>>>>> missing is
> > > > > >>>>>>>>>>>> some
> > > > > >>>>>>>>>>>>>>>>>> changes to metrics (notably the store "flush"
> > > metrics
> > > > > will
> > > > > >>>>>>>> need
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>> renamed to "commit").
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> The reason I brought back Atomic Checkpointing
> was
> > > to
> > > > > >>>>>>>> decouple
> > > > > >>>>>>>>>>>> store
> > > > > >>>>>>>>>>>>>>>> flush
> > > > > >>>>>>>>>>>>>>>>>> from store commit. This is important, because
> with
> > > > > >>>>>>>> Transactional
> > > > > >>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on
> *every*
> > > Task
> > > > > >>>>>>>> commit,
> > > > > >>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>> just when the StateStore is closing, otherwise
> our
> > > > > >>>>>>>> transaction
> > > > > >>>>>>>>>>>> buffer
> > > > > >>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>> never be written and persisted, instead growing
> > > > > unbounded!
> > > > > >>>> I
> > > > > >>>>>>>>>>>>>>>> experimented
> > > > > >>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a store
> > > flush
> > > > > >>>>>>>> whenever
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its
> > > configured
> > > > > >>>>>>>> size, but
> > > > > >>>>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>> was
> > > > > >>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer
> from
> > > being
> > > > > >>>>>>>>>> configured
> > > > > >>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>> unbounded, and it still would have required
> explicit
> > > > > >>>>>>>> flushes of
> > > > > >>>>>>>>>>>>>> RocksDB,
> > > > > >>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory
> > > utilization.
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right"
> way
> > > to
> > > > > >>>>>>>> resolve
> > > > > >>>>>>>>>> this
> > > > > >>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog offsets
> that
> > > > > >>>>>>>> correspond
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> most
> > > > > >>>>>>>>>>>>>>>>>> recently written records are always atomically
> > > written
> > > > > to
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>>>> StateStore
> > > > > >>>>>>>>>>>>>>>>>> (by writing them to the same transaction
> buffer),
> > > we can
> > > > > >>>>>>>> avoid
> > > > > >>>>>>>>>>>>>> forcibly
> > > > > >>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting
> > > RocksDB
> > > > > >>>>>>>> flush
> > > > > >>>>>>>>>> them
> > > > > >>>>>>>>>>>>>> only
> > > > > >>>>>>>>>>>>>>>>>> when necessary, without losing any of our
> > > consistency
> > > > > >>>>>>>> guarantees.
> > > > > >>>>>>>>>>>> See
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>> updated KIP for more info.
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> I have fully implemented these changes,
> although I'm
> > > > > still
> > > > > >>>>>>>> not
> > > > > >>>>>>>>>>>>>> entirely
> > > > > >>>>>>>>>>>>>>>>>> happy with the implementation for segmented
> > > StateStores,
> > > > > >> so
> > > > > >>>>>>>> I
> > > > > >>>>>>>>>> plan
> > > > > >>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests pass. If
> > > you'd
> > > > > like
> > > > > >>>>>>>> to try
> > > > > >>>>>>>>>>>> out
> > > > > >>>>>>>>>>>>>> or
> > > > > >>>>>>>>>>>>>>>>>> review this highly experimental and incomplete
> > > branch,
> > > > > >> it's
> > > > > >>>>>>>>>>>> available
> > > > > >>>>>>>>>>>>>>>> here:
> > > > > >>>>>>>>>>>>>>>>>>
> > > https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > > > > .
> > > > > >>>>>>>> Note:
> > > > > >>>>>>>>>>>> it's
> > > > > >>>>>>>>>>>>>>>> built
> > > > > >>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base
> to
> > > build
> > > > > >>>>>>>> and test
> > > > > >>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>> on,
> > > > > >>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples comparisons
> in a
> > > > > live
> > > > > >>>>>>>>>>>>>> environment. I
> > > > > >>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's nearer
> > > > > >> completion
> > > > > >>>>>>>> and
> > > > > >>>>>>>>>> has
> > > > > >>>>>>>>>>>>>> been
> > > > > >>>>>>>>>>>>>>>>>> proven on our main application.
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing and
> > > testing:
> > > > > >>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window)
> stores
> > > > > >>>>>>>>>>>>>>>>>> - Global stores
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> As I do not currently use either of these, so my
> > > primary
> > > > > >>>>>>>> test
> > > > > >>>>>>>>>>>>>>>> environment
> > > > > >>>>>>>>>>>>>>>>>> doesn't test these areas.
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next week
> for
> > > a few
> > > > > >>>>>>>> weeks,
> > > > > >>>>>>>>>> so
> > > > > >>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>> not have time to move this forward until late
> > > August.
> > > > > That
> > > > > >>>>>>>> said,
> > > > > >>>>>>>>>>>> your
> > > > > >>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just
> won't be
> > > > > able
> > > > > >>>> to
> > > > > >>>>>>>>>>>> respond
> > > > > >>>>>>>>>>>>>> as
> > > > > >>>>>>>>>>>>>>>>>> quickly as usual.
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> Regards,
> > > > > >>>>>>>>>>>>>>>>>> Nick
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > > > > >>>>>>>>>> nick.telford@gmail.com>
> > > > > >>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> Hi Bruno
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ
> is
> > > not
> > > > > >>>>>>>> something
> > > > > >>>>>>>>>> I
> > > > > >>>>>>>>>>>> had
> > > > > >>>>>>>>>>>>>>>>>>> considered.
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> What about atomically updating the state store
> > > from the
> > > > > >>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> > > > > checkpoint
> > > > > >>>>>>>> (thus,
> > > > > >>>>>>>>>>>>>>>> flushing
> > > > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data
> > > and/or
> > > > > >>>>>>>> number of
> > > > > >>>>>>>>>>>>>> commit
> > > > > >>>>>>>>>>>>>>>>>>>> intervals?
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting
> > > that we
> > > > > >>>>>>>> add an
> > > > > >>>>>>>>>>>>>>>> additional
> > > > > >>>>>>>>>>>>>>>>>>> config for the max number of commit intervals
> > > between
> > > > > >>>>>>>>>> checkpoints?
> > > > > >>>>>>>>>>>>>> That
> > > > > >>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the
> > > transaction
> > > > > >>>>>>>> buffers
> > > > > >>>>>>>>>> are
> > > > > >>>>>>>>>>>>>>>> nearly
> > > > > >>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of commit
> > > > > intervals
> > > > > >>>>>>>> have
> > > > > >>>>>>>>>>>>>> elapsed,
> > > > > >>>>>>>>>>>>>>>>>>> whichever comes first?
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although this
> > > > > re-ignites
> > > > > >>>>>>>> an
> > > > > >>>>>>>>>>>> earlier
> > > > > >>>>>>>>>>>>>>>>>>> debate about whether a config should be
> measured in
> > > > > >>>>>>>> "number of
> > > > > >>>>>>>>>>>> commit
> > > > > >>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute time.
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the reason
> I
> > > was
> > > > > >>>>>>>> pursuing
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> Atomic
> > > > > >>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush
> from
> > > > > >>>>>>>> checkpointing,
> > > > > >>>>>>>>>>>>>> which
> > > > > >>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every commit
> > > without
> > > > > any
> > > > > >>>>>>>>>>>> performance
> > > > > >>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the
> > > "best"
> > > > > >>>>>>>> solution,
> > > > > >>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>> I'm not
> > > > > >>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back into
> this
> > > KIP.
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
> > > transactional
> > > > > >>>> logic
> > > > > >>>>>>>>>>>> directly
> > > > > >>>>>>>>>>>>>>>> into
> > > > > >>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with the
> > > > > >>>>>>>>>>>>>> StateStore#newTransaction
> > > > > >>>>>>>>>>>>>>>>>>> method, and reduces the number of new classes
> > > > > introduced,
> > > > > >>>>>>>>>>>>>> significantly
> > > > > >>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and the
> > > > > complexity
> > > > > >>>> is
> > > > > >>>>>>>>>>>>>> drastically
> > > > > >>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic
> Checkpoints
> > > > > into
> > > > > >>>>>>>> this
> > > > > >>>>>>>>>> KIP.
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> Regards,
> > > > > >>>>>>>>>>>>>>>>>>> Nick
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
> > > > > >>>>>>>> cadonna@apache.org>
> > > > > >>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> As far as I understand, you want to atomically
> > > update
> > > > > >> the
> > > > > >>>>>>>> state
> > > > > >>>>>>>>>>>>>> store
> > > > > >>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the
> memtable
> > > of a
> > > > > >>>> state
> > > > > >>>>>>>>>> store
> > > > > >>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit time
> > > elapsed
> > > > > >>>> but
> > > > > >>>>>>>>>> after
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that would
> lead
> > > to
> > > > > >>>>>>>> exceeding
> > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> before the
> > > > > next
> > > > > >>>>>>>> commit
> > > > > >>>>>>>>>>>>>>>> interval
> > > > > >>>>>>>>>>>>>>>>>>>> ends.
> > > > > >>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would commit
> > > every
> > > > > >>>>>>>> commit
> > > > > >>>>>>>>>>>> interval
> > > > > >>>>>>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>>>>>> the state store will only be atomically
> updated
> > > > > roughly
> > > > > >>>>>>>> every
> > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of
> data.
> > > Also
> > > > > IQ
> > > > > >>>>>>>> would
> > > > > >>>>>>>>>>>> then
> > > > > >>>>>>>>>>>>>>>> only
> > > > > >>>>>>>>>>>>>>>>>>>> see new data roughly every
> > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > > >>>>>>>>>>>>>>>>>>>> After a failure the state store needs to
> restore
> > > up to
> > > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> Is this correct?
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> What about atomically updating the state store
> > > from
> > > > > the
> > > > > >>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> > > > > checkpoint
> > > > > >>>>>>>> (thus,
> > > > > >>>>>>>>>>>>>>>> flushing
> > > > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data
> > > and/or
> > > > > >>>>>>>> number of
> > > > > >>>>>>>>>>>>>> commit
> > > > > >>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have the
> same
> > > delay
> > > > > >>>> for
> > > > > >>>>>>>>>>>> records
> > > > > >>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because both
> > > would
> > > > > >>>>>>>> appear
> > > > > >>>>>>>>>> when
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However,
> after a
> > > > > failure
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>> state
> > > > > >>>>>>>>>>>>>>>> store
> > > > > >>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > > >>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>> it might restore data that is already in the
> state
> > > > > store
> > > > > >>>>>>>>>> because
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable offset
> > > (i.e.
> > > > > the
> > > > > >>>>>>>> last
> > > > > >>>>>>>>>>>>>> committed
> > > > > >>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring
> data
> > > that
> > > > > is
> > > > > >>>>>>>> already
> > > > > >>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should not
> > > violated.
> > > > > >>>>>>>>>>>>>>>>>>>> This solution needs at least one new config to
> > > specify
> > > > > >>>>>>>> when a
> > > > > >>>>>>>>>>>>>>>> checkpoint
> > > > > >>>>>>>>>>>>>>>>>>>> should be written.
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail
> that
> > > does
> > > > > not
> > > > > >>>>>>>> change
> > > > > >>>>>>>>>>>>>>>> anything
> > > > > >>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit
> interval
> > > is 30
> > > > > >>>>>>>> seconds,
> > > > > >>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>> five
> > > > > >>>>>>>>>>>>>>>>>>>> seconds.
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> Best,
> > > > > >>>>>>>>>>>>>>>>>>>> Bruno
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch on
> our
> > > > > staging
> > > > > >>>>>>>>>>>>>> environment,
> > > > > >>>>>>>>>>>>>>>>>>>>> putting it through its paces in our
> non-trivial
> > > > > >>>>>>>> application.
> > > > > >>>>>>>>>> I'm
> > > > > >>>>>>>>>>>>>>>>>>>> already
> > > > > >>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate that
> we
> > > saw
> > > > > the
> > > > > >>>>>>>> last
> > > > > >>>>>>>>>>>> time
> > > > > >>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP, but
> this
> > > > > time,
> > > > > >> I
> > > > > >>>>>>>>>> think I
> > > > > >>>>>>>>>>>>>> know
> > > > > >>>>>>>>>>>>>>>>>>>> why.
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is
> > > called
> > > > > at
> > > > > >>>>>>>> the end
> > > > > >>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> Task
> > > > > >>>>>>>>>>>>>>>>>>>>> commit process, has the following behaviour:
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>           - Under ALOS: checkpoint the state
> > > stores.
> > > > > >> This
> > > > > >>>>>>>>>> includes
> > > > > >>>>>>>>>>>>>>>>>>>>>           flushing memtables in RocksDB.
> This is
> > > > > >>>> acceptable
> > > > > >>>>>>>>>>>> because the
> > > > > >>>>>>>>>>>>>>>>>>>> default
> > > > > >>>>>>>>>>>>>>>>>>>>>           commit.interval.ms is 5 seconds,
> so
> > > > > forcibly
> > > > > >>>>>>>> flushing
> > > > > >>>>>>>>>>>>>> memtables
> > > > > >>>>>>>>>>>>>>>>>>>> every 5
> > > > > >>>>>>>>>>>>>>>>>>>>>           seconds is acceptable for most
> > > > > applications.
> > > > > >>>>>>>>>>>>>>>>>>>>>           - Under EOS: checkpointing is not
> done,
> > > > > >> *unless*
> > > > > >>>>>>>> it's
> > > > > >>>>>>>>>>>> being
> > > > > >>>>>>>>>>>>>>>>>>>> forced, due
> > > > > >>>>>>>>>>>>>>>>>>>>>           to e.g. the Task closing or being
> > > revoked.
> > > > > >> This
> > > > > >>>>>>>> means
> > > > > >>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>> under
> > > > > >>>>>>>>>>>>>>>>>>>> normal
> > > > > >>>>>>>>>>>>>>>>>>>>>           processing conditions, the state
> stores
> > > > > will
> > > > > >> not
> > > > > >>>>>>>> be
> > > > > >>>>>>>>>>>>>>>> checkpointed,
> > > > > >>>>>>>>>>>>>>>>>>>> and will
> > > > > >>>>>>>>>>>>>>>>>>>>>           not have memtables flushed at all ,
> > > unless
> > > > > >>>> RocksDB
> > > > > >>>>>>>>>>>> decides to
> > > > > >>>>>>>>>>>>>>>>>>>> flush them on
> > > > > >>>>>>>>>>>>>>>>>>>>>           its own. Checkpointing stores and
> > > > > >> force-flushing
> > > > > >>>>>>>> their
> > > > > >>>>>>>>>>>>>> memtables
> > > > > >>>>>>>>>>>>>>>>>>>> is only
> > > > > >>>>>>>>>>>>>>>>>>>>>           done when a Task is being closed.
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint
> stores on
> > > at
> > > > > >>>> least
> > > > > >>>>>>>>>> *some*
> > > > > >>>>>>>>>>>>>>>> normal
> > > > > >>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB
> > > > > transaction
> > > > > >>>>>>>>>> buffers
> > > > > >>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets are
> > > synced to
> > > > > >>>>>>>> disk to
> > > > > >>>>>>>>>>>>>> prevent
> > > > > >>>>>>>>>>>>>>>>>>>>> restores from getting out of hand.
> Consequently,
> > > my
> > > > > >>>>>>>> current
> > > > > >>>>>>>>>>>>>>>>>>>> implementation
> > > > > >>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit,
> > > which
> > > > > is
> > > > > >>>>>>>> far too
> > > > > >>>>>>>>>>>>>>>>>>>> frequent.
> > > > > >>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records,
> > > which
> > > > > is
> > > > > >> a
> > > > > >>>>>>>>>> change
> > > > > >>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>> flush
> > > > > >>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing performance
> > > problems
> > > > > for
> > > > > >>>>>>>> some
> > > > > >>>>>>>>>>>>>>>>>>>> applications.
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm
> > > > > currently
> > > > > >>>>>>>> leaning
> > > > > >>>>>>>>>>>>>>>> towards
> > > > > >>>>>>>>>>>>>>>>>>>>> using the
> statestore.transaction.buffer.max.bytes
> > > > > >>>>>>>>>> configuration
> > > > > >>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to
> exceed it.
> > > > > This
> > > > > >>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>> complement the
> > > > > >>>>>>>>>>>>>>>>>>>>> existing "early Task commit" functionality
> that
> > > this
> > > > > >>>>>>>>>>>> configuration
> > > > > >>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>           - Currently, we use
> > > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > > >>>>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>> force an
> > > > > >>>>>>>>>>>>>>>>>>>>>           early Task commit if processing
> more
> > > > > records
> > > > > >>>> would
> > > > > >>>>>>>>>> cause
> > > > > >>>>>>>>>>>> our
> > > > > >>>>>>>>>>>>>>>> state
> > > > > >>>>>>>>>>>>>>>>>>>> store
> > > > > >>>>>>>>>>>>>>>>>>>>>           transactions to exceed the memory
> > > assigned
> > > > > to
> > > > > >>>>>>>> them.
> > > > > >>>>>>>>>>>>>>>>>>>>>           - New functionality: when a Task
> *does*
> > > > > >> commit,
> > > > > >>>>>>>> we will
> > > > > >>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>> checkpoint
> > > > > >>>>>>>>>>>>>>>>>>>>>           the stores (and hence flush the
> > > transaction
> > > > > >>>>>>>> buffers)
> > > > > >>>>>>>>>>>> unless
> > > > > >>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>> expect to
> > > > > >>>>>>>>>>>>>>>>>>>>>           cross the
> > > > > >>>> statestore.transaction.buffer.max.bytes
> > > > > >>>>>>>>>>>> threshold
> > > > > >>>>>>>>>>>>>>>> before
> > > > > >>>>>>>>>>>>>>>>>>>> the next
> > > > > >>>>>>>>>>>>>>>>>>>>>           commit
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > >>>>>>>>>>>>>>>>>>>>> Nick
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> > > > > >>>>>>>>>>>> nick.telford@gmail.com
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > >>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I meant in
> > > terms of
> > > > > >>>>>>>>>>>> understanding
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>> performance profile under various
> > > circumstances. The
> > > > > >>>>>>>> more
> > > > > >>>>>>>>>>>> complex
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would be for
> > > users
> > > > > to
> > > > > >>>>>>>>>>>> understand
> > > > > >>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>> performance they see. For example, spilling
> > > records
> > > > > to
> > > > > >>>>>>>> disk
> > > > > >>>>>>>>>>>> when
> > > > > >>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold
> would, I
> > > > > >> expect,
> > > > > >>>>>>>>>> reduce
> > > > > >>>>>>>>>>>>>> write
> > > > > >>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write
> throughput
> > > could
> > > > > >> be
> > > > > >>>>>>>>>>>>>> unexpected,
> > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>> potentially difficult to
> diagnose/understand for
> > > > > >> users.
> > > > > >>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early commit"
> > > concept is
> > > > > >>>>>>>>>> relatively
> > > > > >>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document, and
> > > > > >>>> conceptually
> > > > > >>>>>>>>>> fairly
> > > > > >>>>>>>>>>>>>>>>>>>> obvious to
> > > > > >>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to
> make it
> > > > > >> easier
> > > > > >>>>>>>> to
> > > > > >>>>>>>>>>>>>>>> understand
> > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > >>>>>>>>>>>>>>>>>>>>>> it happens though.
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > > > > >>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an
> indirect
> > > way
> > > > > of
> > > > > >>>>>>>>>> telling
> > > > > >>>>>>>>>>>>>>>>>>>> StateStores
> > > > > >>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
> > > READ_COMMITTED
> > > > > >>>>>>>>>>>> essentially
> > > > > >>>>>>>>>>>>>>>>>>>> requires
> > > > > >>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that two
> > > threads
> > > > > >>>>>>>> calling
> > > > > >>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes
> from
> > > the
> > > > > >> other
> > > > > >>>>>>>>>>>>>> transaction
> > > > > >>>>>>>>>>>>>>>>>>>> until
> > > > > >>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> > > READ_UNCOMMITTED, all
> > > > > >>>>>>>> bets are
> > > > > >>>>>>>>>>>> off,
> > > > > >>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe written
> > > records
> > > > > at
> > > > > >>>>>>>> any
> > > > > >>>>>>>>>>>> time,
> > > > > >>>>>>>>>>>>>>>>>>>> which is
> > > > > >>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That said,
> > > > > StateStores
> > > > > >>>>>>>> are
> > > > > >>>>>>>>>> free
> > > > > >>>>>>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>> implement
> > > > > >>>>>>>>>>>>>>>>>>>>>> these guarantees however they can, which is
> a
> > > bit
> > > > > more
> > > > > >>>>>>>>>> relaxed
> > > > > >>>>>>>>>>>>>> than
> > > > > >>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions". For
> > > example,
> > > > > >>>> with
> > > > > >>>>>>>>>>>> RocksDB
> > > > > >>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>> would
> > > > > >>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED ==
> WBWI-based
> > > > > >>>>>>>>>> "transactions",
> > > > > >>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the
> > > database.
> > > > > But
> > > > > >>>>>>>> with
> > > > > >>>>>>>>>>>> other
> > > > > >>>>>>>>>>>>>>>>>>>> storage
> > > > > >>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to *always*
> use
> > > > > >>>>>>>> transactions,
> > > > > >>>>>>>>>>>> even
> > > > > >>>>>>>>>>>>>>>>>>>> when
> > > > > >>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines
> > > that
> > > > > >> don't
> > > > > >>>>>>>>>> provide
> > > > > >>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation guarantees
> can
> > > be
> > > > > met
> > > > > >>>>>>>> using a
> > > > > >>>>>>>>>>>>>>>>>>>> different
> > > > > >>>>>>>>>>>>>>>>>>>>>> technique.
> > > > > >>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore
> > > interface
> > > > > as
> > > > > >>>>>>>>>> loosely
> > > > > >>>>>>>>>>>>>>>> coupled
> > > > > >>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > > > > >>>>>>>> implementers
> > > > > >>>>>>>>>> more
> > > > > >>>>>>>>>>>>>>>>>>>> freedom, and
> > > > > >>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge
> > > required.
> > > > > >>>>>>>>>>>>>>>>>>>>>> That said, I understand that
> "IsolationLevel"
> > > might
> > > > > >> not
> > > > > >>>>>>>> be
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>> right
> > > > > >>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it much
> more
> > > > > >>>>>>>> explicit if
> > > > > >>>>>>>>>>>>>>>>>>>> required, e.g.
> > > > > >>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> 7-8.
> > > > > >>>>>>>>>>>>>>>>>>>>>> I can make these changes either later today
> or
> > > > > >>>> tomorrow.
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> Small update:
> > > > > >>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a
> > > bunch of
> > > > > >>>>>>>> issues
> > > > > >>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>> needed
> > > > > >>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass,
> > > which is
> > > > > >>>>>>>>>> promising,
> > > > > >>>>>>>>>>>> but
> > > > > >>>>>>>>>>>>>>>> it
> > > > > >>>>>>>>>>>>>>>>>>>> will
> > > > > >>>>>>>>>>>>>>>>>>>>>> need to undergo some performance testing. I
> > > haven't
> > > > > >>>>>>>> (yet)
> > > > > >>>>>>>>>>>> worked
> > > > > >>>>>>>>>>>>>> on
> > > > > >>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I
> > > would
> > > > > >>>>>>>> expect
> > > > > >>>>>>>>>> that,
> > > > > >>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no
> difference. The
> > > > > >> branch
> > > > > >>>>>>>> is
> > > > > >>>>>>>>>>>>>> available
> > > > > >>>>>>>>>>>>>>>>>>>> at
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > https://github.com/nicktelford/kafka/tree/KIP-892-c
> > > > > >> if
> > > > > >>>>>>>>>> anyone
> > > > > >>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > >>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna
> <
> > > > > >>>>>>>>>>>> cadonna@apache.org>
> > > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually
> also
> > > my
> > > > > >>>>>>>> point. I
> > > > > >>>>>>>>>>>>>>>> understood
> > > > > >>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion path
> as
> > > a way
> > > > > >> to
> > > > > >>>>>>>> avoid
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> early
> > > > > >>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the
> intent.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually it is
> > > public
> > > > > >>>>>>>> API. My
> > > > > >>>>>>>>>>>>>> question
> > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal
> > > processing.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the size
> of the
> > > > > >>>>>>>> transaction
> > > > > >>>>>>>>>>>>>> buffer
> > > > > >>>>>>>>>>>>>>>>>>>> and
> > > > > >>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit affects
> the
> > > whole
> > > > > >>>>>>>>>> processing
> > > > > >>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a state
> store
> > > is
> > > > > not
> > > > > >>>>>>>>>>>> confined to
> > > > > >>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and
> > > changes the
> > > > > >>>>>>>> behavior
> > > > > >>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>> other
> > > > > >>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you that
> it
> > > is a
> > > > > >>>>>>>> decent
> > > > > >>>>>>>>>>>>>>>>>>>> compromise. I
> > > > > >>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and
> list
> > > the
> > > > > >>>>>>>> options to
> > > > > >>>>>>>>>>>>>>>> overcome
> > > > > >>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all
> options
> > > seem
> > > > > >>>> quite
> > > > > >>>>>>>>>> heavy
> > > > > >>>>>>>>>>>>>>>>>>>> compared
> > > > > >>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you
> > > mean
> > > > > with
> > > > > >>>>>>>> "less
> > > > > >>>>>>>>>>>>>>>>>>>> predictable
> > > > > >>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the
> alternatives
> > > > > really
> > > > > >>>>>>>>>>>>>> interesting.
> > > > > >>>>>>>>>>>>>>>>>>>> But I
> > > > > >>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good and
> we
> > > should
> > > > > >>>>>>>> continue
> > > > > >>>>>>>>>>>> with
> > > > > >>>>>>>>>>>>>>>> it!
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on
> > > June
> > > > > >> 20th:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind
> > > putting
> > > > > >>>>>>>> isolation
> > > > > >>>>>>>>>>>>>> level
> > > > > >>>>>>>>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should
> that
> > > also
> > > > > be
> > > > > >> a
> > > > > >>>>>>>> way
> > > > > >>>>>>>>>> to
> > > > > >>>>>>>>>>>>>> give
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to decide
> > > whether
> > > > > to
> > > > > >>>>>>>> turn on
> > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > > > > >>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned about
> > > how do
> > > > > >> you
> > > > > >>>>>>>> know
> > > > > >>>>>>>>>>>> if a
> > > > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written under
> EOS,
> > > if
> > > > > you
> > > > > >>>>>>>> do not
> > > > > >>>>>>>>>>>>>> have a
> > > > > >>>>>>>>>>>>>>>>>>>> way
> > > > > >>>>>>>>>>>>>>>>>>>>>>> to know if the state store is
> transactional or
> > > not.
> > > > > >> If
> > > > > >>>>>>>> a
> > > > > >>>>>>>>>> state
> > > > > >>>>>>>>>>>>>>>> store
> > > > > >>>>>>>>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be
> > > written
> > > > > >>>>>>>> during
> > > > > >>>>>>>>>>>> normal
> > > > > >>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state store
> is not
> > > > > >>>>>>>>>> transactional,
> > > > > >>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written under
> EOS.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> 7.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> My point was about not only considering the
> > > bytes
> > > > > in
> > > > > >>>>>>>> memory
> > > > > >>>>>>>>>> in
> > > > > >>>>>>>>>>>>>>>> config
> > > > > >>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also
> > > bytes
> > > > > that
> > > > > >>>>>>>> might
> > > > > >>>>>>>>>> be
> > > > > >>>>>>>>>>>>>>>>>>>> spilled
> > > > > >>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering
> whether you
> > > > > >> should
> > > > > >>>>>>>>>> remove
> > > > > >>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory
> bytes to
> > > be
> > > > > >> used
> > > > > >>>>>>>> to
> > > > > >>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My
> > > > > thinking
> > > > > >>>>>>>> was
> > > > > >>>>>>>>>> that
> > > > > >>>>>>>>>>>>>> even
> > > > > >>>>>>>>>>>>>>>>>>>> if a
> > > > > >>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to
> disk,
> > > > > >> limiting
> > > > > >>>>>>>> the
> > > > > >>>>>>>>>>>>>> overall
> > > > > >>>>>>>>>>>>>>>>>>>> bytes
> > > > > >>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it again
> and
> > > > > >>>>>>>> considering
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>> recent
> > > > > >>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much
> sense
> > > > > anymore.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> I like the name
> > > > > >>>>>>>> statestore.transaction.buffer.max.bytes that
> > > > > >>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>>>>>>>> proposed.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> 8.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> > > implementation
> > > > > >>>>>>>> details) of
> > > > > >>>>>>>>>>>> how
> > > > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
> > > > > >>>>>>>> transactions,
> > > > > >>>>>>>>>>>> state
> > > > > >>>>>>>>>>>>>>>> store
> > > > > >>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be
> great.
> > > > > Would
> > > > > >>>> be
> > > > > >>>>>>>>>> great
> > > > > >>>>>>>>>>>> if
> > > > > >>>>>>>>>>>>>>>> you
> > > > > >>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the
> > > behavior in
> > > > > >>>>>>>> case of
> > > > > >>>>>>>>>> a
> > > > > >>>>>>>>>>>>>>>>>>>> failure.
> > > > > >>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional state
> > > store
> > > > > >>>>>>>> recover
> > > > > >>>>>>>>>>>> after a
> > > > > >>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the
> transaction
> > > > > buffer,
> > > > > >>>>>>>> etc.
> > > > > >>>>>>>>>>>> (that
> > > > > >>>>>>>>>>>>>> is
> > > > > >>>>>>>>>>>>>>>>>>>> what
> > > > > >>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > > >>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> > > > > >>>>>>>> WriteBatchWithIndex
> > > > > >>>>>>>>>>>>>>>>>>>> transactions
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to
> exceed)
> > > > > >>>>>>>> configured
> > > > > >>>>>>>>>>>> memory
> > > > > >>>>>>>>>>>>>>>>>>>> needs to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns.
> Ultimately,
> > > any
> > > > > >>>>>>>> approach
> > > > > >>>>>>>>>>>> based
> > > > > >>>>>>>>>>>>>> on
> > > > > >>>>>>>>>>>>>>>>>>>>>>> cracking
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in
> ways
> > > it's
> > > > > not
> > > > > >>>>>>>> really
> > > > > >>>>>>>>>>>>>>>> designed
> > > > > >>>>>>>>>>>>>>>>>>>>>>> for is
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen performance
> or
> > > > > >>>>>>>> consistency
> > > > > >>>>>>>>>>>> issues.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing these
> > > early
> > > > > >>>>>>>> commits?
> > > > > >>>>>>>>>>>> While
> > > > > >>>>>>>>>>>>>> not
> > > > > >>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to
> ensure
> > > > > >>>>>>>> consistency
> > > > > >>>>>>>>>>>> whilst
> > > > > >>>>>>>>>>>>>>>>>>>>>>> maintaining
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
> > > > > >>>>>>>> complicated, and
> > > > > >>>>>>>>>>>> might
> > > > > >>>>>>>>>>>>>>>>>>>>>>> actually
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for users
> as a
> > > > > >>>>>>>> consequence.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this
> > > KIP is
> > > > > >>>>>>>> growing a
> > > > > >>>>>>>>>>>> bit
> > > > > >>>>>>>>>>>>>>>> out
> > > > > >>>>>>>>>>>>>>>>>>>> of
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas
> for
> > > > > future
> > > > > >>>>>>>>>>>>>>>> improvements, I
> > > > > >>>>>>>>>>>>>>>>>>>>>>> think
>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope down
> to a
> > > > > design
> > > > > >>>>>>>> that
> > > > > >>>>>>>>>>>>>> achieves
> > > > > >>>>>>>>>>>>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>>>>> most
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
> > > restorations
> > > > > >>>>>>>> during
> > > > > >>>>>>>>>> dirty
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that
> > > this KIP
> > > > > >>>>>>>> produces
> > > > > >>>>>>>>>>>> can
> > > > > >>>>>>>>>>>>>>>>>>>>>>> ultimately
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially if
> the
> > > bulk
> > > > > of
> > > > > >>>>>>>> it is
> > > > > >>>>>>>>>>>>>> internal
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week
> trying
> > > to
> > > > > >>>>>>>> re-work
> > > > > >>>>>>>>>> the
> > > > > >>>>>>>>>>>>>>>>>>>> original
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
> > > > > >>>>>>>> newTransaction()
> > > > > >>>>>>>>>>>>>> method,
> > > > > >>>>>>>>>>>>>>>>>>>> such
> > > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
> > > > > RocksDBStore.
> > > > > >>>>>>>> That
> > > > > >>>>>>>>>>>> way, if
> > > > > >>>>>>>>>>>>>>>> we
> > > > > >>>>>>>>>>>>>>>>>>>>>>> want to
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the future,
> > > like
> > > > > the
> > > > > >>>>>>>> SST
> > > > > >>>>>>>>>> file
> > > > > >>>>>>>>>>>>>>>>>>>> management
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with
> > > little/no
> > > > > >>>> API
> > > > > >>>>>>>>>>>> changes.
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>>
> > > > > >>>>>>>>>>>>
> > > > > >>>>>>>>>>>
> > > > > >>>>>>>>>>
> > > > > >>>>>>>>
> > > > > >>>>>>>
> > > > > >>>>>>
> > > > > >>>>>
> > > > > >>>>
> > > > > >>>
> > > > > >>
> > > > > >
> > > > >
> > >
> > >
>

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

Posted by Lucas Brutschy <lb...@confluent.io.INVALID>.
Hi all,

I think I liked your suggestion of allowing EOS with READ_UNCOMMITTED,
but keep wiping the state on error, and I'd vote for this solution
when introducing `default.state.isolation.level`. This way, we'd have
the most low-risk roll-out of this feature (no behavior change without
reconfiguration), with the possibility of switching to the most sane /
battle-tested default settings in 4.0. Essentially, we'd have a
feature flag but call it `default.state.isolation.level` and don't
have to deprecate it later.

So the possible configurations would then be this:

1. ALOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
reads from DB.
2. ALOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from
WriteBatch/DB. Flush on error (see note below).
3. EOS/READ_UNCOMMITTED (default) = processing uses direct-to-DB, IQ
reads from DB. Wipe state on error.
4. EOS/READ_COMMITTED = processing uses WriteBatch, IQ reads from WriteBatch/DB.

I believe the feature is important enough that we will see good
adoption even without changing the default. In 4.0, when we have seen
this being adopted and is battle-tested, we make READ_COMMITTED the
default for EOS, or even READ_COMITTED always the default, depending
on our experiences. And we could add a clever implementation of
READ_UNCOMITTED with WriteBatches later.

The only smell here is that `default.state.isolation.level` wouldn't
be purely an IQ setting, but it would also (slightly) change the
behavior of the processing, but that seems unavoidable as long as we
haven't solve READ_UNCOMITTED IQ with WriteBatches.

Minor: As for Bruno's point 4, I think if we are concerned about this
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).

Cheers,
Lucas

On Mon, Oct 16, 2023 at 12:19 PM Nick Telford <ni...@gmail.com> wrote:
>
> Hi Guozhang,
>
> The KIP as it stands introduces a new configuration,
> default.state.isolation.level, which is independent of processing.mode.
> It's intended that this new configuration be used to configure a global IQ
> isolation level in the short term, with a future KIP introducing the
> capability to change the isolation level on a per-query basis, falling back
> to the "default" defined by this config. That's why I called it "default",
> for future-proofing.
>
> However, it currently includes the caveat that READ_UNCOMMITTED is not
> available under EOS. I think this is the coupling you are alluding to?
>
> This isn't intended to be a restriction of the API, but is currently a
> technical limitation. However, after discussing with some users about
> use-cases that would require READ_UNCOMMITTED under EOS, I'm inclined to
> remove that clause and put in the necessary work to make that combination
> possible now.
>
> I currently see two possible approaches:
>
>    1. Disable TX StateStores internally when the IsolationLevel is
>    READ_UNCOMMITTED and the processing.mode is EOS. This is more difficult
>    than it sounds, as there are many assumptions being made throughout the
>    internals about the guarantees StateStores provide. It would definitely add
>    a lot of extra "if (read_uncommitted && eos)" branches, complicating
>    maintenance and testing.
>    2. Invest the time *now* to make READ_UNCOMMITTED of EOS StateStores
>    possible. I have some ideas on how this could be achieved, but they would
>    need testing and could introduce some additional issues. The benefit of
>    this approach is that it would make query-time IsolationLevels much simpler
>    to implement in the future.
>
> Unfortunately, both will require considerable work that will further delay
> this KIP, which was the reason I placed the restriction in the KIP in the
> first place.
>
> Regards,
> Nick
>
> On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <gu...@gmail.com>
> wrote:
>
> > Hello Nick,
> >
> > First of all, thanks a lot for the great effort you've put in driving
> > this KIP! I really like it coming through finally, as many people in
> > the community have raised this. At the same time I honestly feel a bit
> > ashamed for not putting enough of my time supporting it and pushing it
> > through the finish line (you raised this KIP almost a year ago).
> >
> > I briefly passed through the DISCUSS thread so far, not sure I've 100
> > percent digested all the bullet points. But with the goal of trying to
> > help take it through the finish line in mind, I'd want to throw
> > thoughts on top of my head only on the point #4 above which I felt may
> > be the main hurdle for the current KIP to drive to a consensus now.
> >
> > The general question I asked myself is, whether we want to couple "IQ
> > reading mode" with "processing mode". While technically I tend to
> > agree with you that, it's feels like a bug if some single user chose
> > "EOS" for processing mode while choosing "read uncommitted" for IQ
> > reading mode, at the same time, I'm thinking if it's possible that
> > there could be two different persons (or even two teams) that would be
> > using the stream API to build the app, and the IQ API to query the
> > running state of the app. I know this is less of a technical thing but
> > rather a more design stuff, but if it could be ever the case, I'm
> > wondering if the personale using the IQ API knows about the risks of
> > using read uncommitted but still chose so for the favor of
> > performance, no matter if the underlying stream processing mode
> > configured by another personale is EOS or not. In that regard, I'm
> > leaning towards a "leaving the door open, and close it later if we
> > found it's a bad idea" aspect with a configuration 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 can still enforce
> > coupling them.
> >
> > Again, just my 2c here. Thanks again for the great patience and
> > diligence on this KIP.
> >
> >
> > Guozhang
> >
> >
> >
> > On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <ni...@gmail.com>
> > wrote:
> > >
> > > Hi Bruno,
> > >
> > > 4.
> > > I'll hold off on making that change until we have a consensus as to what
> > > configuration to use to control all of this, as it'll be affected by the
> > > decision on EOS isolation levels.
> > >
> > > 5.
> > > Done. I've chosen "committedOffsets".
> > >
> > > Regards,
> > > Nick
> > >
> > > On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <ca...@apache.org> wrote:
> > >
> > > > Hi Nick,
> > > >
> > > > 1.
> > > > Yeah, you are probably right that it does not make too much sense.
> > > > Thanks for the clarification!
> > > >
> > > >
> > > > 4.
> > > > Yes, sorry for the back and forth, but I think for the sake of the KIP
> > > > it is better to let the ALOS behavior as it is for now due to the
> > > > possible issues you would run into. Maybe we can find a solution in the
> > > > future. Now the question returns to whether we really need
> > > > default.state.isolation.level. Maybe the config could be the feature
> > > > flag Sophie requested.
> > > >
> > > >
> > > > 5.
> > > > There 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 committedOffsetFor().
> > > >
> > > >
> > > > Best,
> > > > Bruno
> > > >
> > > >
> > > > On 10/13/23 3:21 PM, Nick Telford wrote:
> > > > > Hi Bruno,
> > > > >
> > > > > Thanks for getting back to me.
> > > > >
> > > > > 1.
> > > > > I think this should be possible. Are you thinking of the situation
> > where
> > > > a
> > > > > user may downgrade to a previous version of Kafka Streams? In that
> > case,
> > > > > sadly, the RocksDBStore would get wiped by the older version of Kafka
> > > > > Streams anyway, because that version wouldn't understand the extra
> > column
> > > > > family (that holds offsets), so the missing Position file would
> > > > > automatically get rebuilt when the store is rebuilt from the
> > changelog.
> > > > > Are there other situations than downgrade where a transactional store
> > > > could
> > > > > be replaced by a non-transactional one? I can't think of any.
> > > > >
> > > > > 2.
> > > > > Ahh yes, the Test Plan - my Kryptonite! This section definitely
> > needs to
> > > > be
> > > > > fleshed out. I'll work on that. How much detail do you need?
> > > > >
> > > > > 3.
> > > > > See my previous email discussing this.
> > > > >
> > > > > 4.
> > > > > Hmm, this is an interesting point. Are you suggesting that under ALOS
> > > > > READ_COMMITTED should not be supported?
> > > > >
> > > > > Regards,
> > > > > Nick
> > > > >
> > > > > On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <ca...@apache.org>
> > wrote:
> > > > >
> > > > >> Hi Nick,
> > > > >>
> > > > >> I think the KIP is converging!
> > > > >>
> > > > >>
> > > > >> 1.
> > > > >> I am wondering whether it makes sense to write the position file
> > during
> > > > >> close as we do for the checkpoint file, so that in case the state
> > store
> > > > >> is replaced with a non-transactional state store the
> > non-transactional
> > > > >> state store finds the position file. I think, this is not strictly
> > > > >> needed, but would be a nice behavior instead of just deleting the
> > > > >> position file.
> > > > >>
> > > > >>
> > > > >> 2.
> > > > >> The test plan does not mention integration tests. Do you not need to
> > > > >> extend existing ones and add new ones. Also for upgrading and
> > > > >> downgrading you might need integration and/or system tests.
> > > > >>
> > > > >>
> > > > >> 3.
> > > > >> I think Sophie made a point. Although, IQ reading from uncommitted
> > data
> > > > >> under EOS might be considered a bug by some people. Thus, your KIP
> > would
> > > > >> fix a bug rather than changing the intended behavior. However, I
> > also
> > > > >> see that a feature flag would help users that rely on this buggy
> > > > >> behavior (at least until AK 4.0).
> > > > >>
> > > > >>
> > > > >> 4.
> > > > >> This is related to the previous point. I assume that the difference
> > > > >> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is that in the
> > > > >> former you enable transactions on the state store and in the latter
> > you
> > > > >> disable them. If my assumption is correct, I think that is an issue.
> > > > >> Let's assume under ALOS Streams fails over a couple of times more or
> > > > >> less at the same step in processing after value 3 is added to an
> > > > >> aggregation but the offset of the corresponding input record was not
> > > > >> committed. Without transactions disabled, the aggregation value
> > would
> > > > >> increase by 3 for each failover. With transactions enabled, value 3
> > > > >> would only be added to the aggregation once when the offset of the
> > input
> > > > >> record is committed and the transaction finally completes. So the
> > > > >> content of the state store would change depending on the
> > configuration
> > > > >> for IQ. IMO, the content of the state store should be independent
> > from
> > > > >> IQ. Given this issue, I propose to not use transactions with ALOS at
> > > > >> all. I was a big proponent of using transactions with ALOS, but I
> > > > >> realized that transactions with ALOS is not as easy as enabling
> > > > >> transactions on state stores. Another aspect that is problematic is
> > that
> > > > >> the changelog topic which actually replicates the state store is not
> > > > >> transactional under ALOS. Thus, it might happen that the state
> > store and
> > > > >> the changelog differ in their content. All of this is maybe solvable
> > > > >> somehow, but for the sake of this KIP, I would leave it for the
> > future.
> > > > >>
> > > > >>
> > > > >> Best,
> > > > >> Bruno
> > > > >>
> > > > >>
> > > > >>
> > > > >> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > > > >>> Hey Nick! First of all thanks for taking up this awesome feature,
> > I'm
> > > > >> sure
> > > > >>> every single
> > > > >>> Kafka Streams user and dev would agree that it is sorely needed.
> > > > >>>
> > > > >>> I've just been catching up on the KIP and surrounding discussion,
> > so
> > > > >> please
> > > > >>> forgive me
> > > > >>> for any misunderstandings or misinterpretations of the current
> > plan and
> > > > >>> don't hesitate to
> > > > >>> correct me.
> > > > >>>
> > > > >>> Before I jump in, I just want to say that having seen this drag on
> > for
> > > > so
> > > > >>> long, my singular
> > > > >>> goal in responding is to help this KIP past a perceived impasse so
> > we
> > > > can
> > > > >>> finally move on
> > > > >>> to voting and implementing it. Long discussions are to be expected
> > for
> > > > >>> major features like
> > > > >>> this but it's completely on us as the Streams devs to make sure
> > there
> > > > is
> > > > >> an
> > > > >>> end in sight
> > > > >>> for any ongoing discussion.
> > > > >>>
> > > > >>> With that said, it's my understanding that the KIP as currently
> > > > proposed
> > > > >> is
> > > > >>> just not tenable
> > > > >>> for Kafka Streams, and would prevent some EOS users from upgrading
> > to
> > > > the
> > > > >>> version it
> > > > >>> first appears in. Given that we can't predict or guarantee whether
> > any
> > > > of
> > > > >>> the followup KIPs
> > > > >>> would be completed in the same release cycle as this one, we need
> > to
> > > > make
> > > > >>> sure that the
> > > > >>> feature is either compatible with all current users or else
> > > > >> feature-flagged
> > > > >>> so that they may
> > > > >>> opt in/out.
> > > > >>>
> > > > >>> Therefore, IIUC we need to have either (or both) of these as
> > > > >>> fully-implemented config options:
> > > > >>> 1. default.state.isolation.level
> > > > >>> 2. enable.transactional.state.stores
> > > > >>>
> > > > >>> This way EOS users for whom read_committed semantics are not
> > viable can
> > > > >>> still upgrade,
> > > > >>> and either use the isolation.level config to leverage the new txn
> > state
> > > > >>> stores without sacrificing
> > > > >>> their application semantics, or else simply keep the transactional
> > > > state
> > > > >>> stores disabled until we
> > > > >>> are able to fully implement the isolation level configuration at
> > either
> > > > >> an
> > > > >>> application or query level.
> > > > >>>
> > > > >>> Frankly you are the expert here and know much more about the
> > tradeoffs
> > > > in
> > > > >>> both semantics and
> > > > >>> effort level of implementing one of these configs vs the other. In
> > my
> > > > >>> opinion, either option would
> > > > >>> be fine and I would leave the decision of which one to include in
> > this
> > > > >> KIP
> > > > >>> completely up to you.
> > > > >>> I just don't see a way for the KIP to proceed without some
> > variation of
> > > > >> the
> > > > >>> above that would allow
> > > > >>> EOS users to opt-out of read_committed.
> > > > >>>
> > > > >>> (If it's all the same to you, I would recommend 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 that gets released.
> > > > >>> So personally, my choice would be to add the feature flag and
> > leave it
> > > > >> off
> > > > >>> by default. If all 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
> > > > >>>
> > > > >>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> > nick.telford@gmail.com>
> > > > >> wrote:
> > > > >>>
> > > > >>>> Hi Bruno,
> > > > >>>>
> > > > >>>> Agreed, I can live with that for now.
> > > > >>>>
> > > > >>>> In an effort to keep the scope of this KIP from expanding, I'm
> > leaning
> > > > >>>> towards just providing a configurable
> > default.state.isolation.level
> > > > and
> > > > >>>> removing IsolationLevel from the StateStoreContext. This would be
> > > > >>>> compatible with adding support for query-time IsolationLevels in
> > the
> > > > >>>> future, whilst providing a way for users to select an isolation
> > level
> > > > >> now.
> > > > >>>>
> > > > >>>> The big problem with this, however, is that if a user selects
> > > > >>>> processing.mode
> > > > >>>> = "exactly-once(-v2|-beta)", and default.state.isolation.level =
> > > > >>>> "READ_UNCOMMITTED", we need to guarantee that the data isn't
> > written
> > > > to
> > > > >>>> disk until commit() is called, but we also need to permit IQ
> > threads
> > > > to
> > > > >>>> read from the ongoing transaction.
> > > > >>>>
> > > > >>>> A simple solution would be to (temporarily) forbid this
> > combination of
> > > > >>>> configuration, and have default.state.isolation.level
> > automatically
> > > > >> switch
> > > > >>>> to READ_COMMITTED when processing.mode is anything other than
> > > > >>>> at-least-once. Do you think this would be acceptable?
> > > > >>>>
> > > > >>>> In a later KIP, we can add support for query-time isolation
> > levels and
> > > > >>>> solve this particular problem there, which would relax this
> > > > restriction.
> > > > >>>>
> > > > >>>> Regards,
> > > > >>>> Nick
> > > > >>>>
> > > > >>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <ca...@apache.org>
> > > > >> 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,
> > > > >>>>> Bruno
> > > > >>>>>
> > > > >>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > > > >>>>>> 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 <
> > nick.telford@gmail.com
> > > > >
> > > > >>>>> wrote:
> > > > >>>>>>
> > > > >>>>>>> Hi everyone,
> > > > >>>>>>>
> > > > >>>>>>> I agree that having IsolationLevel be determined at query-time
> > is
> > > > the
> > > > >>>>>>> ideal design, but there are a few sticking points:
> > > > >>>>>>>
> > > > >>>>>>> 1.
> > > > >>>>>>> There needs to be some way to communicate the IsolationLevel
> > down
> > > > to
> > > > >>>> the
> > > > >>>>>>> RocksDBStore itself, so that the query can respect it. Since
> > stores
> > > > >>>> are
> > > > >>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
> > MeteredStore,
> > > > >>>>> etc.),
> > > > >>>>>>> we need some way to deliver that information to the bottom
> > layer.
> > > > For
> > > > >>>>> IQv2,
> > > > >>>>>>> we can use the existing State#query() method, but IQv1 has no
> > way
> > > > to
> > > > >>>> do
> > > > >>>>>>> this.
> > > > >>>>>>>
> > > > >>>>>>> A simple approach, which would potentially open up other
> > options,
> > > > >>>> would
> > > > >>>>> be
> > > > >>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> > > > >>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > > > ReadOnlyKeyValueStore
> > > > >>>>> (and
> > > > >>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
> > > > >>>>>>>
> > > > >>>>>>> 2.
> > > > >>>>>>> As mentioned above, RocksDB WriteBatches are not thread-safe,
> > which
> > > > >>>>> causes
> > > > >>>>>>> a problem if we want to provide READ_UNCOMMITTED Iterators. I
> > also
> > > > >>>> had a
> > > > >>>>>>> look at RocksDB Transactions[1], but they solve a very
> > different
> > > > >>>>> problem,
> > > > >>>>>>> and have the same thread-safety issue.
> > > > >>>>>>>
> > > > >>>>>>> One possible approach that I mentioned is chaining
> > WriteBatches:
> > > > >> every
> > > > >>>>>>> time a new Interactive Query is received (i.e. readOnlyView,
> > see
> > > > >>>> above,
> > > > >>>>>>> is called) we "freeze" the existing WriteBatch, and start a
> > new one
> > > > >>>> for
> > > > >>>>> new
> > > > >>>>>>> writes. The Interactive Query queries the "chain" of previous
> > > > >>>>> WriteBatches
> > > > >>>>>>> + the underlying database; while the StreamThread starts
> > writing to
> > > > >>>> the
> > > > >>>>>>> *new* WriteBatch. On-commit, the StreamThread would write *all*
> > > > >>>>>>> WriteBatches in the chain to the database (that have not yet
> > been
> > > > >>>>> written).
> > > > >>>>>>>
> > > > >>>>>>> WriteBatches would be closed/freed only when they have been
> > both
> > > > >>>>>>> committed, and all open Interactive Queries on them have been
> > > > closed.
> > > > >>>>> This
> > > > >>>>>>> would require some reference counting.
> > > > >>>>>>>
> > > > >>>>>>> Obviously a drawback of this approach is the potential for
> > > > increased
> > > > >>>>>>> memory usage: if an Interactive Query is long-lived, for
> > example by
> > > > >>>>> doing a
> > > > >>>>>>> full scan over a large database, or even just pausing in the
> > middle
> > > > >> of
> > > > >>>>> an
> > > > >>>>>>> iteration, then the existing chain of WriteBatches could be
> > kept
> > > > >>>> around
> > > > >>>>> for
> > > > >>>>>>> a long time, potentially forever.
> > > > >>>>>>>
> > > > >>>>>>> --
> > > > >>>>>>>
> > > > >>>>>>> A.
> > > > >>>>>>> Going off on a tangent, it looks like in addition to supporting
> > > > >>>>>>> READ_COMMITTED queries, we could go further and support
> > > > >>>> REPEATABLE_READ
> > > > >>>>>>> queries (i.e. where subsequent reads to the same key in the
> > same
> > > > >>>>>>> Interactive Query are guaranteed to yield the same value) by
> > making
> > > > >>>> use
> > > > >>>>> of
> > > > >>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so the
> > > > >> performance
> > > > >>>>>>> impact is likely to be negligible, but they do require that the
> > > > >>>>> Interactive
> > > > >>>>>>> Query session can be explicitly closed.
> > > > >>>>>>>
> > > > >>>>>>> This could be achieved if we made the above readOnlyView
> > interface
> > > > >>>> look
> > > > >>>>>>> more like:
> > > > >>>>>>>
> > > > >>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> > > > >>>> ReadOnlyKeyValueStore<K,
> > > > >>>>>>> V>, AutoCloseable {}
> > > > >>>>>>>
> > > > >>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > > > >>>>>>>        ...
> > > > >>>>>>>        ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel
> > > > >>>>> isolationLevel);
> > > > >>>>>>> }
> > > > >>>>>>>
> > > > >>>>>>> But this would be a breaking change, as existing IQv1 queries
> > are
> > > > >>>>>>> guaranteed to never call store.close(), and therefore these
> > would
> > > > >> leak
> > > > >>>>>>> memory under REPEATABLE_READ.
> > > > >>>>>>>
> > > > >>>>>>> B.
> > > > >>>>>>> One thing that's notable: MyRocks states that they support
> > > > >>>>> READ_COMMITTED
> > > > >>>>>>> and REPEATABLE_READ, but they make no mention of
> > > > >>>> READ_UNCOMMITTED[3][4].
> > > > >>>>>>> This could be because doing so is technically
> > difficult/impossible
> > > > >>>> using
> > > > >>>>>>> the primitives available in RocksDB.
> > > > >>>>>>>
> > > > >>>>>>> --
> > > > >>>>>>>
> > > > >>>>>>> Lucas, to address your points:
> > > > >>>>>>>
> > > > >>>>>>> U1.
> > > > >>>>>>> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
> > > > >>>>>>> implementations of StateStore that do not support atomic
> > writes.
> > > > >>>>> Obviously
> > > > >>>>>>> in those cases, the guarantees Kafka Streams provides/expects
> > would
> > > > >> be
> > > > >>>>>>> relaxed. Do you think we should require all implementations to
> > > > >> support
> > > > >>>>>>> atomic writes?
> > > > >>>>>>>
> > > > >>>>>>> U2.
> > > > >>>>>>> Stores can support multiple IsolationLevels. As we've discussed
> > > > >> above,
> > > > >>>>> the
> > > > >>>>>>> ideal scenario would be to specify the IsolationLevel at
> > > > query-time.
> > > > >>>>>>> Failing that, I think the second-best approach is to define the
> > > > >>>>>>> IsolationLevel for *all* queries based on the processing.mode,
> > > > which
> > > > >>>> is
> > > > >>>>>>> what the default StateStoreContext#isolationLevel() achieves.
> > Would
> > > > >>>> you
> > > > >>>>>>> prefer an alternative?
> > > > >>>>>>>
> > > > >>>>>>> While the existing implementation is equivalent to
> > > > READ_UNCOMMITTED,
> > > > >>>>> this
> > > > >>>>>>> can yield unexpected results/errors under EOS, if a
> > transaction is
> > > > >>>>> rolled
> > > > >>>>>>> back. While this would be a change in behaviour for users, it
> > would
> > > > >>>> look
> > > > >>>>>>> more like a bug fix than a breaking change. That said, we
> > *could*
> > > > >> make
> > > > >>>>> it
> > > > >>>>>>> configurable, and default to the existing behaviour
> > > > >> (READ_UNCOMMITTED)
> > > > >>>>>>> instead of inferring it from the processing.mode?
> > > > >>>>>>>
> > > > >>>>>>> N1, N2.
> > > > >>>>>>> These were only primitives to avoid boxing costs, but since
> > this is
> > > > >>>> not
> > > > >>>>> a
> > > > >>>>>>> performance sensitive area, it should be fine to change if
> > that's
> > > > >>>>> desirable.
> > > > >>>>>>>
> > > > >>>>>>> N3.
> > > > >>>>>>> It's because the store "manages its own offsets", which
> > includes
> > > > both
> > > > >>>>>>> committing the offset, *and providing it* via
> > getCommittedOffset().
> > > > >>>>>>> Personally, I think "managesOffsets" conveys this best, but I
> > don't
> > > > >>>> mind
> > > > >>>>>>> changing it if the nomenclature is unclear.
> > > > >>>>>>>
> > > > >>>>>>> Sorry for the massive emails/essays!
> > > > >>>>>>> --
> > > > >>>>>>> Nick
> > > > >>>>>>>
> > > > >>>>>>> 1: https://github.com/facebook/rocksdb/wiki/Transactions
> > > > >>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > > > >>>>>>> 3:
> > > > https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > > >>>>>>> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > > >>>>>>>
> > > > >>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > > > >>>>>>> <lb...@confluent.io.invalid> wrote:
> > > > >>>>>>>
> > > > >>>>>>>> Hi Nick,
> > > > >>>>>>>>
> > > > >>>>>>>> since I last read it in April, the KIP has become much
> > cleaner and
> > > > >>>>>>>> easier to read. Great work!
> > > > >>>>>>>>
> > > > >>>>>>>> It feels to me the last big open point is whether we can
> > implement
> > > > >>>>>>>> isolation level as a query parameter. I understand that there
> > are
> > > > >>>>>>>> implementation concerns, but as Colt says, it would be a great
> > > > >>>>>>>> addition, and would also simplify the migration path for this
> > > > >> change.
> > > > >>>>>>>> Is the implementation problem you mentioned caused by the
> > > > WriteBatch
> > > > >>>>>>>> not having a notion of a snapshot, as the underlying DB
> > iterator
> > > > >>>> does?
> > > > >>>>>>>> In that case, I am not sure a chain of WriteBatches as you
> > propose
> > > > >>>>>>>> would fully solve the problem, but maybe I didn't dig enough
> > into
> > > > >> the
> > > > >>>>>>>> details to fully understand it.
> > > > >>>>>>>>
> > > > >>>>>>>> If it's not possible to implement it now, would it be an
> > option to
> > > > >>>>>>>> make sure in this KIP that we do not fully close the door on
> > > > >>>> per-query
> > > > >>>>>>>> isolation levels in the interface, as it may be possible to
> > > > >> implement
> > > > >>>>>>>> the missing primitives in RocksDB or Speedb in the future.
> > > > >>>>>>>>
> > > > >>>>>>>> Understanding:
> > > > >>>>>>>>
> > > > >>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to be
> > persisted
> > > > >>>>>>>> atomically with the records?
> > > > >>>>>>>> * U2) Don't understand the default implementation of
> > > > >>>> `isolationLevel`.
> > > > >>>>>>>> The isolation level should be a property of the underlying
> > store,
> > > > >> and
> > > > >>>>>>>> not be defined by the default config? Existing stores probably
> > > > don't
> > > > >>>>>>>> guarantee READ_COMMITTED, so the default should be to return
> > > > >>>>>>>> READ_UNCOMMITTED.
> > > > >>>>>>>>
> > > > >>>>>>>> Nits:
> > > > >>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong` return
> > type,
> > > > >> to
> > > > >>>>>>>> avoid the `null`?
> > > > >>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
> > `OptionalLong`
> > > > >>>>>>>> return type, to avoid the `-1`?
> > > > >>>>>>>> * N3) I don't understand why `managesOffsets` uses the
> > 'manage'
> > > > >> verb,
> > > > >>>>>>>> whereas all other methods use the "commits" verb. I'd suggest
> > > > >>>>>>>> `commitsOffsets`.
> > > > >>>>>>>>
> > > > >>>>>>>> Either way, it feels this KIP is very close to the finish
> > line,
> > > > I'm
> > > > >>>>>>>> looking forward to seeing this in production!
> > > > >>>>>>>>
> > > > >>>>>>>> Cheers,
> > > > >>>>>>>> Lucas
> > > > >>>>>>>>
> > > > >>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> > colt@littlehorse.io
> > > > >
> > > > >>>>> wrote:
> > > > >>>>>>>>>
> > > > >>>>>>>>>> Making IsolationLevel a query-time constraint, rather than
> > > > linking
> > > > >>>> it
> > > > >>>>>>>> to
> > > > >>>>>>>>> the processing.guarantee.
> > > > >>>>>>>>>
> > > > >>>>>>>>> As I understand it, would this allow even a user of EOS to
> > > > control
> > > > >>>>>>>> whether
> > > > >>>>>>>>> reading committed or uncommitted records? If so, I am highly
> > in
> > > > >>>> favor
> > > > >>>>> of
> > > > >>>>>>>>> this.
> > > > >>>>>>>>>
> > > > >>>>>>>>> I know that I was one of the early people to point out the
> > > > current
> > > > >>>>>>>>> shortcoming that IQ reads uncommitted records, but just this
> > > > >>>> morning I
> > > > >>>>>>>>> realized a pattern we use which means that (for certain
> > queries)
> > > > >> our
> > > > >>>>>>>> system
> > > > >>>>>>>>> needs to be able to read uncommitted records, which is the
> > > > current
> > > > >>>>>>>> behavior
> > > > >>>>>>>>> of Kafka Streams in EOS.***
> > > > >>>>>>>>>
> > > > >>>>>>>>> If IsolationLevel being a query-time decision allows for
> > this,
> > > > then
> > > > >>>>> that
> > > > >>>>>>>>> would be amazing. I would also vote that the default behavior
> > > > >> should
> > > > >>>>> be
> > > > >>>>>>>> for
> > > > >>>>>>>>> reading uncommitted records, because it is totally possible
> > for a
> > > > >>>>> valid
> > > > >>>>>>>>> application to depend on that behavior, and breaking it in a
> > > > minor
> > > > >>>>>>>> release
> > > > >>>>>>>>> might be a bit strong.
> > > > >>>>>>>>>
> > > > >>>>>>>>> *** (Note, for the curious reader....) Our use-case/query
> > pattern
> > > > >>>> is a
> > > > >>>>>>>> bit
> > > > >>>>>>>>> complex, but reading "uncommitted" records is actually safe
> > in
> > > > our
> > > > >>>>> case
> > > > >>>>>>>>> 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 soon as the processor processes the
> > result.
> > > > If
> > > > >>>>> they
> > > > >>>>>>>>> can't read uncommitted records, they may receive a "201 -
> > > > Created"
> > > > >>>>>>>>> response, immediately followed by a "404 - 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 <
> > > > >>>> nick.telford@gmail.com>
> > > > >>>>>>>> wrote:
> > > > >>>>>>>>>
> > > > >>>>>>>>>> Addendum:
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> I think we would also face the same problem with the
> > approach
> > > > John
> > > > >>>>>>>> outlined
> > > > >>>>>>>>>> earlier (using the record cache as a transaction buffer and
> > > > >>>> flushing
> > > > >>>>>>>> it
> > > > >>>>>>>>>> straight to SST files). This is because the record cache
> > (the
> > > > >>>>>>>> ThreadCache
> > > > >>>>>>>>>> class) is not thread-safe, so every commit would invalidate
> > open
> > > > >> IQ
> > > > >>>>>>>>>> Iterators in the same way that RocksDB WriteBatches do.
> > > > >>>>>>>>>> --
> > > > >>>>>>>>>> Nick
> > > > >>>>>>>>>>
> > > > >>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > > > >>>> nick.telford@gmail.com>
> > > > >>>>>>>>>> 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.
> > > > >>>>>>>>>>> 2. Making IsolationLevel a query-time constraint, rather
> > than
> > > > >>>>>>>> linking it
> > > > >>>>>>>>>>> to the processing.guarantee.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> There's a wrinkle that makes this a challenge: Interactive
> > > > >> Queries
> > > > >>>>>>>> that
> > > > >>>>>>>>>>> open an Iterator, when using transactions and
> > READ_UNCOMMITTED.
> > > > >>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries need
> > to be
> > > > >>>> able
> > > > >>>>>>>> to
> > > > >>>>>>>>>>> read records from the currently uncommitted transaction
> > buffer
> > > > >>>>>>>>>>> (WriteBatch). This includes for Iterators, which should
> > iterate
> > > > >>>>>>>> both the
> > > > >>>>>>>>>>> transaction buffer and underlying database (using
> > > > >>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> The issue is that when the StreamThread commits, it writes
> > the
> > > > >>>>>>>> current
> > > > >>>>>>>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*.
> > > > Clearing
> > > > >>>> the
> > > > >>>>>>>>>>> WriteBatch while an Interactive Query holds an open
> > Iterator on
> > > > >> it
> > > > >>>>>>>> will
> > > > >>>>>>>>>>> invalidate the Iterator. Worse, it turns out that Iterators
> > > > over
> > > > >> a
> > > > >>>>>>>>>>> WriteBatch become invalidated not just when the WriteBatch
> > is
> > > > >>>>>>>> cleared,
> > > > >>>>>>>>>> but
> > > > >>>>>>>>>>> also when the Iterators' current key receives a new write.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Now that I'm writing this, I remember that this is the
> > major
> > > > >>>> reason
> > > > >>>>>>>> that
> > > > >>>>>>>>>> I
> > > > >>>>>>>>>>> switched the original design from having a query-time
> > > > >>>>>>>> IsolationLevel to
> > > > >>>>>>>>>>> having the IsolationLevel linked to the transactionality
> > of the
> > > > >>>>>>>> stores
> > > > >>>>>>>>>>> themselves.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> It *might* be possible to resolve this, by having a
> > "chain" of
> > > > >>>>>>>>>>> WriteBatches, with the StreamThread switching to a new
> > > > WriteBatch
> > > > >>>>>>>>>> whenever
> > > > >>>>>>>>>>> a new Interactive Query attempts to read from the
> > database, but
> > > > >>>> that
> > > > >>>>>>>>>> could
> > > > >>>>>>>>>>> cause some performance problems/memory pressure when
> > subjected
> > > > to
> > > > >>>> a
> > > > >>>>>>>> high
> > > > >>>>>>>>>>> Interactive Query load. It would also reduce the
> > efficiency of
> > > > >>>>>>>>>> WriteBatches
> > > > >>>>>>>>>>> on-commit, as we'd have to write N WriteBatches, where N
> > is the
> > > > >>>>>>>> number of
> > > > >>>>>>>>>>> Interactive Queries since the last commit.
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> I realise this is getting into the weeds of the
> > implementation,
> > > > >>>> and
> > > > >>>>>>>> you'd
> > > > >>>>>>>>>>> rather we focus on the API for now, but I think it's
> > important
> > > > to
> > > > >>>>>>>>>> consider
> > > > >>>>>>>>>>> how to implement the desired API, in case we come up with
> > an
> > > > API
> > > > >>>>>>>> that
> > > > >>>>>>>>>>> cannot be implemented efficiently, or even at all!
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> Thoughts?
> > > > >>>>>>>>>>> --
> > > > >>>>>>>>>>> Nick
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> > > > cadonna@apache.org
> > > > >>>
> > > > >>>>>>>> wrote:
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>>> Hi Nick,
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> 6.
> > > > >>>>>>>>>>>> Of course, you are right! My bad!
> > > > >>>>>>>>>>>> Wiping out the state in the downgrading case is fine.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> 3a.
> > > > >>>>>>>>>>>> Focus on the public facing changes for the KIP. We will
> > manage
> > > > >> 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.
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>> Bruno
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > > > >>>>>>>>>>>>> Hi Bruno,
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Thanks for getting back to me!
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>> The fact that implementations can always track estimated
> > > > memory
> > > > >>>>>>>> usage
> > > > >>>>>>>>>> in
> > > > >>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as an
> > option,
> > > > and
> > > > >>>>>>>> I'll
> > > > >>>>>>>>>>>> clarify
> > > > >>>>>>>>>>>>> the JavaDoc that 0 is not just for non-transactional
> > stores,
> > > > >>>>>>>> which is
> > > > >>>>>>>>>>>>> currently misleading.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> 6.
> > > > >>>>>>>>>>>>> The problem with catching the exception in the downgrade
> > > > >> process
> > > > >>>>>>>> is
> > > > >>>>>>>>>> that
> > > > >>>>>>>>>>>>> would require new code in the Kafka version being
> > downgraded
> > > > >> to.
> > > > >>>>>>>> Since
> > > > >>>>>>>>>>>>> users could conceivably downgrade to almost *any* older
> > > > version
> > > > >>>>>>>> of
> > > > >>>>>>>>>> Kafka
> > > > >>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
> > > > >>>>>>>>>>>>> The only way I can think of doing it would be to provide
> > a
> > > > >>>>>>>> dedicated
> > > > >>>>>>>>>>>>> downgrade tool, that goes through every local store and
> > > > removes
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>> offsets column families. But that seems like an
> > unnecessary
> > > > >>>>>>>> amount of
> > > > >>>>>>>>>>>> extra
> > > > >>>>>>>>>>>>> code to maintain just to handle a somewhat niche
> > situation,
> > > > >> when
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>> alternative (automatically wipe and restore stores)
> > should be
> > > > >>>>>>>>>>>> acceptable.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've requested.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> 3a.
> > > > >>>>>>>>>>>>> I agree that IsolationLevel makes more sense at
> > query-time,
> > > > and
> > > > >>>> I
> > > > >>>>>>>>>>>> actually
> > > > >>>>>>>>>>>>> initially attempted to place the IsolationLevel at
> > > > query-time,
> > > > >>>>>>>> but I
> > > > >>>>>>>>>> ran
> > > > >>>>>>>>>>>>> into some problems:
> > > > >>>>>>>>>>>>> - The key issue is that, under ALOS we're not staging
> > writes
> > > > in
> > > > >>>>>>>>>>>>> transactions, so can't perform writes at the
> > READ_COMMITTED
> > > > >>>>>>>> isolation
> > > > >>>>>>>>>>>>> level. However, this may be addressed if we decide to
> > > > *always*
> > > > >>>>>>>> use
> > > > >>>>>>>>>>>>> transactions as discussed under 3b.
> > > > >>>>>>>>>>>>> - IQv1 and IQv2 have quite different implementations. I
> > > > >> remember
> > > > >>>>>>>>>> having
> > > > >>>>>>>>>>>>> some difficulty understanding the IQv1 internals, which
> > made
> > > > it
> > > > >>>>>>>>>>>> difficult
> > > > >>>>>>>>>>>>> to determine what needed to be changed. However, I
> > *think*
> > > > this
> > > > >>>>>>>> can be
> > > > >>>>>>>>>>>>> addressed for both implementations by wrapping the
> > > > RocksDBStore
> > > > >>>>>>>> in an
> > > > >>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides read
> > methods
> > > > >>>>>>>> (get,
> > > > >>>>>>>>>>>> etc.)
> > > > >>>>>>>>>>>>> to either read directly from the database or from the
> > ongoing
> > > > >>>>>>>>>>>> transaction.
> > > > >>>>>>>>>>>>> But IQv1 might still be difficult.
> > > > >>>>>>>>>>>>> - If IsolationLevel becomes a query constraint, then all
> > > > other
> > > > >>>>>>>>>>>> StateStores
> > > > >>>>>>>>>>>>> will need to respect it, including the in-memory stores.
> > This
> > > > >>>>>>>> would
> > > > >>>>>>>>>>>> require
> > > > >>>>>>>>>>>>> us to adapt in-memory stores to stage their writes so
> > they
> > > > can
> > > > >>>> be
> > > > >>>>>>>>>>>> isolated
> > > > >>>>>>>>>>>>> from READ_COMMITTTED queries. It would also become an
> > > > important
> > > > >>>>>>>>>>>>> consideration for third-party stores on upgrade, as
> > without
> > > > >>>>>>>> changes,
> > > > >>>>>>>>>>>> they
> > > > >>>>>>>>>>>>> would not support READ_COMMITTED queries correctly.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Ultimately, I may need some help making the necessary
> > change
> > > > to
> > > > >>>>>>>> IQv1
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>> support this, but I don't think it's fundamentally
> > > > impossible,
> > > > >>>>>>>> if we
> > > > >>>>>>>>>>>> want
> > > > >>>>>>>>>>>>> to pursue this route.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> 3b.
> > > > >>>>>>>>>>>>> The main reason I chose to keep ALOS un-transactional
> > was to
> > > > >>>>>>>> minimize
> > > > >>>>>>>>>>>>> behavioural change for most users (I believe most Streams
> > > > users
> > > > >>>>>>>> use
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>> default configuration, which is ALOS). That said, it's
> > clear
> > > > >>>>>>>> that if
> > > > >>>>>>>>>>>> ALOS
> > > > >>>>>>>>>>>>> also used transactional stores, the only change in
> > behaviour
> > > > >>>>>>>> would be
> > > > >>>>>>>>>>>> that
> > > > >>>>>>>>>>>>> it would become *more correct*, which could be
> > considered a
> > > > >> "bug
> > > > >>>>>>>> fix"
> > > > >>>>>>>>>> by
> > > > >>>>>>>>>>>>> users, rather than a change they need to handle.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> I believe that performance using transactions (aka.
> > RocksDB
> > > > >>>>>>>>>>>> WriteBatches)
> > > > >>>>>>>>>>>>> should actually be *better* than the un-batched
> > write-path
> > > > that
> > > > >>>>>>>> is
> > > > >>>>>>>>>>>>> currently used[1]. The only "performance" consideration
> > will
> > > > be
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>> increased memory usage that transactions require. Given
> > the
> > > > >>>>>>>>>> mitigations
> > > > >>>>>>>>>>>> for
> > > > >>>>>>>>>>>>> this memory that we have in place, I would expect that
> > this
> > > > is
> > > > >>>>>>>> not a
> > > > >>>>>>>>>>>>> problem for most users.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> If we're happy to do so, we can make ALOS also use
> > > > >> transactions.
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> Link 1:
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>
> > > > >>
> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> > > > >>>> cadonna@apache.org
> > > > >>>>>>>>>
> > > > >>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Hi Nick,
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Thanks for the updates and sorry for the delay on my
> > side!
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> 1.
> > > > >>>>>>>>>>>>>> Making the default implementation for flush() a no-op
> > sounds
> > > > >>>>>>>> good to
> > > > >>>>>>>>>>>> me.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>> I think what was bugging me here is that a third-party
> > state
> > > > >>>>>>>> store
> > > > >>>>>>>>>>>> needs
> > > > >>>>>>>>>>>>>> to implement the state store interface. That means they
> > need
> > > > >> to
> > > > >>>>>>>>>>>>>> implement a wrapper around the actual state store as we
> > do
> > > > for
> > > > >>>>>>>>>> RocksDB
> > > > >>>>>>>>>>>>>> with RocksDBStore. So, a third-party state store can
> > always
> > > > >>>>>>>> estimate
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>> uncommitted bytes, if it wants, because the wrapper can
> > > > record
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>> added
> > > > >>>>>>>>>>>>>> bytes.
> > > > >>>>>>>>>>>>>> One case I can think of where returning -1 makes sense
> > is
> > > > when
> > > > >>>>>>>>>> Streams
> > > > >>>>>>>>>>>>>> does not need to estimate the size of the write batch
> > and
> > > > >>>>>>>> trigger
> > > > >>>>>>>>>>>>>> extraordinary commits, because the third-party state
> > store
> > > > >>>>>>>> takes care
> > > > >>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>> memory. But in that case the method could also just
> > return
> > > > 0.
> > > > >>>>>>>> Even
> > > > >>>>>>>>>> that
> > > > >>>>>>>>>>>>>> case would be better solved with a method that returns
> > > > whether
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>> store manages itself the memory used for uncommitted
> > bytes
> > > > or
> > > > >>>>>>>> not.
> > > > >>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return value,
> > I was
> > > > >>>>>>>> just
> > > > >>>>>>>>>>>>>> wondering when and if it will be used.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Regarding returning 0 for transactional state stores
> > when
> > > > the
> > > > >>>>>>>> batch
> > > > >>>>>>>>>> is
> > > > >>>>>>>>>>>>>> empty, I was just wondering because you explicitly
> > stated
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> "or {@code 0} if this StateStore does not support
> > > > >>>> transactions."
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> So it seemed to me returning 0 could only happen for
> > > > >>>>>>>>>> non-transactional
> > > > >>>>>>>>>>>>>> state stores.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> a) What do you think if we move the isolation level to
> > IQ
> > > > (v1
> > > > >>>>>>>> and
> > > > >>>>>>>>>> v2)?
> > > > >>>>>>>>>>>>>> In the end this is the only component that really needs
> > to
> > > > >>>>>>>> specify
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>> isolation level. It is similar to the Kafka consumer
> > that
> > > > can
> > > > >>>>>>>> choose
> > > > >>>>>>>>>>>>>> with what isolation level to read the input topic.
> > > > >>>>>>>>>>>>>> For IQv1 the isolation level should go into
> > > > >>>>>>>> StoreQueryParameters. For
> > > > >>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> b) Point a) raises the question what should happen
> > during
> > > > >>>>>>>>>> at-least-once
> > > > >>>>>>>>>>>>>> processing when the state store does not use
> > transactions?
> > > > >> John
> > > > >>>>>>>> in
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>> past proposed to also use transactions on state stores
> > for
> > > > >>>>>>>>>>>>>> at-least-once. I like that idea, because it avoids
> > > > aggregating
> > > > >>>>>>>> the
> > > > >>>>>>>>>> same
> > > > >>>>>>>>>>>>>> records over and over again in the case of a failure. We
> > > > had a
> > > > >>>>>>>> case
> > > > >>>>>>>>>> in
> > > > >>>>>>>>>>>>>> the past where a Streams applications in at-least-once
> > mode
> > > > >> was
> > > > >>>>>>>>>> failing
> > > > >>>>>>>>>>>>>> continuously for some reasons I do not remember before
> > > > >>>>>>>> committing the
> > > > >>>>>>>>>>>>>> offsets. After each failover, the app aggregated again
> > and
> > > > >>>>>>>> again the
> > > > >>>>>>>>>>>>>> same records. Of course the aggregate increased to very
> > > > wrong
> > > > >>>>>>>> values
> > > > >>>>>>>>>>>>>> just because of the failover. With transactions on the
> > state
> > > > >>>>>>>> stores
> > > > >>>>>>>>>> we
> > > > >>>>>>>>>>>>>> could have avoided this. The app would have output the
> > same
> > > > >>>>>>>> aggregate
> > > > >>>>>>>>>>>>>> multiple times (i.e., after each failover) but at least
> > the
> > > > >>>>>>>> value of
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>> aggregate would not depend on the number of failovers.
> > > > >>>>>>>> Outputting the
> > > > >>>>>>>>>>>>>> same aggregate multiple times would be incorrect under
> > > > >>>>>>>> exactly-once
> > > > >>>>>>>>>> but
> > > > >>>>>>>>>>>>>> it is OK for at-least-once.
> > > > >>>>>>>>>>>>>> If it makes sense to add a config to turn on and off
> > > > >>>>>>>> transactions on
> > > > >>>>>>>>>>>>>> state stores under at-least-once or just use
> > transactions in
> > > > >>>>>>>> any case
> > > > >>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>> a question we should also discuss in this KIP. It
> > depends a
> > > > >> bit
> > > > >>>>>>>> on
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would add a
> > > > config.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> 4.
> > > > >>>>>>>>>>>>>> Your points are all valid. I tend to say to keep the
> > metrics
> > > > >>>>>>>> around
> > > > >>>>>>>>>>>>>> flush() until we remove flush() completely from the
> > > > interface.
> > > > >>>>>>>> Calls
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>> flush() might still exist since existing processors
> > might
> > > > >> still
> > > > >>>>>>>> call
> > > > >>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For sure, we
> > need
> > > > >> to
> > > > >>>>>>>>>>>> document
> > > > >>>>>>>>>>>>>> how the metrics change due to the transactions in the
> > > > upgrade
> > > > >>>>>>>> notes.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> 5.
> > > > >>>>>>>>>>>>>> I see. Then you should describe how the .position files
> > are
> > > > >>>>>>>> handled
> > > > >>>>>>>>>> in
> > > > >>>>>>>>>>>>>> a dedicated section of the KIP or incorporate the
> > > > description
> > > > >>>>>>>> in the
> > > > >>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only
> > mentioning it
> > > > >> in
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>> "Compatibility, 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:
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>
> > > > >>>>
> > > > >>
> > > >
> > https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> It is a bit brittle, but it works.
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>> Bruno
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > > > >>>>>>>>>>>>>>> Hi Bruno,
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Thanks for taking the time to review the KIP. I'm back
> > from
> > > > >>>>>>>> leave
> > > > >>>>>>>>>> now
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>> intend to move this forwards as quickly as I can.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Addressing your points:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> 1.
> > > > >>>>>>>>>>>>>>> Because flush() is part of the StateStore API, it's
> > exposed
> > > > >> to
> > > > >>>>>>>>>> custom
> > > > >>>>>>>>>>>>>>> Processors, which might be making calls to flush().
> > This
> > > > was
> > > > >>>>>>>>>> actually
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> case in a few integration tests.
> > > > >>>>>>>>>>>>>>> To maintain as much compatibility as possible, I'd
> > prefer
> > > > not
> > > > >>>>>>>> to
> > > > >>>>>>>>>> make
> > > > >>>>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>> an UnsupportedOperationException, as it will cause
> > > > previously
> > > > >>>>>>>>>> working
> > > > >>>>>>>>>>>>>>> Processors to start throwing exceptions at runtime.
> > > > >>>>>>>>>>>>>>> I agree that it doesn't make sense for it to proxy
> > > > commit(),
> > > > >>>>>>>> though,
> > > > >>>>>>>>>>>> as
> > > > >>>>>>>>>>>>>>> that would cause it to violate the "StateStores commit
> > only
> > > > >>>>>>>> when the
> > > > >>>>>>>>>>>> Task
> > > > >>>>>>>>>>>>>>> commits" rule.
> > > > >>>>>>>>>>>>>>> Instead, I think we should make this a no-op. That way,
> > > > >>>>>>>> existing
> > > > >>>>>>>>>> user
> > > > >>>>>>>>>>>>>>> Processors will continue to work as-before, without
> > > > violation
> > > > >>>>>>>> of
> > > > >>>>>>>>>> store
> > > > >>>>>>>>>>>>>>> consistency that would be caused by premature
> > flush/commit
> > > > of
> > > > >>>>>>>>>>>> StateStore
> > > > >>>>>>>>>>>>>>> data to disk.
> > > > >>>>>>>>>>>>>>> What do you think?
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
> > implementation
> > > > is
> > > > >>>>>>>>>>>>>>> transactional, but is unable to estimate the
> > uncommitted
> > > > >>>> memory
> > > > >>>>>>>>>> usage,
> > > > >>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> method will return -1.
> > > > >>>>>>>>>>>>>>> The intention here is to permit third-party
> > implementations
> > > > >>>>>>>> that may
> > > > >>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>> able to estimate memory usage.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been written to the
> > > > store
> > > > >>>>>>>> yet. I
> > > > >>>>>>>>>>>>>> thought
> > > > >>>>>>>>>>>>>>> that was implied by "This method will return an
> > > > approximation
> > > > >>>>>>>> of the
> > > > >>>>>>>>>>>>>> memory
> > > > >>>>>>>>>>>>>>> would be freed by the next call to {@link
> > #commit(Map)}"
> > > > and
> > > > >>>>>>>>>> "@return
> > > > >>>>>>>>>>>> The
> > > > >>>>>>>>>>>>>>> approximate size of all records awaiting {@link
> > > > >>>> #commit(Map)}",
> > > > >>>>>>>>>>>> however,
> > > > >>>>>>>>>>>>>> I
> > > > >>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you think this
> > is
> > > > >>>>>>>> unclear?
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>> I realise this is probably the most contentious point
> > in my
> > > > >>>>>>>> design,
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>> I'm
> > > > >>>>>>>>>>>>>>> open to changing it if I'm unable to convince you of
> > the
> > > > >>>>>>>> benefits.
> > > > >>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > > > >>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly provided
> > > > >>>>>>>> StateStores
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>>> query, and it may be important for users to
> > > > programmatically
> > > > >>>>>>>> know
> > > > >>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>>> mode the StateStore is operating under. If we simply
> > > > provide
> > > > >>>> an
> > > > >>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the internal
> > > > streams
> > > > >>>>>>>>>>>> engine), or
> > > > >>>>>>>>>>>>>>> similar, then users will need to understand the
> > operation
> > > > and
> > > > >>>>>>>>>>>>>> consequences
> > > > >>>>>>>>>>>>>>> of each available processing mode and how it pertains
> > to
> > > > >> their
> > > > >>>>>>>>>>>>>> StateStore.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Interactive Query users aren't the only people that
> > care
> > > > >> about
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
> > > > implementers
> > > > >>>> of
> > > > >>>>>>>>>> custom
> > > > >>>>>>>>>>>>>>> StateStores also need to understand the behaviour
> > expected
> > > > of
> > > > >>>>>>>> their
> > > > >>>>>>>>>>>>>>> implementation. KIP-892 introduces some assumptions
> > into
> > > > the
> > > > >>>>>>>> Streams
> > > > >>>>>>>>>>>>>> Engine
> > > > >>>>>>>>>>>>>>> about how StateStores operate under each processing
> > mode,
> > > > and
> > > > >>>>>>>> it's
> > > > >>>>>>>>>>>>>>> important that custom implementations adhere to those
> > > > >>>>>>>> assumptions in
> > > > >>>>>>>>>>>>>> order
> > > > >>>>>>>>>>>>>>> to maintain the consistency guarantees.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> IsolationLevels provide a high-level contract on the
> > > > >> behaviour
> > > > >>>>>>>> of
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>> StateStore: a user knows that under READ_COMMITTED,
> > they
> > > > will
> > > > >>>>>>>> see
> > > > >>>>>>>>>>>> writes
> > > > >>>>>>>>>>>>>>> only after the Task has committed, and under
> > > > READ_UNCOMMITTED
> > > > >>>>>>>> they
> > > > >>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>> see
> > > > >>>>>>>>>>>>>>> writes immediately. No understanding of the details of
> > each
> > > > >>>>>>>>>>>>>> processing.mode
> > > > >>>>>>>>>>>>>>> is required, either for IQ users or StateStore
> > > > implementers.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> An argument can be made that these contractual
> > guarantees
> > > > can
> > > > >>>>>>>> simply
> > > > >>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>> documented for the processing.mode (i.e. that
> > exactly-once
> > > > >> and
> > > > >>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
> > > > at-least-once
> > > > >>>>>>>> behaves
> > > > >>>>>>>>>>>> like
> > > > >>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small issues
> > with
> > > > >>>>>>>> this I'd
> > > > >>>>>>>>>>>>>> prefer
> > > > >>>>>>>>>>>>>>> to avoid:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>         - Where would we document these contracts, in
> > a way
> > > > >> that
> > > > >>>>>>>> is
> > > > >>>>>>>>>>>> difficult
> > > > >>>>>>>>>>>>>>>         for users/implementers to miss/ignore?
> > > > >>>>>>>>>>>>>>>         - It's not clear to users that the processing
> > mode
> > > > is
> > > > >>>>>>>>>>>> communicating
> > > > >>>>>>>>>>>>>>>         an expectation of read isolation, unless they
> > read
> > > > the
> > > > >>>>>>>>>>>>>> documentation. Users
> > > > >>>>>>>>>>>>>>>         rarely consult documentation unless they feel
> > they
> > > > >> need
> > > > >>>>>>>> to, so
> > > > >>>>>>>>>>>> it's
> > > > >>>>>>>>>>>>>> likely
> > > > >>>>>>>>>>>>>>>         this detail would get missed by many users.
> > > > >>>>>>>>>>>>>>>         - It tightly couples processing modes to read
> > > > >> isolation.
> > > > >>>>>>>> Adding
> > > > >>>>>>>>>>>> new
> > > > >>>>>>>>>>>>>>>         processing modes, or changing the read
> > isolation of
> > > > >>>>>>>> existing
> > > > >>>>>>>>>>>>>> processing
> > > > >>>>>>>>>>>>>>>         modes would be difficult/impossible.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Ultimately, the cost of introducing IsolationLevels is
> > > > just a
> > > > >>>>>>>> single
> > > > >>>>>>>>>>>>>>> method, since we re-use the existing IsolationLevel
> > enum
> > > > from
> > > > >>>>>>>> Kafka.
> > > > >>>>>>>>>>>> This
> > > > >>>>>>>>>>>>>>> gives us a clear place to document the contractual
> > > > guarantees
> > > > >>>>>>>>>> expected
> > > > >>>>>>>>>>>>>>> of/provided by StateStores, that is accessible both by
> > the
> > > > >>>>>>>>>> StateStore
> > > > >>>>>>>>>>>>>>> itself, and by IQ users.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> (Writing this I've just realised that the StateStore
> > and IQ
> > > > >>>>>>>> APIs
> > > > >>>>>>>>>>>> actually
> > > > >>>>>>>>>>>>>>> don't provide access to StateStoreContext that IQ users
> > > > would
> > > > >>>>>>>> have
> > > > >>>>>>>>>>>> direct
> > > > >>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> > > > >> isolationLevel()
> > > > >>>>>>>>>> itself
> > > > >>>>>>>>>>>>>> too?)
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> 4.
> > > > >>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics in-place
> > > > >>>>>>>> either, as
> > > > >>>>>>>>>>>> it's a
> > > > >>>>>>>>>>>>>>> backwards incompatible change. My concern is that, if
> > we
> > > > >> leave
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>> existing
> > > > >>>>>>>>>>>>>>> "flush" metrics in place, they will be confusing to
> > users.
> > > > >>>>>>>> Right
> > > > >>>>>>>>>> now,
> > > > >>>>>>>>>>>>>>> "flush" metrics record explicit flushes to disk, but
> > under
> > > > >>>>>>>> KIP-892,
> > > > >>>>>>>>>>>> even
> > > > >>>>>>>>>>>>>> a
> > > > >>>>>>>>>>>>>>> commit() will not explicitly flush data to disk -
> > RocksDB
> > > > >> will
> > > > >>>>>>>>>> decide
> > > > >>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>> when to flush memtables to disk itself.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd have two
> > > > >>>> options,
> > > > >>>>>>>>>> which
> > > > >>>>>>>>>>>>>> both
> > > > >>>>>>>>>>>>>>> seem pretty bad to me:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>         1. Have them record calls to commit(), which
> > would
> > > > be
> > > > >>>>>>>>>>>> misleading, as
> > > > >>>>>>>>>>>>>>>         data is no longer explicitly "flushed" to disk
> > by
> > > > this
> > > > >>>>>>>> call.
> > > > >>>>>>>>>>>>>>>         2. Have them record nothing at all, which is
> > > > >> equivalent
> > > > >>>> to
> > > > >>>>>>>>>>>> removing
> > > > >>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>         metrics, except that users will see the metric
> > > > still
> > > > >>>>>>>> exists and
> > > > >>>>>>>>>>>> so
> > > > >>>>>>>>>>>>>> assume
> > > > >>>>>>>>>>>>>>>         that the metric is correct, and that there's a
> > > > problem
> > > > >>>>>>>> with
> > > > >>>>>>>>>> their
> > > > >>>>>>>>>>>>>> system
> > > > >>>>>>>>>>>>>>>         when there isn't.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> I agree that removing them is also a bad solution, and
> > I'd
> > > > >>>>>>>> like some
> > > > >>>>>>>>>>>>>>> guidance on the best path forward here.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> 5.
> > > > >>>>>>>>>>>>>>> Position files are updated on every write to a
> > StateStore.
> > > > >>>>>>>> Since our
> > > > >>>>>>>>>>>>>> writes
> > > > >>>>>>>>>>>>>>> are now buffered until commit(), we can't update the
> > > > Position
> > > > >>>>>>>> file
> > > > >>>>>>>>>>>> until
> > > > >>>>>>>>>>>>>>> commit() has been called, otherwise it would be
> > > > inconsistent
> > > > >>>>>>>> with
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>> data
> > > > >>>>>>>>>>>>>>> in the event of a rollback. Consequently, we need to
> > manage
> > > > >>>>>>>> these
> > > > >>>>>>>>>>>> offsets
> > > > >>>>>>>>>>>>>>> the same way we manage the checkpoint offsets, and
> > ensure
> > > > >>>>>>>> they're
> > > > >>>>>>>>>> only
> > > > >>>>>>>>>>>>>>> written on commit().
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> 6.
> > > > >>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what tests to
> > > > >> write.
> > > > >>>>>>>> How
> > > > >>>>>>>>>>>>>> explicit
> > > > >>>>>>>>>>>>>>> do we need to be here in the KIP?
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be
> > > > seamless,
> > > > >>>>>>>> and we
> > > > >>>>>>>>>>>>>> should
> > > > >>>>>>>>>>>>>>> definitely add some tests around that. Downgrade, it
> > > > >>>>>>>> transpires,
> > > > >>>>>>>>>> isn't
> > > > >>>>>>>>>>>>>>> currently possible, as the extra column family for
> > offset
> > > > >>>>>>>> storage is
> > > > >>>>>>>>>>>>>>> incompatible with the pre-KIP-892 implementation: when
> > you
> > > > >>>>>>>> open a
> > > > >>>>>>>>>>>> RocksDB
> > > > >>>>>>>>>>>>>>> database, you must open all available column families
> > or
> > > > >>>>>>>> receive an
> > > > >>>>>>>>>>>>>> error.
> > > > >>>>>>>>>>>>>>> What currently happens on downgrade is that it
> > attempts to
> > > > >>>>>>>> open the
> > > > >>>>>>>>>>>>>> store,
> > > > >>>>>>>>>>>>>>> throws an error about the offsets column family not
> > being
> > > > >>>>>>>> opened,
> > > > >>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given that
> > > > >> downgrades
> > > > >>>>>>>>>> should
> > > > >>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>> uncommon, I think this is acceptable behaviour, as the
> > > > >>>>>>>> end-state is
> > > > >>>>>>>>>>>>>>> consistent, even if it results in an undesirable state
> > > > >>>> restore.
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Should I document the upgrade/downgrade behaviour
> > > > explicitly
> > > > >>>>>>>> in the
> > > > >>>>>>>>>>>> KIP?
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> --
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > > > >>>>>>>> cadonna@apache.org>
> > > > >>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Hi Nick!
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Thanks for the updates!
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> 1.
> > > > >>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> > > > >>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > > > >>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore after
> > this
> > > > KIP
> > > > >>>>>>>> is
> > > > >>>>>>>>>>>>>>>> released, I would rather throw an unsupported
> > operation
> > > > >>>>>>>> exception
> > > > >>>>>>>>>> by
> > > > >>>>>>>>>>>>>>>> default.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>>>> When would a state store return -1 from
> > > > >>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while
> > being
> > > > >>>>>>>>>>>> transactional?
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Wouldn't StateStore#approximateNumUncommittedBytes()
> > also
> > > > >>>>>>>> return 0
> > > > >>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>> the state store is transactional but nothing has been
> > > > >> written
> > > > >>>>>>>> to
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> state store yet?
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this KIP really
> > > > need
> > > > >>>> to
> > > > >>>>>>>>>>>> introduce
> > > > >>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()? StateStoreContext
> > has
> > > > >>>>>>>> already
> > > > >>>>>>>>>>>>>>>> appConfigs() which basically exposes the same
> > information,
> > > > >>>>>>>> i.e., if
> > > > >>>>>>>>>>>> EOS
> > > > >>>>>>>>>>>>>>>> is enabled or not.
> > > > >>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore interface
> > as
> > > > >>>>>>>> loosely
> > > > >>>>>>>>>>>> coupled
> > > > >>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > implementers
> > > > >>>> more
> > > > >>>>>>>>>>>> freedom,
> > > > >>>>>>>>>>>>>>>> and reduce the amount of internal knowledge required."
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> While I understand the intent, I doubt that it
> > decreases
> > > > the
> > > > >>>>>>>>>>>> coupling of
> > > > >>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> > > > >> READ_COMMITTED
> > > > >>>>>>>> only
> > > > >>>>>>>>>>>>>>>> applies to IQ but not to reads by processors. Thus,
> > > > >>>>>>>> implementers
> > > > >>>>>>>>>>>> need to
> > > > >>>>>>>>>>>>>>>> understand how Streams accesses the state stores.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> I would like to hear what others think about this.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> 4.
> > > > >>>>>>>>>>>>>>>> Great exposing new metrics for transactional state
> > stores!
> > > > >>>>>>>>>> However, I
> > > > >>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate (in the
> > > > docs)
> > > > >>>>>>>> the 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 also need to adapt/add integration
> > tests
> > > > >>>>>>>> besides
> > > > >>>>>>>>>> unit
> > > > >>>>>>>>>>>>>>>> tests. Additionally, we probably need integration or
> > > > system
> > > > >>>>>>>> tests
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>> verify that upgrades and downgrades between
> > transactional
> > > > >> and
> > > > >>>>>>>>>>>>>>>> non-transactional state stores work as expected.
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>> Bruno
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > > > >>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in the KIP,
> > > > under
> > > > >>>>>>>>>>>> "Rejected
> > > > >>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea worth
> > pursuing,
> > > > >>>>>>>> but I
> > > > >>>>>>>>>>>> believe
> > > > >>>>>>>>>>>>>>>>> that it's out of the scope of this KIP, because it
> > > > solves a
> > > > >>>>>>>>>>>> different
> > > > >>>>>>>>>>>>>> set
> > > > >>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of this one
> > has
> > > > >>>>>>>> already
> > > > >>>>>>>>>> grown
> > > > >>>>>>>>>>>>>>>> quite
> > > > >>>>>>>>>>>>>>>>> large!
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > > > >>>>>>>>>> nick.telford@gmail.com>
> > > > >>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> Hi everyone,
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> I've updated the KIP (
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>
> > > > >>>>
> > > > >>
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > >>>>>>>>>>>>>>>> )
> > > > >>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing back
> > "Atomic
> > > > >>>>>>>>>>>> Checkpointing"
> > > > >>>>>>>>>>>>>>>> (for
> > > > >>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think the one
> > thing
> > > > >>>>>>>> missing is
> > > > >>>>>>>>>>>> some
> > > > >>>>>>>>>>>>>>>>>> changes to metrics (notably the store "flush"
> > metrics
> > > > will
> > > > >>>>>>>> need
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>> renamed to "commit").
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> The reason I brought back Atomic Checkpointing was
> > to
> > > > >>>>>>>> decouple
> > > > >>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>> flush
> > > > >>>>>>>>>>>>>>>>>> from store commit. This is important, because with
> > > > >>>>>>>> Transactional
> > > > >>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on *every*
> > Task
> > > > >>>>>>>> commit,
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>> just when the StateStore is closing, otherwise our
> > > > >>>>>>>> transaction
> > > > >>>>>>>>>>>> buffer
> > > > >>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>> never be written and persisted, instead growing
> > > > unbounded!
> > > > >>>> I
> > > > >>>>>>>>>>>>>>>> experimented
> > > > >>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a store
> > flush
> > > > >>>>>>>> whenever
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its
> > configured
> > > > >>>>>>>> size, but
> > > > >>>>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>> was
> > > > >>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer from
> > being
> > > > >>>>>>>>>> configured
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>> unbounded, and it still would have required explicit
> > > > >>>>>>>> flushes of
> > > > >>>>>>>>>>>>>> RocksDB,
> > > > >>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory
> > utilization.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right" way
> > to
> > > > >>>>>>>> resolve
> > > > >>>>>>>>>> this
> > > > >>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog offsets that
> > > > >>>>>>>> correspond
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> most
> > > > >>>>>>>>>>>>>>>>>> recently written records are always atomically
> > written
> > > > to
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>> StateStore
> > > > >>>>>>>>>>>>>>>>>> (by writing them to the same transaction buffer),
> > we can
> > > > >>>>>>>> avoid
> > > > >>>>>>>>>>>>>> forcibly
> > > > >>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting
> > RocksDB
> > > > >>>>>>>> flush
> > > > >>>>>>>>>> them
> > > > >>>>>>>>>>>>>> only
> > > > >>>>>>>>>>>>>>>>>> when necessary, without losing any of our
> > consistency
> > > > >>>>>>>> guarantees.
> > > > >>>>>>>>>>>> See
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>> updated KIP for more info.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> I have fully implemented these changes, although I'm
> > > > still
> > > > >>>>>>>> not
> > > > >>>>>>>>>>>>>> entirely
> > > > >>>>>>>>>>>>>>>>>> happy with the implementation for segmented
> > StateStores,
> > > > >> so
> > > > >>>>>>>> I
> > > > >>>>>>>>>> plan
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests pass. If
> > you'd
> > > > like
> > > > >>>>>>>> to try
> > > > >>>>>>>>>>>> out
> > > > >>>>>>>>>>>>>> or
> > > > >>>>>>>>>>>>>>>>>> review this highly experimental and incomplete
> > branch,
> > > > >> it's
> > > > >>>>>>>>>>>> available
> > > > >>>>>>>>>>>>>>>> here:
> > > > >>>>>>>>>>>>>>>>>>
> > https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > > > .
> > > > >>>>>>>> Note:
> > > > >>>>>>>>>>>> it's
> > > > >>>>>>>>>>>>>>>> built
> > > > >>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base to
> > build
> > > > >>>>>>>> and test
> > > > >>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>> on,
> > > > >>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples comparisons in a
> > > > live
> > > > >>>>>>>>>>>>>> environment. I
> > > > >>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's nearer
> > > > >> completion
> > > > >>>>>>>> and
> > > > >>>>>>>>>> has
> > > > >>>>>>>>>>>>>> been
> > > > >>>>>>>>>>>>>>>>>> proven on our main application.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing and
> > testing:
> > > > >>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window) stores
> > > > >>>>>>>>>>>>>>>>>> - Global stores
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> As I do not currently use either of these, so my
> > primary
> > > > >>>>>>>> test
> > > > >>>>>>>>>>>>>>>> environment
> > > > >>>>>>>>>>>>>>>>>> doesn't test these areas.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next week for
> > a few
> > > > >>>>>>>> weeks,
> > > > >>>>>>>>>> so
> > > > >>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>> not have time to move this forward until late
> > August.
> > > > That
> > > > >>>>>>>> said,
> > > > >>>>>>>>>>>> your
> > > > >>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just won't be
> > > > able
> > > > >>>> to
> > > > >>>>>>>>>>>> respond
> > > > >>>>>>>>>>>>>> as
> > > > >>>>>>>>>>>>>>>>>> quickly as usual.
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > > > >>>>>>>>>> nick.telford@gmail.com>
> > > > >>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Hi Bruno
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ is
> > not
> > > > >>>>>>>> something
> > > > >>>>>>>>>> I
> > > > >>>>>>>>>>>> had
> > > > >>>>>>>>>>>>>>>>>>> considered.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> What about atomically updating the state store
> > from the
> > > > >>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> > > > checkpoint
> > > > >>>>>>>> (thus,
> > > > >>>>>>>>>>>>>>>> flushing
> > > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data
> > and/or
> > > > >>>>>>>> number of
> > > > >>>>>>>>>>>>>> commit
> > > > >>>>>>>>>>>>>>>>>>>> intervals?
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting
> > that we
> > > > >>>>>>>> add an
> > > > >>>>>>>>>>>>>>>> additional
> > > > >>>>>>>>>>>>>>>>>>> config for the max number of commit intervals
> > between
> > > > >>>>>>>>>> checkpoints?
> > > > >>>>>>>>>>>>>> That
> > > > >>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the
> > transaction
> > > > >>>>>>>> buffers
> > > > >>>>>>>>>> are
> > > > >>>>>>>>>>>>>>>> nearly
> > > > >>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of commit
> > > > intervals
> > > > >>>>>>>> have
> > > > >>>>>>>>>>>>>> elapsed,
> > > > >>>>>>>>>>>>>>>>>>> whichever comes first?
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although this
> > > > re-ignites
> > > > >>>>>>>> an
> > > > >>>>>>>>>>>> earlier
> > > > >>>>>>>>>>>>>>>>>>> debate about whether a config should be measured in
> > > > >>>>>>>> "number of
> > > > >>>>>>>>>>>> commit
> > > > >>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute time.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the reason I
> > was
> > > > >>>>>>>> pursuing
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> Atomic
> > > > >>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush from
> > > > >>>>>>>> checkpointing,
> > > > >>>>>>>>>>>>>> which
> > > > >>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every commit
> > without
> > > > any
> > > > >>>>>>>>>>>> performance
> > > > >>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the
> > "best"
> > > > >>>>>>>> solution,
> > > > >>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>> I'm not
> > > > >>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back into this
> > KIP.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
> > transactional
> > > > >>>> logic
> > > > >>>>>>>>>>>> directly
> > > > >>>>>>>>>>>>>>>> into
> > > > >>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with the
> > > > >>>>>>>>>>>>>> StateStore#newTransaction
> > > > >>>>>>>>>>>>>>>>>>> method, and reduces the number of new classes
> > > > introduced,
> > > > >>>>>>>>>>>>>> significantly
> > > > >>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and the
> > > > complexity
> > > > >>>> is
> > > > >>>>>>>>>>>>>> drastically
> > > > >>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic Checkpoints
> > > > into
> > > > >>>>>>>> this
> > > > >>>>>>>>>> KIP.
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
> > > > >>>>>>>> cadonna@apache.org>
> > > > >>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> As far as I understand, you want to atomically
> > update
> > > > >> the
> > > > >>>>>>>> state
> > > > >>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the memtable
> > of a
> > > > >>>> state
> > > > >>>>>>>>>> store
> > > > >>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit time
> > elapsed
> > > > >>>> but
> > > > >>>>>>>>>> after
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that would lead
> > to
> > > > >>>>>>>> exceeding
> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes before the
> > > > next
> > > > >>>>>>>> commit
> > > > >>>>>>>>>>>>>>>> interval
> > > > >>>>>>>>>>>>>>>>>>>> ends.
> > > > >>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would commit
> > every
> > > > >>>>>>>> commit
> > > > >>>>>>>>>>>> interval
> > > > >>>>>>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>>>>>> the state store will only be atomically updated
> > > > roughly
> > > > >>>>>>>> every
> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of data.
> > Also
> > > > IQ
> > > > >>>>>>>> would
> > > > >>>>>>>>>>>> then
> > > > >>>>>>>>>>>>>>>> only
> > > > >>>>>>>>>>>>>>>>>>>> see new data roughly every
> > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > >>>>>>>>>>>>>>>>>>>> After a failure the state store needs to restore
> > up to
> > > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> Is this correct?
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> What about atomically updating the state store
> > from
> > > > the
> > > > >>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> > > > checkpoint
> > > > >>>>>>>> (thus,
> > > > >>>>>>>>>>>>>>>> flushing
> > > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data
> > and/or
> > > > >>>>>>>> number of
> > > > >>>>>>>>>>>>>> commit
> > > > >>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have the same
> > delay
> > > > >>>> for
> > > > >>>>>>>>>>>> records
> > > > >>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because both
> > would
> > > > >>>>>>>> appear
> > > > >>>>>>>>>> when
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However, after a
> > > > failure
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > >>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>> it might restore data that is already in the state
> > > > store
> > > > >>>>>>>>>> because
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable offset
> > (i.e.
> > > > the
> > > > >>>>>>>> last
> > > > >>>>>>>>>>>>>> committed
> > > > >>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring data
> > that
> > > > is
> > > > >>>>>>>> already
> > > > >>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should not
> > violated.
> > > > >>>>>>>>>>>>>>>>>>>> This solution needs at least one new config to
> > specify
> > > > >>>>>>>> when a
> > > > >>>>>>>>>>>>>>>> checkpoint
> > > > >>>>>>>>>>>>>>>>>>>> should be written.
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail that
> > does
> > > > not
> > > > >>>>>>>> change
> > > > >>>>>>>>>>>>>>>> anything
> > > > >>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit interval
> > is 30
> > > > >>>>>>>> seconds,
> > > > >>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>> five
> > > > >>>>>>>>>>>>>>>>>>>> seconds.
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>>>> Bruno
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > > > >>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch on our
> > > > staging
> > > > >>>>>>>>>>>>>> environment,
> > > > >>>>>>>>>>>>>>>>>>>>> putting it through its paces in our non-trivial
> > > > >>>>>>>> application.
> > > > >>>>>>>>>> I'm
> > > > >>>>>>>>>>>>>>>>>>>> already
> > > > >>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate that we
> > saw
> > > > the
> > > > >>>>>>>> last
> > > > >>>>>>>>>>>> time
> > > > >>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP, but this
> > > > time,
> > > > >> I
> > > > >>>>>>>>>> think I
> > > > >>>>>>>>>>>>>> know
> > > > >>>>>>>>>>>>>>>>>>>> why.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is
> > called
> > > > at
> > > > >>>>>>>> the end
> > > > >>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> Task
> > > > >>>>>>>>>>>>>>>>>>>>> commit process, has the following behaviour:
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>           - Under ALOS: checkpoint the state
> > stores.
> > > > >> This
> > > > >>>>>>>>>> includes
> > > > >>>>>>>>>>>>>>>>>>>>>           flushing memtables in RocksDB. This is
> > > > >>>> acceptable
> > > > >>>>>>>>>>>> because the
> > > > >>>>>>>>>>>>>>>>>>>> default
> > > > >>>>>>>>>>>>>>>>>>>>>           commit.interval.ms is 5 seconds, so
> > > > forcibly
> > > > >>>>>>>> flushing
> > > > >>>>>>>>>>>>>> memtables
> > > > >>>>>>>>>>>>>>>>>>>> every 5
> > > > >>>>>>>>>>>>>>>>>>>>>           seconds is acceptable for most
> > > > applications.
> > > > >>>>>>>>>>>>>>>>>>>>>           - Under EOS: checkpointing is not done,
> > > > >> *unless*
> > > > >>>>>>>> it's
> > > > >>>>>>>>>>>> being
> > > > >>>>>>>>>>>>>>>>>>>> forced, due
> > > > >>>>>>>>>>>>>>>>>>>>>           to e.g. the Task closing or being
> > revoked.
> > > > >> This
> > > > >>>>>>>> means
> > > > >>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>> under
> > > > >>>>>>>>>>>>>>>>>>>> normal
> > > > >>>>>>>>>>>>>>>>>>>>>           processing conditions, the state stores
> > > > will
> > > > >> not
> > > > >>>>>>>> be
> > > > >>>>>>>>>>>>>>>> checkpointed,
> > > > >>>>>>>>>>>>>>>>>>>> and will
> > > > >>>>>>>>>>>>>>>>>>>>>           not have memtables flushed at all ,
> > unless
> > > > >>>> RocksDB
> > > > >>>>>>>>>>>> decides to
> > > > >>>>>>>>>>>>>>>>>>>> flush them on
> > > > >>>>>>>>>>>>>>>>>>>>>           its own. Checkpointing stores and
> > > > >> force-flushing
> > > > >>>>>>>> their
> > > > >>>>>>>>>>>>>> memtables
> > > > >>>>>>>>>>>>>>>>>>>> is only
> > > > >>>>>>>>>>>>>>>>>>>>>           done when a Task is being closed.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on
> > at
> > > > >>>> least
> > > > >>>>>>>>>> *some*
> > > > >>>>>>>>>>>>>>>> normal
> > > > >>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB
> > > > transaction
> > > > >>>>>>>>>> buffers
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets are
> > synced to
> > > > >>>>>>>> disk to
> > > > >>>>>>>>>>>>>> prevent
> > > > >>>>>>>>>>>>>>>>>>>>> restores from getting out of hand. Consequently,
> > my
> > > > >>>>>>>> current
> > > > >>>>>>>>>>>>>>>>>>>> implementation
> > > > >>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit,
> > which
> > > > is
> > > > >>>>>>>> far too
> > > > >>>>>>>>>>>>>>>>>>>> frequent.
> > > > >>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records,
> > which
> > > > is
> > > > >> a
> > > > >>>>>>>>>> change
> > > > >>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>> flush
> > > > >>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing performance
> > problems
> > > > for
> > > > >>>>>>>> some
> > > > >>>>>>>>>>>>>>>>>>>> applications.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm
> > > > currently
> > > > >>>>>>>> leaning
> > > > >>>>>>>>>>>>>>>> towards
> > > > >>>>>>>>>>>>>>>>>>>>> using the statestore.transaction.buffer.max.bytes
> > > > >>>>>>>>>> configuration
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to exceed it.
> > > > This
> > > > >>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>> complement the
> > > > >>>>>>>>>>>>>>>>>>>>> existing "early Task commit" functionality that
> > this
> > > > >>>>>>>>>>>> configuration
> > > > >>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>           - Currently, we use
> > > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > > >>>>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> force an
> > > > >>>>>>>>>>>>>>>>>>>>>           early Task commit if processing more
> > > > records
> > > > >>>> would
> > > > >>>>>>>>>> cause
> > > > >>>>>>>>>>>> our
> > > > >>>>>>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>>>           transactions to exceed the memory
> > assigned
> > > > to
> > > > >>>>>>>> them.
> > > > >>>>>>>>>>>>>>>>>>>>>           - New functionality: when a Task *does*
> > > > >> commit,
> > > > >>>>>>>> we will
> > > > >>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>> checkpoint
> > > > >>>>>>>>>>>>>>>>>>>>>           the stores (and hence flush the
> > transaction
> > > > >>>>>>>> buffers)
> > > > >>>>>>>>>>>> unless
> > > > >>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>> expect to
> > > > >>>>>>>>>>>>>>>>>>>>>           cross the
> > > > >>>> statestore.transaction.buffer.max.bytes
> > > > >>>>>>>>>>>> threshold
> > > > >>>>>>>>>>>>>>>> before
> > > > >>>>>>>>>>>>>>>>>>>> the next
> > > > >>>>>>>>>>>>>>>>>>>>>           commit
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> > > > >>>>>>>>>>>> nick.telford@gmail.com
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I meant in
> > terms of
> > > > >>>>>>>>>>>> understanding
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> performance profile under various
> > circumstances. The
> > > > >>>>>>>> more
> > > > >>>>>>>>>>>> complex
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would be for
> > users
> > > > to
> > > > >>>>>>>>>>>> understand
> > > > >>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> performance they see. For example, spilling
> > records
> > > > to
> > > > >>>>>>>> disk
> > > > >>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold would, I
> > > > >> expect,
> > > > >>>>>>>>>> reduce
> > > > >>>>>>>>>>>>>> write
> > > > >>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write throughput
> > could
> > > > >> be
> > > > >>>>>>>>>>>>>> unexpected,
> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>> potentially difficult to diagnose/understand for
> > > > >> users.
> > > > >>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early commit"
> > concept is
> > > > >>>>>>>>>> relatively
> > > > >>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document, and
> > > > >>>> conceptually
> > > > >>>>>>>>>> fairly
> > > > >>>>>>>>>>>>>>>>>>>> obvious to
> > > > >>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to make it
> > > > >> easier
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>>>>>> understand
> > > > >>>>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>>>> it happens though.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > > > >>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an indirect
> > way
> > > > of
> > > > >>>>>>>>>> telling
> > > > >>>>>>>>>>>>>>>>>>>> StateStores
> > > > >>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
> > READ_COMMITTED
> > > > >>>>>>>>>>>> essentially
> > > > >>>>>>>>>>>>>>>>>>>> requires
> > > > >>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that two
> > threads
> > > > >>>>>>>> calling
> > > > >>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes from
> > the
> > > > >> other
> > > > >>>>>>>>>>>>>> transaction
> > > > >>>>>>>>>>>>>>>>>>>> until
> > > > >>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> > READ_UNCOMMITTED, all
> > > > >>>>>>>> bets are
> > > > >>>>>>>>>>>> off,
> > > > >>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe written
> > records
> > > > at
> > > > >>>>>>>> any
> > > > >>>>>>>>>>>> time,
> > > > >>>>>>>>>>>>>>>>>>>> which is
> > > > >>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That said,
> > > > StateStores
> > > > >>>>>>>> are
> > > > >>>>>>>>>> free
> > > > >>>>>>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>> implement
> > > > >>>>>>>>>>>>>>>>>>>>>> these guarantees however they can, which is a
> > bit
> > > > more
> > > > >>>>>>>>>> relaxed
> > > > >>>>>>>>>>>>>> than
> > > > >>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions". For
> > example,
> > > > >>>> with
> > > > >>>>>>>>>>>> RocksDB
> > > > >>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>> would
> > > > >>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
> > > > >>>>>>>>>> "transactions",
> > > > >>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the
> > database.
> > > > But
> > > > >>>>>>>> with
> > > > >>>>>>>>>>>> other
> > > > >>>>>>>>>>>>>>>>>>>> storage
> > > > >>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to *always* use
> > > > >>>>>>>> transactions,
> > > > >>>>>>>>>>>> even
> > > > >>>>>>>>>>>>>>>>>>>> when
> > > > >>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines
> > that
> > > > >> don't
> > > > >>>>>>>>>> provide
> > > > >>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation guarantees can
> > be
> > > > met
> > > > >>>>>>>> using a
> > > > >>>>>>>>>>>>>>>>>>>> different
> > > > >>>>>>>>>>>>>>>>>>>>>> technique.
> > > > >>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore
> > interface
> > > > as
> > > > >>>>>>>>>> loosely
> > > > >>>>>>>>>>>>>>>> coupled
> > > > >>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > > > >>>>>>>> implementers
> > > > >>>>>>>>>> more
> > > > >>>>>>>>>>>>>>>>>>>> freedom, and
> > > > >>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge
> > required.
> > > > >>>>>>>>>>>>>>>>>>>>>> That said, I understand that "IsolationLevel"
> > might
> > > > >> not
> > > > >>>>>>>> be
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>> right
> > > > >>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it much more
> > > > >>>>>>>> explicit if
> > > > >>>>>>>>>>>>>>>>>>>> required, e.g.
> > > > >>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> 7-8.
> > > > >>>>>>>>>>>>>>>>>>>>>> I can make these changes either later today or
> > > > >>>> tomorrow.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Small update:
> > > > >>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a
> > bunch of
> > > > >>>>>>>> issues
> > > > >>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>> needed
> > > > >>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass,
> > which is
> > > > >>>>>>>>>> promising,
> > > > >>>>>>>>>>>> but
> > > > >>>>>>>>>>>>>>>> it
> > > > >>>>>>>>>>>>>>>>>>>> will
> > > > >>>>>>>>>>>>>>>>>>>>>> need to undergo some performance testing. I
> > haven't
> > > > >>>>>>>> (yet)
> > > > >>>>>>>>>>>> worked
> > > > >>>>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I
> > would
> > > > >>>>>>>> expect
> > > > >>>>>>>>>> that,
> > > > >>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no difference. The
> > > > >> branch
> > > > >>>>>>>> is
> > > > >>>>>>>>>>>>>> available
> > > > >>>>>>>>>>>>>>>>>>>> at
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > https://github.com/nicktelford/kafka/tree/KIP-892-c
> > > > >> if
> > > > >>>>>>>>>> anyone
> > > > >>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> > > > >>>>>>>>>>>> cadonna@apache.org>
> > > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually also
> > my
> > > > >>>>>>>> point. I
> > > > >>>>>>>>>>>>>>>> understood
> > > > >>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion path as
> > a way
> > > > >> to
> > > > >>>>>>>> avoid
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> early
> > > > >>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually it is
> > public
> > > > >>>>>>>> API. My
> > > > >>>>>>>>>>>>>> question
> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal
> > processing.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the size of the
> > > > >>>>>>>> transaction
> > > > >>>>>>>>>>>>>> buffer
> > > > >>>>>>>>>>>>>>>>>>>> and
> > > > >>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit affects the
> > whole
> > > > >>>>>>>>>> processing
> > > > >>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > > >>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a state store
> > is
> > > > not
> > > > >>>>>>>>>>>> confined to
> > > > >>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and
> > changes the
> > > > >>>>>>>> behavior
> > > > >>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>> other
> > > > >>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you that it
> > is a
> > > > >>>>>>>> decent
> > > > >>>>>>>>>>>>>>>>>>>> compromise. I
> > > > >>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and list
> > the
> > > > >>>>>>>> options to
> > > > >>>>>>>>>>>>>>>> overcome
> > > > >>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all options
> > seem
> > > > >>>> quite
> > > > >>>>>>>>>> heavy
> > > > >>>>>>>>>>>>>>>>>>>> compared
> > > > >>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you
> > mean
> > > > with
> > > > >>>>>>>> "less
> > > > >>>>>>>>>>>>>>>>>>>> predictable
> > > > >>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the alternatives
> > > > really
> > > > >>>>>>>>>>>>>> interesting.
> > > > >>>>>>>>>>>>>>>>>>>> But I
> > > > >>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good and we
> > should
> > > > >>>>>>>> continue
> > > > >>>>>>>>>>>> with
> > > > >>>>>>>>>>>>>>>> it!
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on
> > June
> > > > >> 20th:
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind
> > putting
> > > > >>>>>>>> isolation
> > > > >>>>>>>>>>>>>> level
> > > > >>>>>>>>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should that
> > also
> > > > be
> > > > >> a
> > > > >>>>>>>> way
> > > > >>>>>>>>>> to
> > > > >>>>>>>>>>>>>> give
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to decide
> > whether
> > > > to
> > > > >>>>>>>> turn on
> > > > >>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > > > >>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned about
> > how do
> > > > >> you
> > > > >>>>>>>> know
> > > > >>>>>>>>>>>> if a
> > > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written under EOS,
> > if
> > > > you
> > > > >>>>>>>> do not
> > > > >>>>>>>>>>>>>> have a
> > > > >>>>>>>>>>>>>>>>>>>> way
> > > > >>>>>>>>>>>>>>>>>>>>>>> to know if the state store is transactional or
> > not.
> > > > >> If
> > > > >>>>>>>> a
> > > > >>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be
> > written
> > > > >>>>>>>> during
> > > > >>>>>>>>>>>> normal
> > > > >>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state store is not
> > > > >>>>>>>>>> transactional,
> > > > >>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written under EOS.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 7.
> > > > >>>>>>>>>>>>>>>>>>>>>>> My point was about not only considering the
> > bytes
> > > > in
> > > > >>>>>>>> memory
> > > > >>>>>>>>>> in
> > > > >>>>>>>>>>>>>>>> config
> > > > >>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also
> > bytes
> > > > that
> > > > >>>>>>>> might
> > > > >>>>>>>>>> be
> > > > >>>>>>>>>>>>>>>>>>>> spilled
> > > > >>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering whether you
> > > > >> should
> > > > >>>>>>>>>> remove
> > > > >>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory bytes to
> > be
> > > > >> used
> > > > >>>>>>>> to
> > > > >>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My
> > > > thinking
> > > > >>>>>>>> was
> > > > >>>>>>>>>> that
> > > > >>>>>>>>>>>>>> even
> > > > >>>>>>>>>>>>>>>>>>>> if a
> > > > >>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to disk,
> > > > >> limiting
> > > > >>>>>>>> the
> > > > >>>>>>>>>>>>>> overall
> > > > >>>>>>>>>>>>>>>>>>>> bytes
> > > > >>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it again and
> > > > >>>>>>>> considering
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>> recent
> > > > >>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much sense
> > > > anymore.
> > > > >>>>>>>>>>>>>>>>>>>>>>> I like the name
> > > > >>>>>>>> statestore.transaction.buffer.max.bytes that
> > > > >>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>> proposed.
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> 8.
> > > > >>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> > implementation
> > > > >>>>>>>> details) of
> > > > >>>>>>>>>>>> how
> > > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > > >>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
> > > > >>>>>>>> transactions,
> > > > >>>>>>>>>>>> state
> > > > >>>>>>>>>>>>>>>> store
> > > > >>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be great.
> > > > Would
> > > > >>>> be
> > > > >>>>>>>>>> great
> > > > >>>>>>>>>>>> if
> > > > >>>>>>>>>>>>>>>> you
> > > > >>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the
> > behavior in
> > > > >>>>>>>> case of
> > > > >>>>>>>>>> a
> > > > >>>>>>>>>>>>>>>>>>>> failure.
> > > > >>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional state
> > store
> > > > >>>>>>>> recover
> > > > >>>>>>>>>>>> after a
> > > > >>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the transaction
> > > > buffer,
> > > > >>>>>>>> etc.
> > > > >>>>>>>>>>>> (that
> > > > >>>>>>>>>>>>>> is
> > > > >>>>>>>>>>>>>>>>>>>> what
> > > > >>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > > >>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> > > > >>>>>>>> WriteBatchWithIndex
> > > > >>>>>>>>>>>>>>>>>>>> transactions
> > > > >>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
> > > > >>>>>>>> configured
> > > > >>>>>>>>>>>> memory
> > > > >>>>>>>>>>>>>>>>>>>> needs to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns. Ultimately,
> > any
> > > > >>>>>>>> approach
> > > > >>>>>>>>>>>> based
> > > > >>>>>>>>>>>>>> on
> > > > >>>>>>>>>>>>>>>>>>>>>>> cracking
> > > > >>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in ways
> > it's
> > > > not
> > > > >>>>>>>> really
> > > > >>>>>>>>>>>>>>>> designed
> > > > >>>>>>>>>>>>>>>>>>>>>>> for is
> > > > >>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen performance or
> > > > >>>>>>>> consistency
> > > > >>>>>>>>>>>> issues.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing these
> > early
> > > > >>>>>>>> commits?
> > > > >>>>>>>>>>>> While
> > > > >>>>>>>>>>>>>> not
> > > > >>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > > > >>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to ensure
> > > > >>>>>>>> consistency
> > > > >>>>>>>>>>>> whilst
> > > > >>>>>>>>>>>>>>>>>>>>>>> maintaining
> > > > >>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> > > > >>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
> > > > >>>>>>>> complicated, and
> > > > >>>>>>>>>>>> might
> > > > >>>>>>>>>>>>>>>>>>>>>>> actually
> > > > >>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for users as a
> > > > >>>>>>>> consequence.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this
> > KIP is
> > > > >>>>>>>> growing a
> > > > >>>>>>>>>>>> bit
> > > > >>>>>>>>>>>>>>>> out
> > > > >>>>>>>>>>>>>>>>>>>> of
> > > > >>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas for
> > > > future
> > > > >>>>>>>>>>>>>>>> improvements, I
> > > > >>>>>>>>>>>>>>>>>>>>>>> think

> > > > >>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope down to a
> > > > design
> > > > >>>>>>>> that
> > > > >>>>>>>>>>>>>> achieves
> > > > >>>>>>>>>>>>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>>>>> most
> > > > >>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
> > restorations
> > > > >>>>>>>> during
> > > > >>>>>>>>>> dirty
> > > > >>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that
> > this KIP
> > > > >>>>>>>> produces
> > > > >>>>>>>>>>>> can
> > > > >>>>>>>>>>>>>>>>>>>>>>> ultimately
> > > > >>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially if the
> > bulk
> > > > of
> > > > >>>>>>>> it is
> > > > >>>>>>>>>>>>>> internal
> > > > >>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week trying
> > to
> > > > >>>>>>>> re-work
> > > > >>>>>>>>>> the
> > > > >>>>>>>>>>>>>>>>>>>> original
> > > > >>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
> > > > >>>>>>>> newTransaction()
> > > > >>>>>>>>>>>>>> method,
> > > > >>>>>>>>>>>>>>>>>>>> such
> > > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > > >>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
> > > > RocksDBStore.
> > > > >>>>>>>> That
> > > > >>>>>>>>>>>> way, if
> > > > >>>>>>>>>>>>>>>> we
> > > > >>>>>>>>>>>>>>>>>>>>>>> want to
> > > > >>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the future,
> > like
> > > > the
> > > > >>>>>>>> SST
> > > > >>>>>>>>>> file
> > > > >>>>>>>>>>>>>>>>>>>> management
> > > > >>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with
> > little/no
> > > > >>>> API
> > > > >>>>>>>>>>>> changes.
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>>
> > > > >>>>>>>>>>>>>
> > > > >>>>>>>>>>>>
> > > > >>>>>>>>>>>
> > > > >>>>>>>>>>
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>>>
> > > > >>>
> > > > >>
> > > > >
> > > >
> >
> >

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Guozhang,

The KIP as it stands introduces a new configuration,
default.state.isolation.level, which is independent of processing.mode.
It's intended that this new configuration be used to configure a global IQ
isolation level in the short term, with a future KIP introducing the
capability to change the isolation level on a per-query basis, falling back
to the "default" defined by this config. That's why I called it "default",
for future-proofing.

However, it currently includes the caveat that READ_UNCOMMITTED is not
available under EOS. I think this is the coupling you are alluding to?

This isn't intended to be a restriction of the API, but is currently a
technical limitation. However, after discussing with some users about
use-cases that would require READ_UNCOMMITTED under EOS, I'm inclined to
remove that clause and put in the necessary work to make that combination
possible now.

I currently see two possible approaches:

   1. Disable TX StateStores internally when the IsolationLevel is
   READ_UNCOMMITTED and the processing.mode is EOS. This is more difficult
   than it sounds, as there are many assumptions being made throughout the
   internals about the guarantees StateStores provide. It would definitely add
   a lot of extra "if (read_uncommitted && eos)" branches, complicating
   maintenance and testing.
   2. Invest the time *now* to make READ_UNCOMMITTED of EOS StateStores
   possible. I have some ideas on how this could be achieved, but they would
   need testing and could introduce some additional issues. The benefit of
   this approach is that it would make query-time IsolationLevels much simpler
   to implement in the future.

Unfortunately, both will require considerable work that will further delay
this KIP, which was the reason I placed the restriction in the KIP in the
first place.

Regards,
Nick

On Sat, 14 Oct 2023 at 03:30, Guozhang Wang <gu...@gmail.com>
wrote:

> Hello Nick,
>
> First of all, thanks a lot for the great effort you've put in driving
> this KIP! I really like it coming through finally, as many people in
> the community have raised this. At the same time I honestly feel a bit
> ashamed for not putting enough of my time supporting it and pushing it
> through the finish line (you raised this KIP almost a year ago).
>
> I briefly passed through the DISCUSS thread so far, not sure I've 100
> percent digested all the bullet points. But with the goal of trying to
> help take it through the finish line in mind, I'd want to throw
> thoughts on top of my head only on the point #4 above which I felt may
> be the main hurdle for the current KIP to drive to a consensus now.
>
> The general question I asked myself is, whether we want to couple "IQ
> reading mode" with "processing mode". While technically I tend to
> agree with you that, it's feels like a bug if some single user chose
> "EOS" for processing mode while choosing "read uncommitted" for IQ
> reading mode, at the same time, I'm thinking if it's possible that
> there could be two different persons (or even two teams) that would be
> using the stream API to build the app, and the IQ API to query the
> running state of the app. I know this is less of a technical thing but
> rather a more design stuff, but if it could be ever the case, I'm
> wondering if the personale using the IQ API knows about the risks of
> using read uncommitted but still chose so for the favor of
> performance, no matter if the underlying stream processing mode
> configured by another personale is EOS or not. In that regard, I'm
> leaning towards a "leaving the door open, and close it later if we
> found it's a bad idea" aspect with a configuration 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 can still enforce
> coupling them.
>
> Again, just my 2c here. Thanks again for the great patience and
> diligence on this KIP.
>
>
> Guozhang
>
>
>
> On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <ni...@gmail.com>
> wrote:
> >
> > Hi Bruno,
> >
> > 4.
> > I'll hold off on making that change until we have a consensus as to what
> > configuration to use to control all of this, as it'll be affected by the
> > decision on EOS isolation levels.
> >
> > 5.
> > Done. I've chosen "committedOffsets".
> >
> > Regards,
> > Nick
> >
> > On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <ca...@apache.org> wrote:
> >
> > > Hi Nick,
> > >
> > > 1.
> > > Yeah, you are probably right that it does not make too much sense.
> > > Thanks for the clarification!
> > >
> > >
> > > 4.
> > > Yes, sorry for the back and forth, but I think for the sake of the KIP
> > > it is better to let the ALOS behavior as it is for now due to the
> > > possible issues you would run into. Maybe we can find a solution in the
> > > future. Now the question returns to whether we really need
> > > default.state.isolation.level. Maybe the config could be the feature
> > > flag Sophie requested.
> > >
> > >
> > > 5.
> > > There 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 committedOffsetFor().
> > >
> > >
> > > Best,
> > > Bruno
> > >
> > >
> > > On 10/13/23 3:21 PM, Nick Telford wrote:
> > > > Hi Bruno,
> > > >
> > > > Thanks for getting back to me.
> > > >
> > > > 1.
> > > > I think this should be possible. Are you thinking of the situation
> where
> > > a
> > > > user may downgrade to a previous version of Kafka Streams? In that
> case,
> > > > sadly, the RocksDBStore would get wiped by the older version of Kafka
> > > > Streams anyway, because that version wouldn't understand the extra
> column
> > > > family (that holds offsets), so the missing Position file would
> > > > automatically get rebuilt when the store is rebuilt from the
> changelog.
> > > > Are there other situations than downgrade where a transactional store
> > > could
> > > > be replaced by a non-transactional one? I can't think of any.
> > > >
> > > > 2.
> > > > Ahh yes, the Test Plan - my Kryptonite! This section definitely
> needs to
> > > be
> > > > fleshed out. I'll work on that. How much detail do you need?
> > > >
> > > > 3.
> > > > See my previous email discussing this.
> > > >
> > > > 4.
> > > > Hmm, this is an interesting point. Are you suggesting that under ALOS
> > > > READ_COMMITTED should not be supported?
> > > >
> > > > Regards,
> > > > Nick
> > > >
> > > > On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <ca...@apache.org>
> wrote:
> > > >
> > > >> Hi Nick,
> > > >>
> > > >> I think the KIP is converging!
> > > >>
> > > >>
> > > >> 1.
> > > >> I am wondering whether it makes sense to write the position file
> during
> > > >> close as we do for the checkpoint file, so that in case the state
> store
> > > >> is replaced with a non-transactional state store the
> non-transactional
> > > >> state store finds the position file. I think, this is not strictly
> > > >> needed, but would be a nice behavior instead of just deleting the
> > > >> position file.
> > > >>
> > > >>
> > > >> 2.
> > > >> The test plan does not mention integration tests. Do you not need to
> > > >> extend existing ones and add new ones. Also for upgrading and
> > > >> downgrading you might need integration and/or system tests.
> > > >>
> > > >>
> > > >> 3.
> > > >> I think Sophie made a point. Although, IQ reading from uncommitted
> data
> > > >> under EOS might be considered a bug by some people. Thus, your KIP
> would
> > > >> fix a bug rather than changing the intended behavior. However, I
> also
> > > >> see that a feature flag would help users that rely on this buggy
> > > >> behavior (at least until AK 4.0).
> > > >>
> > > >>
> > > >> 4.
> > > >> This is related to the previous point. I assume that the difference
> > > >> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is that in the
> > > >> former you enable transactions on the state store and in the latter
> you
> > > >> disable them. If my assumption is correct, I think that is an issue.
> > > >> Let's assume under ALOS Streams fails over a couple of times more or
> > > >> less at the same step in processing after value 3 is added to an
> > > >> aggregation but the offset of the corresponding input record was not
> > > >> committed. Without transactions disabled, the aggregation value
> would
> > > >> increase by 3 for each failover. With transactions enabled, value 3
> > > >> would only be added to the aggregation once when the offset of the
> input
> > > >> record is committed and the transaction finally completes. So the
> > > >> content of the state store would change depending on the
> configuration
> > > >> for IQ. IMO, the content of the state store should be independent
> from
> > > >> IQ. Given this issue, I propose to not use transactions with ALOS at
> > > >> all. I was a big proponent of using transactions with ALOS, but I
> > > >> realized that transactions with ALOS is not as easy as enabling
> > > >> transactions on state stores. Another aspect that is problematic is
> that
> > > >> the changelog topic which actually replicates the state store is not
> > > >> transactional under ALOS. Thus, it might happen that the state
> store and
> > > >> the changelog differ in their content. All of this is maybe solvable
> > > >> somehow, but for the sake of this KIP, I would leave it for the
> future.
> > > >>
> > > >>
> > > >> Best,
> > > >> Bruno
> > > >>
> > > >>
> > > >>
> > > >> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > > >>> Hey Nick! First of all thanks for taking up this awesome feature,
> I'm
> > > >> sure
> > > >>> every single
> > > >>> Kafka Streams user and dev would agree that it is sorely needed.
> > > >>>
> > > >>> I've just been catching up on the KIP and surrounding discussion,
> so
> > > >> please
> > > >>> forgive me
> > > >>> for any misunderstandings or misinterpretations of the current
> plan and
> > > >>> don't hesitate to
> > > >>> correct me.
> > > >>>
> > > >>> Before I jump in, I just want to say that having seen this drag on
> for
> > > so
> > > >>> long, my singular
> > > >>> goal in responding is to help this KIP past a perceived impasse so
> we
> > > can
> > > >>> finally move on
> > > >>> to voting and implementing it. Long discussions are to be expected
> for
> > > >>> major features like
> > > >>> this but it's completely on us as the Streams devs to make sure
> there
> > > is
> > > >> an
> > > >>> end in sight
> > > >>> for any ongoing discussion.
> > > >>>
> > > >>> With that said, it's my understanding that the KIP as currently
> > > proposed
> > > >> is
> > > >>> just not tenable
> > > >>> for Kafka Streams, and would prevent some EOS users from upgrading
> to
> > > the
> > > >>> version it
> > > >>> first appears in. Given that we can't predict or guarantee whether
> any
> > > of
> > > >>> the followup KIPs
> > > >>> would be completed in the same release cycle as this one, we need
> to
> > > make
> > > >>> sure that the
> > > >>> feature is either compatible with all current users or else
> > > >> feature-flagged
> > > >>> so that they may
> > > >>> opt in/out.
> > > >>>
> > > >>> Therefore, IIUC we need to have either (or both) of these as
> > > >>> fully-implemented config options:
> > > >>> 1. default.state.isolation.level
> > > >>> 2. enable.transactional.state.stores
> > > >>>
> > > >>> This way EOS users for whom read_committed semantics are not
> viable can
> > > >>> still upgrade,
> > > >>> and either use the isolation.level config to leverage the new txn
> state
> > > >>> stores without sacrificing
> > > >>> their application semantics, or else simply keep the transactional
> > > state
> > > >>> stores disabled until we
> > > >>> are able to fully implement the isolation level configuration at
> either
> > > >> an
> > > >>> application or query level.
> > > >>>
> > > >>> Frankly you are the expert here and know much more about the
> tradeoffs
> > > in
> > > >>> both semantics and
> > > >>> effort level of implementing one of these configs vs the other. In
> my
> > > >>> opinion, either option would
> > > >>> be fine and I would leave the decision of which one to include in
> this
> > > >> KIP
> > > >>> completely up to you.
> > > >>> I just don't see a way for the KIP to proceed without some
> variation of
> > > >> the
> > > >>> above that would allow
> > > >>> EOS users to opt-out of read_committed.
> > > >>>
> > > >>> (If it's all the same to you, I would recommend 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 that gets released.
> > > >>> So personally, my choice would be to add the feature flag and
> leave it
> > > >> off
> > > >>> by default. If all 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
> > > >>>
> > > >>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <
> nick.telford@gmail.com>
> > > >> wrote:
> > > >>>
> > > >>>> Hi Bruno,
> > > >>>>
> > > >>>> Agreed, I can live with that for now.
> > > >>>>
> > > >>>> In an effort to keep the scope of this KIP from expanding, I'm
> leaning
> > > >>>> towards just providing a configurable
> default.state.isolation.level
> > > and
> > > >>>> removing IsolationLevel from the StateStoreContext. This would be
> > > >>>> compatible with adding support for query-time IsolationLevels in
> the
> > > >>>> future, whilst providing a way for users to select an isolation
> level
> > > >> now.
> > > >>>>
> > > >>>> The big problem with this, however, is that if a user selects
> > > >>>> processing.mode
> > > >>>> = "exactly-once(-v2|-beta)", and default.state.isolation.level =
> > > >>>> "READ_UNCOMMITTED", we need to guarantee that the data isn't
> written
> > > to
> > > >>>> disk until commit() is called, but we also need to permit IQ
> threads
> > > to
> > > >>>> read from the ongoing transaction.
> > > >>>>
> > > >>>> A simple solution would be to (temporarily) forbid this
> combination of
> > > >>>> configuration, and have default.state.isolation.level
> automatically
> > > >> switch
> > > >>>> to READ_COMMITTED when processing.mode is anything other than
> > > >>>> at-least-once. Do you think this would be acceptable?
> > > >>>>
> > > >>>> In a later KIP, we can add support for query-time isolation
> levels and
> > > >>>> solve this particular problem there, which would relax this
> > > restriction.
> > > >>>>
> > > >>>> Regards,
> > > >>>> Nick
> > > >>>>
> > > >>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <ca...@apache.org>
> > > >> 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,
> > > >>>>> Bruno
> > > >>>>>
> > > >>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > > >>>>>> 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 <
> nick.telford@gmail.com
> > > >
> > > >>>>> wrote:
> > > >>>>>>
> > > >>>>>>> Hi everyone,
> > > >>>>>>>
> > > >>>>>>> I agree that having IsolationLevel be determined at query-time
> is
> > > the
> > > >>>>>>> ideal design, but there are a few sticking points:
> > > >>>>>>>
> > > >>>>>>> 1.
> > > >>>>>>> There needs to be some way to communicate the IsolationLevel
> down
> > > to
> > > >>>> the
> > > >>>>>>> RocksDBStore itself, so that the query can respect it. Since
> stores
> > > >>>> are
> > > >>>>>>> "layered" in functionality (i.e. ChangeLoggingStore,
> MeteredStore,
> > > >>>>> etc.),
> > > >>>>>>> we need some way to deliver that information to the bottom
> layer.
> > > For
> > > >>>>> IQv2,
> > > >>>>>>> we can use the existing State#query() method, but IQv1 has no
> way
> > > to
> > > >>>> do
> > > >>>>>>> this.
> > > >>>>>>>
> > > >>>>>>> A simple approach, which would potentially open up other
> options,
> > > >>>> would
> > > >>>>> be
> > > >>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> > > >>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > > ReadOnlyKeyValueStore
> > > >>>>> (and
> > > >>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
> > > >>>>>>>
> > > >>>>>>> 2.
> > > >>>>>>> As mentioned above, RocksDB WriteBatches are not thread-safe,
> which
> > > >>>>> causes
> > > >>>>>>> a problem if we want to provide READ_UNCOMMITTED Iterators. I
> also
> > > >>>> had a
> > > >>>>>>> look at RocksDB Transactions[1], but they solve a very
> different
> > > >>>>> problem,
> > > >>>>>>> and have the same thread-safety issue.
> > > >>>>>>>
> > > >>>>>>> One possible approach that I mentioned is chaining
> WriteBatches:
> > > >> every
> > > >>>>>>> time a new Interactive Query is received (i.e. readOnlyView,
> see
> > > >>>> above,
> > > >>>>>>> is called) we "freeze" the existing WriteBatch, and start a
> new one
> > > >>>> for
> > > >>>>> new
> > > >>>>>>> writes. The Interactive Query queries the "chain" of previous
> > > >>>>> WriteBatches
> > > >>>>>>> + the underlying database; while the StreamThread starts
> writing to
> > > >>>> the
> > > >>>>>>> *new* WriteBatch. On-commit, the StreamThread would write *all*
> > > >>>>>>> WriteBatches in the chain to the database (that have not yet
> been
> > > >>>>> written).
> > > >>>>>>>
> > > >>>>>>> WriteBatches would be closed/freed only when they have been
> both
> > > >>>>>>> committed, and all open Interactive Queries on them have been
> > > closed.
> > > >>>>> This
> > > >>>>>>> would require some reference counting.
> > > >>>>>>>
> > > >>>>>>> Obviously a drawback of this approach is the potential for
> > > increased
> > > >>>>>>> memory usage: if an Interactive Query is long-lived, for
> example by
> > > >>>>> doing a
> > > >>>>>>> full scan over a large database, or even just pausing in the
> middle
> > > >> of
> > > >>>>> an
> > > >>>>>>> iteration, then the existing chain of WriteBatches could be
> kept
> > > >>>> around
> > > >>>>> for
> > > >>>>>>> a long time, potentially forever.
> > > >>>>>>>
> > > >>>>>>> --
> > > >>>>>>>
> > > >>>>>>> A.
> > > >>>>>>> Going off on a tangent, it looks like in addition to supporting
> > > >>>>>>> READ_COMMITTED queries, we could go further and support
> > > >>>> REPEATABLE_READ
> > > >>>>>>> queries (i.e. where subsequent reads to the same key in the
> same
> > > >>>>>>> Interactive Query are guaranteed to yield the same value) by
> making
> > > >>>> use
> > > >>>>> of
> > > >>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so the
> > > >> performance
> > > >>>>>>> impact is likely to be negligible, but they do require that the
> > > >>>>> Interactive
> > > >>>>>>> Query session can be explicitly closed.
> > > >>>>>>>
> > > >>>>>>> This could be achieved if we made the above readOnlyView
> interface
> > > >>>> look
> > > >>>>>>> more like:
> > > >>>>>>>
> > > >>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> > > >>>> ReadOnlyKeyValueStore<K,
> > > >>>>>>> V>, AutoCloseable {}
> > > >>>>>>>
> > > >>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > > >>>>>>>        ...
> > > >>>>>>>        ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel
> > > >>>>> isolationLevel);
> > > >>>>>>> }
> > > >>>>>>>
> > > >>>>>>> But this would be a breaking change, as existing IQv1 queries
> are
> > > >>>>>>> guaranteed to never call store.close(), and therefore these
> would
> > > >> leak
> > > >>>>>>> memory under REPEATABLE_READ.
> > > >>>>>>>
> > > >>>>>>> B.
> > > >>>>>>> One thing that's notable: MyRocks states that they support
> > > >>>>> READ_COMMITTED
> > > >>>>>>> and REPEATABLE_READ, but they make no mention of
> > > >>>> READ_UNCOMMITTED[3][4].
> > > >>>>>>> This could be because doing so is technically
> difficult/impossible
> > > >>>> using
> > > >>>>>>> the primitives available in RocksDB.
> > > >>>>>>>
> > > >>>>>>> --
> > > >>>>>>>
> > > >>>>>>> Lucas, to address your points:
> > > >>>>>>>
> > > >>>>>>> U1.
> > > >>>>>>> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
> > > >>>>>>> implementations of StateStore that do not support atomic
> writes.
> > > >>>>> Obviously
> > > >>>>>>> in those cases, the guarantees Kafka Streams provides/expects
> would
> > > >> be
> > > >>>>>>> relaxed. Do you think we should require all implementations to
> > > >> support
> > > >>>>>>> atomic writes?
> > > >>>>>>>
> > > >>>>>>> U2.
> > > >>>>>>> Stores can support multiple IsolationLevels. As we've discussed
> > > >> above,
> > > >>>>> the
> > > >>>>>>> ideal scenario would be to specify the IsolationLevel at
> > > query-time.
> > > >>>>>>> Failing that, I think the second-best approach is to define the
> > > >>>>>>> IsolationLevel for *all* queries based on the processing.mode,
> > > which
> > > >>>> is
> > > >>>>>>> what the default StateStoreContext#isolationLevel() achieves.
> Would
> > > >>>> you
> > > >>>>>>> prefer an alternative?
> > > >>>>>>>
> > > >>>>>>> While the existing implementation is equivalent to
> > > READ_UNCOMMITTED,
> > > >>>>> this
> > > >>>>>>> can yield unexpected results/errors under EOS, if a
> transaction is
> > > >>>>> rolled
> > > >>>>>>> back. While this would be a change in behaviour for users, it
> would
> > > >>>> look
> > > >>>>>>> more like a bug fix than a breaking change. That said, we
> *could*
> > > >> make
> > > >>>>> it
> > > >>>>>>> configurable, and default to the existing behaviour
> > > >> (READ_UNCOMMITTED)
> > > >>>>>>> instead of inferring it from the processing.mode?
> > > >>>>>>>
> > > >>>>>>> N1, N2.
> > > >>>>>>> These were only primitives to avoid boxing costs, but since
> this is
> > > >>>> not
> > > >>>>> a
> > > >>>>>>> performance sensitive area, it should be fine to change if
> that's
> > > >>>>> desirable.
> > > >>>>>>>
> > > >>>>>>> N3.
> > > >>>>>>> It's because the store "manages its own offsets", which
> includes
> > > both
> > > >>>>>>> committing the offset, *and providing it* via
> getCommittedOffset().
> > > >>>>>>> Personally, I think "managesOffsets" conveys this best, but I
> don't
> > > >>>> mind
> > > >>>>>>> changing it if the nomenclature is unclear.
> > > >>>>>>>
> > > >>>>>>> Sorry for the massive emails/essays!
> > > >>>>>>> --
> > > >>>>>>> Nick
> > > >>>>>>>
> > > >>>>>>> 1: https://github.com/facebook/rocksdb/wiki/Transactions
> > > >>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > > >>>>>>> 3:
> > > https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > >>>>>>> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > >>>>>>>
> > > >>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > > >>>>>>> <lb...@confluent.io.invalid> wrote:
> > > >>>>>>>
> > > >>>>>>>> Hi Nick,
> > > >>>>>>>>
> > > >>>>>>>> since I last read it in April, the KIP has become much
> cleaner and
> > > >>>>>>>> easier to read. Great work!
> > > >>>>>>>>
> > > >>>>>>>> It feels to me the last big open point is whether we can
> implement
> > > >>>>>>>> isolation level as a query parameter. I understand that there
> are
> > > >>>>>>>> implementation concerns, but as Colt says, it would be a great
> > > >>>>>>>> addition, and would also simplify the migration path for this
> > > >> change.
> > > >>>>>>>> Is the implementation problem you mentioned caused by the
> > > WriteBatch
> > > >>>>>>>> not having a notion of a snapshot, as the underlying DB
> iterator
> > > >>>> does?
> > > >>>>>>>> In that case, I am not sure a chain of WriteBatches as you
> propose
> > > >>>>>>>> would fully solve the problem, but maybe I didn't dig enough
> into
> > > >> the
> > > >>>>>>>> details to fully understand it.
> > > >>>>>>>>
> > > >>>>>>>> If it's not possible to implement it now, would it be an
> option to
> > > >>>>>>>> make sure in this KIP that we do not fully close the door on
> > > >>>> per-query
> > > >>>>>>>> isolation levels in the interface, as it may be possible to
> > > >> implement
> > > >>>>>>>> the missing primitives in RocksDB or Speedb in the future.
> > > >>>>>>>>
> > > >>>>>>>> Understanding:
> > > >>>>>>>>
> > > >>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to be
> persisted
> > > >>>>>>>> atomically with the records?
> > > >>>>>>>> * U2) Don't understand the default implementation of
> > > >>>> `isolationLevel`.
> > > >>>>>>>> The isolation level should be a property of the underlying
> store,
> > > >> and
> > > >>>>>>>> not be defined by the default config? Existing stores probably
> > > don't
> > > >>>>>>>> guarantee READ_COMMITTED, so the default should be to return
> > > >>>>>>>> READ_UNCOMMITTED.
> > > >>>>>>>>
> > > >>>>>>>> Nits:
> > > >>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong` return
> type,
> > > >> to
> > > >>>>>>>> avoid the `null`?
> > > >>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an
> `OptionalLong`
> > > >>>>>>>> return type, to avoid the `-1`?
> > > >>>>>>>> * N3) I don't understand why `managesOffsets` uses the
> 'manage'
> > > >> verb,
> > > >>>>>>>> whereas all other methods use the "commits" verb. I'd suggest
> > > >>>>>>>> `commitsOffsets`.
> > > >>>>>>>>
> > > >>>>>>>> Either way, it feels this KIP is very close to the finish
> line,
> > > I'm
> > > >>>>>>>> looking forward to seeing this in production!
> > > >>>>>>>>
> > > >>>>>>>> Cheers,
> > > >>>>>>>> Lucas
> > > >>>>>>>>
> > > >>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <
> colt@littlehorse.io
> > > >
> > > >>>>> wrote:
> > > >>>>>>>>>
> > > >>>>>>>>>> Making IsolationLevel a query-time constraint, rather than
> > > linking
> > > >>>> it
> > > >>>>>>>> to
> > > >>>>>>>>> the processing.guarantee.
> > > >>>>>>>>>
> > > >>>>>>>>> As I understand it, would this allow even a user of EOS to
> > > control
> > > >>>>>>>> whether
> > > >>>>>>>>> reading committed or uncommitted records? If so, I am highly
> in
> > > >>>> favor
> > > >>>>> of
> > > >>>>>>>>> this.
> > > >>>>>>>>>
> > > >>>>>>>>> I know that I was one of the early people to point out the
> > > current
> > > >>>>>>>>> shortcoming that IQ reads uncommitted records, but just this
> > > >>>> morning I
> > > >>>>>>>>> realized a pattern we use which means that (for certain
> queries)
> > > >> our
> > > >>>>>>>> system
> > > >>>>>>>>> needs to be able to read uncommitted records, which is the
> > > current
> > > >>>>>>>> behavior
> > > >>>>>>>>> of Kafka Streams in EOS.***
> > > >>>>>>>>>
> > > >>>>>>>>> If IsolationLevel being a query-time decision allows for
> this,
> > > then
> > > >>>>> that
> > > >>>>>>>>> would be amazing. I would also vote that the default behavior
> > > >> should
> > > >>>>> be
> > > >>>>>>>> for
> > > >>>>>>>>> reading uncommitted records, because it is totally possible
> for a
> > > >>>>> valid
> > > >>>>>>>>> application to depend on that behavior, and breaking it in a
> > > minor
> > > >>>>>>>> release
> > > >>>>>>>>> might be a bit strong.
> > > >>>>>>>>>
> > > >>>>>>>>> *** (Note, for the curious reader....) Our use-case/query
> pattern
> > > >>>> is a
> > > >>>>>>>> bit
> > > >>>>>>>>> complex, but reading "uncommitted" records is actually safe
> in
> > > our
> > > >>>>> case
> > > >>>>>>>>> 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 soon as the processor processes the
> result.
> > > If
> > > >>>>> they
> > > >>>>>>>>> can't read uncommitted records, they may receive a "201 -
> > > Created"
> > > >>>>>>>>> response, immediately followed by a "404 - 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 <
> > > >>>> nick.telford@gmail.com>
> > > >>>>>>>> wrote:
> > > >>>>>>>>>
> > > >>>>>>>>>> Addendum:
> > > >>>>>>>>>>
> > > >>>>>>>>>> I think we would also face the same problem with the
> approach
> > > John
> > > >>>>>>>> outlined
> > > >>>>>>>>>> earlier (using the record cache as a transaction buffer and
> > > >>>> flushing
> > > >>>>>>>> it
> > > >>>>>>>>>> straight to SST files). This is because the record cache
> (the
> > > >>>>>>>> ThreadCache
> > > >>>>>>>>>> class) is not thread-safe, so every commit would invalidate
> open
> > > >> IQ
> > > >>>>>>>>>> Iterators in the same way that RocksDB WriteBatches do.
> > > >>>>>>>>>> --
> > > >>>>>>>>>> Nick
> > > >>>>>>>>>>
> > > >>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > > >>>> nick.telford@gmail.com>
> > > >>>>>>>>>> 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.
> > > >>>>>>>>>>> 2. Making IsolationLevel a query-time constraint, rather
> than
> > > >>>>>>>> linking it
> > > >>>>>>>>>>> to the processing.guarantee.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> There's a wrinkle that makes this a challenge: Interactive
> > > >> Queries
> > > >>>>>>>> that
> > > >>>>>>>>>>> open an Iterator, when using transactions and
> READ_UNCOMMITTED.
> > > >>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries need
> to be
> > > >>>> able
> > > >>>>>>>> to
> > > >>>>>>>>>>> read records from the currently uncommitted transaction
> buffer
> > > >>>>>>>>>>> (WriteBatch). This includes for Iterators, which should
> iterate
> > > >>>>>>>> both the
> > > >>>>>>>>>>> transaction buffer and underlying database (using
> > > >>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> The issue is that when the StreamThread commits, it writes
> the
> > > >>>>>>>> current
> > > >>>>>>>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*.
> > > Clearing
> > > >>>> the
> > > >>>>>>>>>>> WriteBatch while an Interactive Query holds an open
> Iterator on
> > > >> it
> > > >>>>>>>> will
> > > >>>>>>>>>>> invalidate the Iterator. Worse, it turns out that Iterators
> > > over
> > > >> a
> > > >>>>>>>>>>> WriteBatch become invalidated not just when the WriteBatch
> is
> > > >>>>>>>> cleared,
> > > >>>>>>>>>> but
> > > >>>>>>>>>>> also when the Iterators' current key receives a new write.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Now that I'm writing this, I remember that this is the
> major
> > > >>>> reason
> > > >>>>>>>> that
> > > >>>>>>>>>> I
> > > >>>>>>>>>>> switched the original design from having a query-time
> > > >>>>>>>> IsolationLevel to
> > > >>>>>>>>>>> having the IsolationLevel linked to the transactionality
> of the
> > > >>>>>>>> stores
> > > >>>>>>>>>>> themselves.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> It *might* be possible to resolve this, by having a
> "chain" of
> > > >>>>>>>>>>> WriteBatches, with the StreamThread switching to a new
> > > WriteBatch
> > > >>>>>>>>>> whenever
> > > >>>>>>>>>>> a new Interactive Query attempts to read from the
> database, but
> > > >>>> that
> > > >>>>>>>>>> could
> > > >>>>>>>>>>> cause some performance problems/memory pressure when
> subjected
> > > to
> > > >>>> a
> > > >>>>>>>> high
> > > >>>>>>>>>>> Interactive Query load. It would also reduce the
> efficiency of
> > > >>>>>>>>>> WriteBatches
> > > >>>>>>>>>>> on-commit, as we'd have to write N WriteBatches, where N
> is the
> > > >>>>>>>> number of
> > > >>>>>>>>>>> Interactive Queries since the last commit.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> I realise this is getting into the weeds of the
> implementation,
> > > >>>> and
> > > >>>>>>>> you'd
> > > >>>>>>>>>>> rather we focus on the API for now, but I think it's
> important
> > > to
> > > >>>>>>>>>> consider
> > > >>>>>>>>>>> how to implement the desired API, in case we come up with
> an
> > > API
> > > >>>>>>>> that
> > > >>>>>>>>>>> cannot be implemented efficiently, or even at all!
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Thoughts?
> > > >>>>>>>>>>> --
> > > >>>>>>>>>>> Nick
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> > > cadonna@apache.org
> > > >>>
> > > >>>>>>>> wrote:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> 6.
> > > >>>>>>>>>>>> Of course, you are right! My bad!
> > > >>>>>>>>>>>> Wiping out the state in the downgrading case is fine.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> 3a.
> > > >>>>>>>>>>>> Focus on the public facing changes for the KIP. We will
> manage
> > > >> 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.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Best,
> > > >>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > > >>>>>>>>>>>>> Hi Bruno,
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Thanks for getting back to me!
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>> The fact that implementations can always track estimated
> > > memory
> > > >>>>>>>> usage
> > > >>>>>>>>>> in
> > > >>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as an
> option,
> > > and
> > > >>>>>>>> I'll
> > > >>>>>>>>>>>> clarify
> > > >>>>>>>>>>>>> the JavaDoc that 0 is not just for non-transactional
> stores,
> > > >>>>>>>> which is
> > > >>>>>>>>>>>>> currently misleading.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> 6.
> > > >>>>>>>>>>>>> The problem with catching the exception in the downgrade
> > > >> process
> > > >>>>>>>> is
> > > >>>>>>>>>> that
> > > >>>>>>>>>>>>> would require new code in the Kafka version being
> downgraded
> > > >> to.
> > > >>>>>>>> Since
> > > >>>>>>>>>>>>> users could conceivably downgrade to almost *any* older
> > > version
> > > >>>>>>>> of
> > > >>>>>>>>>> Kafka
> > > >>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
> > > >>>>>>>>>>>>> The only way I can think of doing it would be to provide
> a
> > > >>>>>>>> dedicated
> > > >>>>>>>>>>>>> downgrade tool, that goes through every local store and
> > > removes
> > > >>>>>>>> the
> > > >>>>>>>>>>>>> offsets column families. But that seems like an
> unnecessary
> > > >>>>>>>> amount of
> > > >>>>>>>>>>>> extra
> > > >>>>>>>>>>>>> code to maintain just to handle a somewhat niche
> situation,
> > > >> when
> > > >>>>>>>> the
> > > >>>>>>>>>>>>> alternative (automatically wipe and restore stores)
> should be
> > > >>>>>>>>>>>> acceptable.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've requested.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> 3a.
> > > >>>>>>>>>>>>> I agree that IsolationLevel makes more sense at
> query-time,
> > > and
> > > >>>> I
> > > >>>>>>>>>>>> actually
> > > >>>>>>>>>>>>> initially attempted to place the IsolationLevel at
> > > query-time,
> > > >>>>>>>> but I
> > > >>>>>>>>>> ran
> > > >>>>>>>>>>>>> into some problems:
> > > >>>>>>>>>>>>> - The key issue is that, under ALOS we're not staging
> writes
> > > in
> > > >>>>>>>>>>>>> transactions, so can't perform writes at the
> READ_COMMITTED
> > > >>>>>>>> isolation
> > > >>>>>>>>>>>>> level. However, this may be addressed if we decide to
> > > *always*
> > > >>>>>>>> use
> > > >>>>>>>>>>>>> transactions as discussed under 3b.
> > > >>>>>>>>>>>>> - IQv1 and IQv2 have quite different implementations. I
> > > >> remember
> > > >>>>>>>>>> having
> > > >>>>>>>>>>>>> some difficulty understanding the IQv1 internals, which
> made
> > > it
> > > >>>>>>>>>>>> difficult
> > > >>>>>>>>>>>>> to determine what needed to be changed. However, I
> *think*
> > > this
> > > >>>>>>>> can be
> > > >>>>>>>>>>>>> addressed for both implementations by wrapping the
> > > RocksDBStore
> > > >>>>>>>> in an
> > > >>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides read
> methods
> > > >>>>>>>> (get,
> > > >>>>>>>>>>>> etc.)
> > > >>>>>>>>>>>>> to either read directly from the database or from the
> ongoing
> > > >>>>>>>>>>>> transaction.
> > > >>>>>>>>>>>>> But IQv1 might still be difficult.
> > > >>>>>>>>>>>>> - If IsolationLevel becomes a query constraint, then all
> > > other
> > > >>>>>>>>>>>> StateStores
> > > >>>>>>>>>>>>> will need to respect it, including the in-memory stores.
> This
> > > >>>>>>>> would
> > > >>>>>>>>>>>> require
> > > >>>>>>>>>>>>> us to adapt in-memory stores to stage their writes so
> they
> > > can
> > > >>>> be
> > > >>>>>>>>>>>> isolated
> > > >>>>>>>>>>>>> from READ_COMMITTTED queries. It would also become an
> > > important
> > > >>>>>>>>>>>>> consideration for third-party stores on upgrade, as
> without
> > > >>>>>>>> changes,
> > > >>>>>>>>>>>> they
> > > >>>>>>>>>>>>> would not support READ_COMMITTED queries correctly.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Ultimately, I may need some help making the necessary
> change
> > > to
> > > >>>>>>>> IQv1
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>> support this, but I don't think it's fundamentally
> > > impossible,
> > > >>>>>>>> if we
> > > >>>>>>>>>>>> want
> > > >>>>>>>>>>>>> to pursue this route.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> 3b.
> > > >>>>>>>>>>>>> The main reason I chose to keep ALOS un-transactional
> was to
> > > >>>>>>>> minimize
> > > >>>>>>>>>>>>> behavioural change for most users (I believe most Streams
> > > users
> > > >>>>>>>> use
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>> default configuration, which is ALOS). That said, it's
> clear
> > > >>>>>>>> that if
> > > >>>>>>>>>>>> ALOS
> > > >>>>>>>>>>>>> also used transactional stores, the only change in
> behaviour
> > > >>>>>>>> would be
> > > >>>>>>>>>>>> that
> > > >>>>>>>>>>>>> it would become *more correct*, which could be
> considered a
> > > >> "bug
> > > >>>>>>>> fix"
> > > >>>>>>>>>> by
> > > >>>>>>>>>>>>> users, rather than a change they need to handle.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> I believe that performance using transactions (aka.
> RocksDB
> > > >>>>>>>>>>>> WriteBatches)
> > > >>>>>>>>>>>>> should actually be *better* than the un-batched
> write-path
> > > that
> > > >>>>>>>> is
> > > >>>>>>>>>>>>> currently used[1]. The only "performance" consideration
> will
> > > be
> > > >>>>>>>> the
> > > >>>>>>>>>>>>> increased memory usage that transactions require. Given
> the
> > > >>>>>>>>>> mitigations
> > > >>>>>>>>>>>> for
> > > >>>>>>>>>>>>> this memory that we have in place, I would expect that
> this
> > > is
> > > >>>>>>>> not a
> > > >>>>>>>>>>>>> problem for most users.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> If we're happy to do so, we can make ALOS also use
> > > >> transactions.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Link 1:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>
> > > >>>>>
> > > >>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> > > >>>> cadonna@apache.org
> > > >>>>>>>>>
> > > >>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Thanks for the updates and sorry for the delay on my
> side!
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>> Making the default implementation for flush() a no-op
> sounds
> > > >>>>>>>> good to
> > > >>>>>>>>>>>> me.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>> I think what was bugging me here is that a third-party
> state
> > > >>>>>>>> store
> > > >>>>>>>>>>>> needs
> > > >>>>>>>>>>>>>> to implement the state store interface. That means they
> need
> > > >> to
> > > >>>>>>>>>>>>>> implement a wrapper around the actual state store as we
> do
> > > for
> > > >>>>>>>>>> RocksDB
> > > >>>>>>>>>>>>>> with RocksDBStore. So, a third-party state store can
> always
> > > >>>>>>>> estimate
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>> uncommitted bytes, if it wants, because the wrapper can
> > > record
> > > >>>>>>>> the
> > > >>>>>>>>>>>> added
> > > >>>>>>>>>>>>>> bytes.
> > > >>>>>>>>>>>>>> One case I can think of where returning -1 makes sense
> is
> > > when
> > > >>>>>>>>>> Streams
> > > >>>>>>>>>>>>>> does not need to estimate the size of the write batch
> and
> > > >>>>>>>> trigger
> > > >>>>>>>>>>>>>> extraordinary commits, because the third-party state
> store
> > > >>>>>>>> takes care
> > > >>>>>>>>>>>> of
> > > >>>>>>>>>>>>>> memory. But in that case the method could also just
> return
> > > 0.
> > > >>>>>>>> Even
> > > >>>>>>>>>> that
> > > >>>>>>>>>>>>>> case would be better solved with a method that returns
> > > whether
> > > >>>>>>>> the
> > > >>>>>>>>>>>> state
> > > >>>>>>>>>>>>>> store manages itself the memory used for uncommitted
> bytes
> > > or
> > > >>>>>>>> not.
> > > >>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return value,
> I was
> > > >>>>>>>> just
> > > >>>>>>>>>>>>>> wondering when and if it will be used.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Regarding returning 0 for transactional state stores
> when
> > > the
> > > >>>>>>>> batch
> > > >>>>>>>>>> is
> > > >>>>>>>>>>>>>> empty, I was just wondering because you explicitly
> stated
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> "or {@code 0} if this StateStore does not support
> > > >>>> transactions."
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> So it seemed to me returning 0 could only happen for
> > > >>>>>>>>>> non-transactional
> > > >>>>>>>>>>>>>> state stores.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> a) What do you think if we move the isolation level to
> IQ
> > > (v1
> > > >>>>>>>> and
> > > >>>>>>>>>> v2)?
> > > >>>>>>>>>>>>>> In the end this is the only component that really needs
> to
> > > >>>>>>>> specify
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>> isolation level. It is similar to the Kafka consumer
> that
> > > can
> > > >>>>>>>> choose
> > > >>>>>>>>>>>>>> with what isolation level to read the input topic.
> > > >>>>>>>>>>>>>> For IQv1 the isolation level should go into
> > > >>>>>>>> StoreQueryParameters. For
> > > >>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> b) Point a) raises the question what should happen
> during
> > > >>>>>>>>>> at-least-once
> > > >>>>>>>>>>>>>> processing when the state store does not use
> transactions?
> > > >> John
> > > >>>>>>>> in
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>> past proposed to also use transactions on state stores
> for
> > > >>>>>>>>>>>>>> at-least-once. I like that idea, because it avoids
> > > aggregating
> > > >>>>>>>> the
> > > >>>>>>>>>> same
> > > >>>>>>>>>>>>>> records over and over again in the case of a failure. We
> > > had a
> > > >>>>>>>> case
> > > >>>>>>>>>> in
> > > >>>>>>>>>>>>>> the past where a Streams applications in at-least-once
> mode
> > > >> was
> > > >>>>>>>>>> failing
> > > >>>>>>>>>>>>>> continuously for some reasons I do not remember before
> > > >>>>>>>> committing the
> > > >>>>>>>>>>>>>> offsets. After each failover, the app aggregated again
> and
> > > >>>>>>>> again the
> > > >>>>>>>>>>>>>> same records. Of course the aggregate increased to very
> > > wrong
> > > >>>>>>>> values
> > > >>>>>>>>>>>>>> just because of the failover. With transactions on the
> state
> > > >>>>>>>> stores
> > > >>>>>>>>>> we
> > > >>>>>>>>>>>>>> could have avoided this. The app would have output the
> same
> > > >>>>>>>> aggregate
> > > >>>>>>>>>>>>>> multiple times (i.e., after each failover) but at least
> the
> > > >>>>>>>> value of
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>> aggregate would not depend on the number of failovers.
> > > >>>>>>>> Outputting the
> > > >>>>>>>>>>>>>> same aggregate multiple times would be incorrect under
> > > >>>>>>>> exactly-once
> > > >>>>>>>>>> but
> > > >>>>>>>>>>>>>> it is OK for at-least-once.
> > > >>>>>>>>>>>>>> If it makes sense to add a config to turn on and off
> > > >>>>>>>> transactions on
> > > >>>>>>>>>>>>>> state stores under at-least-once or just use
> transactions in
> > > >>>>>>>> any case
> > > >>>>>>>>>>>> is
> > > >>>>>>>>>>>>>> a question we should also discuss in this KIP. It
> depends a
> > > >> bit
> > > >>>>>>>> on
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would add a
> > > config.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> 4.
> > > >>>>>>>>>>>>>> Your points are all valid. I tend to say to keep the
> metrics
> > > >>>>>>>> around
> > > >>>>>>>>>>>>>> flush() until we remove flush() completely from the
> > > interface.
> > > >>>>>>>> Calls
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>> flush() might still exist since existing processors
> might
> > > >> still
> > > >>>>>>>> call
> > > >>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For sure, we
> need
> > > >> to
> > > >>>>>>>>>>>> document
> > > >>>>>>>>>>>>>> how the metrics change due to the transactions in the
> > > upgrade
> > > >>>>>>>> notes.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> 5.
> > > >>>>>>>>>>>>>> I see. Then you should describe how the .position files
> are
> > > >>>>>>>> handled
> > > >>>>>>>>>> in
> > > >>>>>>>>>>>>>> a dedicated section of the KIP or incorporate the
> > > description
> > > >>>>>>>> in the
> > > >>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only
> mentioning it
> > > >> in
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>> "Compatibility, 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:
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>
> > > >>>>>
> > > >>>>
> > > >>
> > >
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> It is a bit brittle, but it works.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > > >>>>>>>>>>>>>>> Hi Bruno,
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Thanks for taking the time to review the KIP. I'm back
> from
> > > >>>>>>>> leave
> > > >>>>>>>>>> now
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>> intend to move this forwards as quickly as I can.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Addressing your points:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>> Because flush() is part of the StateStore API, it's
> exposed
> > > >> to
> > > >>>>>>>>>> custom
> > > >>>>>>>>>>>>>>> Processors, which might be making calls to flush().
> This
> > > was
> > > >>>>>>>>>> actually
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>> case in a few integration tests.
> > > >>>>>>>>>>>>>>> To maintain as much compatibility as possible, I'd
> prefer
> > > not
> > > >>>>>>>> to
> > > >>>>>>>>>> make
> > > >>>>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>> an UnsupportedOperationException, as it will cause
> > > previously
> > > >>>>>>>>>> working
> > > >>>>>>>>>>>>>>> Processors to start throwing exceptions at runtime.
> > > >>>>>>>>>>>>>>> I agree that it doesn't make sense for it to proxy
> > > commit(),
> > > >>>>>>>> though,
> > > >>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>> that would cause it to violate the "StateStores commit
> only
> > > >>>>>>>> when the
> > > >>>>>>>>>>>> Task
> > > >>>>>>>>>>>>>>> commits" rule.
> > > >>>>>>>>>>>>>>> Instead, I think we should make this a no-op. That way,
> > > >>>>>>>> existing
> > > >>>>>>>>>> user
> > > >>>>>>>>>>>>>>> Processors will continue to work as-before, without
> > > violation
> > > >>>>>>>> of
> > > >>>>>>>>>> store
> > > >>>>>>>>>>>>>>> consistency that would be caused by premature
> flush/commit
> > > of
> > > >>>>>>>>>>>> StateStore
> > > >>>>>>>>>>>>>>> data to disk.
> > > >>>>>>>>>>>>>>> What do you think?
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore
> implementation
> > > is
> > > >>>>>>>>>>>>>>> transactional, but is unable to estimate the
> uncommitted
> > > >>>> memory
> > > >>>>>>>>>> usage,
> > > >>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>> method will return -1.
> > > >>>>>>>>>>>>>>> The intention here is to permit third-party
> implementations
> > > >>>>>>>> that may
> > > >>>>>>>>>>>> not
> > > >>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>> able to estimate memory usage.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been written to the
> > > store
> > > >>>>>>>> yet. I
> > > >>>>>>>>>>>>>> thought
> > > >>>>>>>>>>>>>>> that was implied by "This method will return an
> > > approximation
> > > >>>>>>>> of the
> > > >>>>>>>>>>>>>> memory
> > > >>>>>>>>>>>>>>> would be freed by the next call to {@link
> #commit(Map)}"
> > > and
> > > >>>>>>>>>> "@return
> > > >>>>>>>>>>>> The
> > > >>>>>>>>>>>>>>> approximate size of all records awaiting {@link
> > > >>>> #commit(Map)}",
> > > >>>>>>>>>>>> however,
> > > >>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you think this
> is
> > > >>>>>>>> unclear?
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>> I realise this is probably the most contentious point
> in my
> > > >>>>>>>> design,
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>> I'm
> > > >>>>>>>>>>>>>>> open to changing it if I'm unable to convince you of
> the
> > > >>>>>>>> benefits.
> > > >>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > > >>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly provided
> > > >>>>>>>> StateStores
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>> query, and it may be important for users to
> > > programmatically
> > > >>>>>>>> know
> > > >>>>>>>>>>>> which
> > > >>>>>>>>>>>>>>> mode the StateStore is operating under. If we simply
> > > provide
> > > >>>> an
> > > >>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the internal
> > > streams
> > > >>>>>>>>>>>> engine), or
> > > >>>>>>>>>>>>>>> similar, then users will need to understand the
> operation
> > > and
> > > >>>>>>>>>>>>>> consequences
> > > >>>>>>>>>>>>>>> of each available processing mode and how it pertains
> to
> > > >> their
> > > >>>>>>>>>>>>>> StateStore.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Interactive Query users aren't the only people that
> care
> > > >> about
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
> > > implementers
> > > >>>> of
> > > >>>>>>>>>> custom
> > > >>>>>>>>>>>>>>> StateStores also need to understand the behaviour
> expected
> > > of
> > > >>>>>>>> their
> > > >>>>>>>>>>>>>>> implementation. KIP-892 introduces some assumptions
> into
> > > the
> > > >>>>>>>> Streams
> > > >>>>>>>>>>>>>> Engine
> > > >>>>>>>>>>>>>>> about how StateStores operate under each processing
> mode,
> > > and
> > > >>>>>>>> it's
> > > >>>>>>>>>>>>>>> important that custom implementations adhere to those
> > > >>>>>>>> assumptions in
> > > >>>>>>>>>>>>>> order
> > > >>>>>>>>>>>>>>> to maintain the consistency guarantees.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> IsolationLevels provide a high-level contract on the
> > > >> behaviour
> > > >>>>>>>> of
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>> StateStore: a user knows that under READ_COMMITTED,
> they
> > > will
> > > >>>>>>>> see
> > > >>>>>>>>>>>> writes
> > > >>>>>>>>>>>>>>> only after the Task has committed, and under
> > > READ_UNCOMMITTED
> > > >>>>>>>> they
> > > >>>>>>>>>>>> will
> > > >>>>>>>>>>>>>> see
> > > >>>>>>>>>>>>>>> writes immediately. No understanding of the details of
> each
> > > >>>>>>>>>>>>>> processing.mode
> > > >>>>>>>>>>>>>>> is required, either for IQ users or StateStore
> > > implementers.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> An argument can be made that these contractual
> guarantees
> > > can
> > > >>>>>>>> simply
> > > >>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>> documented for the processing.mode (i.e. that
> exactly-once
> > > >> and
> > > >>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
> > > at-least-once
> > > >>>>>>>> behaves
> > > >>>>>>>>>>>> like
> > > >>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small issues
> with
> > > >>>>>>>> this I'd
> > > >>>>>>>>>>>>>> prefer
> > > >>>>>>>>>>>>>>> to avoid:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>         - Where would we document these contracts, in
> a way
> > > >> that
> > > >>>>>>>> is
> > > >>>>>>>>>>>> difficult
> > > >>>>>>>>>>>>>>>         for users/implementers to miss/ignore?
> > > >>>>>>>>>>>>>>>         - It's not clear to users that the processing
> mode
> > > is
> > > >>>>>>>>>>>> communicating
> > > >>>>>>>>>>>>>>>         an expectation of read isolation, unless they
> read
> > > the
> > > >>>>>>>>>>>>>> documentation. Users
> > > >>>>>>>>>>>>>>>         rarely consult documentation unless they feel
> they
> > > >> need
> > > >>>>>>>> to, so
> > > >>>>>>>>>>>> it's
> > > >>>>>>>>>>>>>> likely
> > > >>>>>>>>>>>>>>>         this detail would get missed by many users.
> > > >>>>>>>>>>>>>>>         - It tightly couples processing modes to read
> > > >> isolation.
> > > >>>>>>>> Adding
> > > >>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>         processing modes, or changing the read
> isolation of
> > > >>>>>>>> existing
> > > >>>>>>>>>>>>>> processing
> > > >>>>>>>>>>>>>>>         modes would be difficult/impossible.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Ultimately, the cost of introducing IsolationLevels is
> > > just a
> > > >>>>>>>> single
> > > >>>>>>>>>>>>>>> method, since we re-use the existing IsolationLevel
> enum
> > > from
> > > >>>>>>>> Kafka.
> > > >>>>>>>>>>>> This
> > > >>>>>>>>>>>>>>> gives us a clear place to document the contractual
> > > guarantees
> > > >>>>>>>>>> expected
> > > >>>>>>>>>>>>>>> of/provided by StateStores, that is accessible both by
> the
> > > >>>>>>>>>> StateStore
> > > >>>>>>>>>>>>>>> itself, and by IQ users.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> (Writing this I've just realised that the StateStore
> and IQ
> > > >>>>>>>> APIs
> > > >>>>>>>>>>>> actually
> > > >>>>>>>>>>>>>>> don't provide access to StateStoreContext that IQ users
> > > would
> > > >>>>>>>> have
> > > >>>>>>>>>>>> direct
> > > >>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> > > >> isolationLevel()
> > > >>>>>>>>>> itself
> > > >>>>>>>>>>>>>> too?)
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> 4.
> > > >>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics in-place
> > > >>>>>>>> either, as
> > > >>>>>>>>>>>> it's a
> > > >>>>>>>>>>>>>>> backwards incompatible change. My concern is that, if
> we
> > > >> leave
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>> existing
> > > >>>>>>>>>>>>>>> "flush" metrics in place, they will be confusing to
> users.
> > > >>>>>>>> Right
> > > >>>>>>>>>> now,
> > > >>>>>>>>>>>>>>> "flush" metrics record explicit flushes to disk, but
> under
> > > >>>>>>>> KIP-892,
> > > >>>>>>>>>>>> even
> > > >>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>> commit() will not explicitly flush data to disk -
> RocksDB
> > > >> will
> > > >>>>>>>>>> decide
> > > >>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>> when to flush memtables to disk itself.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd have two
> > > >>>> options,
> > > >>>>>>>>>> which
> > > >>>>>>>>>>>>>> both
> > > >>>>>>>>>>>>>>> seem pretty bad to me:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>         1. Have them record calls to commit(), which
> would
> > > be
> > > >>>>>>>>>>>> misleading, as
> > > >>>>>>>>>>>>>>>         data is no longer explicitly "flushed" to disk
> by
> > > this
> > > >>>>>>>> call.
> > > >>>>>>>>>>>>>>>         2. Have them record nothing at all, which is
> > > >> equivalent
> > > >>>> to
> > > >>>>>>>>>>>> removing
> > > >>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>         metrics, except that users will see the metric
> > > still
> > > >>>>>>>> exists and
> > > >>>>>>>>>>>> so
> > > >>>>>>>>>>>>>> assume
> > > >>>>>>>>>>>>>>>         that the metric is correct, and that there's a
> > > problem
> > > >>>>>>>> with
> > > >>>>>>>>>> their
> > > >>>>>>>>>>>>>> system
> > > >>>>>>>>>>>>>>>         when there isn't.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> I agree that removing them is also a bad solution, and
> I'd
> > > >>>>>>>> like some
> > > >>>>>>>>>>>>>>> guidance on the best path forward here.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> 5.
> > > >>>>>>>>>>>>>>> Position files are updated on every write to a
> StateStore.
> > > >>>>>>>> Since our
> > > >>>>>>>>>>>>>> writes
> > > >>>>>>>>>>>>>>> are now buffered until commit(), we can't update the
> > > Position
> > > >>>>>>>> file
> > > >>>>>>>>>>>> until
> > > >>>>>>>>>>>>>>> commit() has been called, otherwise it would be
> > > inconsistent
> > > >>>>>>>> with
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>> data
> > > >>>>>>>>>>>>>>> in the event of a rollback. Consequently, we need to
> manage
> > > >>>>>>>> these
> > > >>>>>>>>>>>> offsets
> > > >>>>>>>>>>>>>>> the same way we manage the checkpoint offsets, and
> ensure
> > > >>>>>>>> they're
> > > >>>>>>>>>> only
> > > >>>>>>>>>>>>>>> written on commit().
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> 6.
> > > >>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what tests to
> > > >> write.
> > > >>>>>>>> How
> > > >>>>>>>>>>>>>> explicit
> > > >>>>>>>>>>>>>>> do we need to be here in the KIP?
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be
> > > seamless,
> > > >>>>>>>> and we
> > > >>>>>>>>>>>>>> should
> > > >>>>>>>>>>>>>>> definitely add some tests around that. Downgrade, it
> > > >>>>>>>> transpires,
> > > >>>>>>>>>> isn't
> > > >>>>>>>>>>>>>>> currently possible, as the extra column family for
> offset
> > > >>>>>>>> storage is
> > > >>>>>>>>>>>>>>> incompatible with the pre-KIP-892 implementation: when
> you
> > > >>>>>>>> open a
> > > >>>>>>>>>>>> RocksDB
> > > >>>>>>>>>>>>>>> database, you must open all available column families
> or
> > > >>>>>>>> receive an
> > > >>>>>>>>>>>>>> error.
> > > >>>>>>>>>>>>>>> What currently happens on downgrade is that it
> attempts to
> > > >>>>>>>> open the
> > > >>>>>>>>>>>>>> store,
> > > >>>>>>>>>>>>>>> throws an error about the offsets column family not
> being
> > > >>>>>>>> opened,
> > > >>>>>>>>>>>> which
> > > >>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given that
> > > >> downgrades
> > > >>>>>>>>>> should
> > > >>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>> uncommon, I think this is acceptable behaviour, as the
> > > >>>>>>>> end-state is
> > > >>>>>>>>>>>>>>> consistent, even if it results in an undesirable state
> > > >>>> restore.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Should I document the upgrade/downgrade behaviour
> > > explicitly
> > > >>>>>>>> in the
> > > >>>>>>>>>>>> KIP?
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> --
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > > >>>>>>>> cadonna@apache.org>
> > > >>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Hi Nick!
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Thanks for the updates!
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> > > >>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > > >>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore after
> this
> > > KIP
> > > >>>>>>>> is
> > > >>>>>>>>>>>>>>>> released, I would rather throw an unsupported
> operation
> > > >>>>>>>> exception
> > > >>>>>>>>>> by
> > > >>>>>>>>>>>>>>>> default.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>>> When would a state store return -1 from
> > > >>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while
> being
> > > >>>>>>>>>>>> transactional?
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Wouldn't StateStore#approximateNumUncommittedBytes()
> also
> > > >>>>>>>> return 0
> > > >>>>>>>>>> if
> > > >>>>>>>>>>>>>>>> the state store is transactional but nothing has been
> > > >> written
> > > >>>>>>>> to
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> state store yet?
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this KIP really
> > > need
> > > >>>> to
> > > >>>>>>>>>>>> introduce
> > > >>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()? StateStoreContext
> has
> > > >>>>>>>> already
> > > >>>>>>>>>>>>>>>> appConfigs() which basically exposes the same
> information,
> > > >>>>>>>> i.e., if
> > > >>>>>>>>>>>> EOS
> > > >>>>>>>>>>>>>>>> is enabled or not.
> > > >>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore interface
> as
> > > >>>>>>>> loosely
> > > >>>>>>>>>>>> coupled
> > > >>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> implementers
> > > >>>> more
> > > >>>>>>>>>>>> freedom,
> > > >>>>>>>>>>>>>>>> and reduce the amount of internal knowledge required."
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> While I understand the intent, I doubt that it
> decreases
> > > the
> > > >>>>>>>>>>>> coupling of
> > > >>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> > > >> READ_COMMITTED
> > > >>>>>>>> only
> > > >>>>>>>>>>>>>>>> applies to IQ but not to reads by processors. Thus,
> > > >>>>>>>> implementers
> > > >>>>>>>>>>>> need to
> > > >>>>>>>>>>>>>>>> understand how Streams accesses the state stores.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> I would like to hear what others think about this.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> 4.
> > > >>>>>>>>>>>>>>>> Great exposing new metrics for transactional state
> stores!
> > > >>>>>>>>>> However, I
> > > >>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate (in the
> > > docs)
> > > >>>>>>>> the 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 also need to adapt/add integration
> tests
> > > >>>>>>>> besides
> > > >>>>>>>>>> unit
> > > >>>>>>>>>>>>>>>> tests. Additionally, we probably need integration or
> > > system
> > > >>>>>>>> tests
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>>> verify that upgrades and downgrades between
> transactional
> > > >> and
> > > >>>>>>>>>>>>>>>> non-transactional state stores work as expected.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > > >>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in the KIP,
> > > under
> > > >>>>>>>>>>>> "Rejected
> > > >>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea worth
> pursuing,
> > > >>>>>>>> but I
> > > >>>>>>>>>>>> believe
> > > >>>>>>>>>>>>>>>>> that it's out of the scope of this KIP, because it
> > > solves a
> > > >>>>>>>>>>>> different
> > > >>>>>>>>>>>>>> set
> > > >>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of this one
> has
> > > >>>>>>>> already
> > > >>>>>>>>>> grown
> > > >>>>>>>>>>>>>>>> quite
> > > >>>>>>>>>>>>>>>>> large!
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > > >>>>>>>>>> nick.telford@gmail.com>
> > > >>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Hi everyone,
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> I've updated the KIP (
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>
> > > >>>>>
> > > >>>>
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > >>>>>>>>>>>>>>>> )
> > > >>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing back
> "Atomic
> > > >>>>>>>>>>>> Checkpointing"
> > > >>>>>>>>>>>>>>>> (for
> > > >>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think the one
> thing
> > > >>>>>>>> missing is
> > > >>>>>>>>>>>> some
> > > >>>>>>>>>>>>>>>>>> changes to metrics (notably the store "flush"
> metrics
> > > will
> > > >>>>>>>> need
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>> renamed to "commit").
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> The reason I brought back Atomic Checkpointing was
> to
> > > >>>>>>>> decouple
> > > >>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>> flush
> > > >>>>>>>>>>>>>>>>>> from store commit. This is important, because with
> > > >>>>>>>> Transactional
> > > >>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on *every*
> Task
> > > >>>>>>>> commit,
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>> just when the StateStore is closing, otherwise our
> > > >>>>>>>> transaction
> > > >>>>>>>>>>>> buffer
> > > >>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>> never be written and persisted, instead growing
> > > unbounded!
> > > >>>> I
> > > >>>>>>>>>>>>>>>> experimented
> > > >>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a store
> flush
> > > >>>>>>>> whenever
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its
> configured
> > > >>>>>>>> size, but
> > > >>>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>> was
> > > >>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer from
> being
> > > >>>>>>>>>> configured
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>> unbounded, and it still would have required explicit
> > > >>>>>>>> flushes of
> > > >>>>>>>>>>>>>> RocksDB,
> > > >>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory
> utilization.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right" way
> to
> > > >>>>>>>> resolve
> > > >>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog offsets that
> > > >>>>>>>> correspond
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> most
> > > >>>>>>>>>>>>>>>>>> recently written records are always atomically
> written
> > > to
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>> StateStore
> > > >>>>>>>>>>>>>>>>>> (by writing them to the same transaction buffer),
> we can
> > > >>>>>>>> avoid
> > > >>>>>>>>>>>>>> forcibly
> > > >>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting
> RocksDB
> > > >>>>>>>> flush
> > > >>>>>>>>>> them
> > > >>>>>>>>>>>>>> only
> > > >>>>>>>>>>>>>>>>>> when necessary, without losing any of our
> consistency
> > > >>>>>>>> guarantees.
> > > >>>>>>>>>>>> See
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> updated KIP for more info.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> I have fully implemented these changes, although I'm
> > > still
> > > >>>>>>>> not
> > > >>>>>>>>>>>>>> entirely
> > > >>>>>>>>>>>>>>>>>> happy with the implementation for segmented
> StateStores,
> > > >> so
> > > >>>>>>>> I
> > > >>>>>>>>>> plan
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests pass. If
> you'd
> > > like
> > > >>>>>>>> to try
> > > >>>>>>>>>>>> out
> > > >>>>>>>>>>>>>> or
> > > >>>>>>>>>>>>>>>>>> review this highly experimental and incomplete
> branch,
> > > >> it's
> > > >>>>>>>>>>>> available
> > > >>>>>>>>>>>>>>>> here:
> > > >>>>>>>>>>>>>>>>>>
> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > > .
> > > >>>>>>>> Note:
> > > >>>>>>>>>>>> it's
> > > >>>>>>>>>>>>>>>> built
> > > >>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base to
> build
> > > >>>>>>>> and test
> > > >>>>>>>>>>>> it
> > > >>>>>>>>>>>>>> on,
> > > >>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples comparisons in a
> > > live
> > > >>>>>>>>>>>>>> environment. I
> > > >>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's nearer
> > > >> completion
> > > >>>>>>>> and
> > > >>>>>>>>>> has
> > > >>>>>>>>>>>>>> been
> > > >>>>>>>>>>>>>>>>>> proven on our main application.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing and
> testing:
> > > >>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window) stores
> > > >>>>>>>>>>>>>>>>>> - Global stores
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> As I do not currently use either of these, so my
> primary
> > > >>>>>>>> test
> > > >>>>>>>>>>>>>>>> environment
> > > >>>>>>>>>>>>>>>>>> doesn't test these areas.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next week for
> a few
> > > >>>>>>>> weeks,
> > > >>>>>>>>>> so
> > > >>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>> not have time to move this forward until late
> August.
> > > That
> > > >>>>>>>> said,
> > > >>>>>>>>>>>> your
> > > >>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just won't be
> > > able
> > > >>>> to
> > > >>>>>>>>>>>> respond
> > > >>>>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>>>>> quickly as usual.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > > >>>>>>>>>> nick.telford@gmail.com>
> > > >>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Hi Bruno
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ is
> not
> > > >>>>>>>> something
> > > >>>>>>>>>> I
> > > >>>>>>>>>>>> had
> > > >>>>>>>>>>>>>>>>>>> considered.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> What about atomically updating the state store
> from the
> > > >>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> > > checkpoint
> > > >>>>>>>> (thus,
> > > >>>>>>>>>>>>>>>> flushing
> > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data
> and/or
> > > >>>>>>>> number of
> > > >>>>>>>>>>>>>> commit
> > > >>>>>>>>>>>>>>>>>>>> intervals?
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting
> that we
> > > >>>>>>>> add an
> > > >>>>>>>>>>>>>>>> additional
> > > >>>>>>>>>>>>>>>>>>> config for the max number of commit intervals
> between
> > > >>>>>>>>>> checkpoints?
> > > >>>>>>>>>>>>>> That
> > > >>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the
> transaction
> > > >>>>>>>> buffers
> > > >>>>>>>>>> are
> > > >>>>>>>>>>>>>>>> nearly
> > > >>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of commit
> > > intervals
> > > >>>>>>>> have
> > > >>>>>>>>>>>>>> elapsed,
> > > >>>>>>>>>>>>>>>>>>> whichever comes first?
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although this
> > > re-ignites
> > > >>>>>>>> an
> > > >>>>>>>>>>>> earlier
> > > >>>>>>>>>>>>>>>>>>> debate about whether a config should be measured in
> > > >>>>>>>> "number of
> > > >>>>>>>>>>>> commit
> > > >>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute time.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the reason I
> was
> > > >>>>>>>> pursuing
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> Atomic
> > > >>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush from
> > > >>>>>>>> checkpointing,
> > > >>>>>>>>>>>>>> which
> > > >>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every commit
> without
> > > any
> > > >>>>>>>>>>>> performance
> > > >>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the
> "best"
> > > >>>>>>>> solution,
> > > >>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>> I'm not
> > > >>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back into this
> KIP.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> I'm currently working on moving all the
> transactional
> > > >>>> logic
> > > >>>>>>>>>>>> directly
> > > >>>>>>>>>>>>>>>> into
> > > >>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with the
> > > >>>>>>>>>>>>>> StateStore#newTransaction
> > > >>>>>>>>>>>>>>>>>>> method, and reduces the number of new classes
> > > introduced,
> > > >>>>>>>>>>>>>> significantly
> > > >>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and the
> > > complexity
> > > >>>> is
> > > >>>>>>>>>>>>>> drastically
> > > >>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic Checkpoints
> > > into
> > > >>>>>>>> this
> > > >>>>>>>>>> KIP.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
> > > >>>>>>>> cadonna@apache.org>
> > > >>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> As far as I understand, you want to atomically
> update
> > > >> the
> > > >>>>>>>> state
> > > >>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the memtable
> of a
> > > >>>> state
> > > >>>>>>>>>> store
> > > >>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit time
> elapsed
> > > >>>> but
> > > >>>>>>>>>> after
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that would lead
> to
> > > >>>>>>>> exceeding
> > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes before the
> > > next
> > > >>>>>>>> commit
> > > >>>>>>>>>>>>>>>> interval
> > > >>>>>>>>>>>>>>>>>>>> ends.
> > > >>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would commit
> every
> > > >>>>>>>> commit
> > > >>>>>>>>>>>> interval
> > > >>>>>>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>>>>>> the state store will only be atomically updated
> > > roughly
> > > >>>>>>>> every
> > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of data.
> Also
> > > IQ
> > > >>>>>>>> would
> > > >>>>>>>>>>>> then
> > > >>>>>>>>>>>>>>>> only
> > > >>>>>>>>>>>>>>>>>>>> see new data roughly every
> > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > >>>>>>>>>>>>>>>>>>>> After a failure the state store needs to restore
> up to
> > > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Is this correct?
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> What about atomically updating the state store
> from
> > > the
> > > >>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> > > checkpoint
> > > >>>>>>>> (thus,
> > > >>>>>>>>>>>>>>>> flushing
> > > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data
> and/or
> > > >>>>>>>> number of
> > > >>>>>>>>>>>>>> commit
> > > >>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have the same
> delay
> > > >>>> for
> > > >>>>>>>>>>>> records
> > > >>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because both
> would
> > > >>>>>>>> appear
> > > >>>>>>>>>> when
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However, after a
> > > failure
> > > >>>>>>>> the
> > > >>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > >>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>> it might restore data that is already in the state
> > > store
> > > >>>>>>>>>> because
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable offset
> (i.e.
> > > the
> > > >>>>>>>> last
> > > >>>>>>>>>>>>>> committed
> > > >>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring data
> that
> > > is
> > > >>>>>>>> already
> > > >>>>>>>>>>>> in
> > > >>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should not
> violated.
> > > >>>>>>>>>>>>>>>>>>>> This solution needs at least one new config to
> specify
> > > >>>>>>>> when a
> > > >>>>>>>>>>>>>>>> checkpoint
> > > >>>>>>>>>>>>>>>>>>>> should be written.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail that
> does
> > > not
> > > >>>>>>>> change
> > > >>>>>>>>>>>>>>>> anything
> > > >>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit interval
> is 30
> > > >>>>>>>> seconds,
> > > >>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>> five
> > > >>>>>>>>>>>>>>>>>>>> seconds.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > > >>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch on our
> > > staging
> > > >>>>>>>>>>>>>> environment,
> > > >>>>>>>>>>>>>>>>>>>>> putting it through its paces in our non-trivial
> > > >>>>>>>> application.
> > > >>>>>>>>>> I'm
> > > >>>>>>>>>>>>>>>>>>>> already
> > > >>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate that we
> saw
> > > the
> > > >>>>>>>> last
> > > >>>>>>>>>>>> time
> > > >>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP, but this
> > > time,
> > > >> I
> > > >>>>>>>>>> think I
> > > >>>>>>>>>>>>>> know
> > > >>>>>>>>>>>>>>>>>>>> why.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is
> called
> > > at
> > > >>>>>>>> the end
> > > >>>>>>>>>>>> of
> > > >>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> Task
> > > >>>>>>>>>>>>>>>>>>>>> commit process, has the following behaviour:
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>           - Under ALOS: checkpoint the state
> stores.
> > > >> This
> > > >>>>>>>>>> includes
> > > >>>>>>>>>>>>>>>>>>>>>           flushing memtables in RocksDB. This is
> > > >>>> acceptable
> > > >>>>>>>>>>>> because the
> > > >>>>>>>>>>>>>>>>>>>> default
> > > >>>>>>>>>>>>>>>>>>>>>           commit.interval.ms is 5 seconds, so
> > > forcibly
> > > >>>>>>>> flushing
> > > >>>>>>>>>>>>>> memtables
> > > >>>>>>>>>>>>>>>>>>>> every 5
> > > >>>>>>>>>>>>>>>>>>>>>           seconds is acceptable for most
> > > applications.
> > > >>>>>>>>>>>>>>>>>>>>>           - Under EOS: checkpointing is not done,
> > > >> *unless*
> > > >>>>>>>> it's
> > > >>>>>>>>>>>> being
> > > >>>>>>>>>>>>>>>>>>>> forced, due
> > > >>>>>>>>>>>>>>>>>>>>>           to e.g. the Task closing or being
> revoked.
> > > >> This
> > > >>>>>>>> means
> > > >>>>>>>>>>>> that
> > > >>>>>>>>>>>>>> under
> > > >>>>>>>>>>>>>>>>>>>> normal
> > > >>>>>>>>>>>>>>>>>>>>>           processing conditions, the state stores
> > > will
> > > >> not
> > > >>>>>>>> be
> > > >>>>>>>>>>>>>>>> checkpointed,
> > > >>>>>>>>>>>>>>>>>>>> and will
> > > >>>>>>>>>>>>>>>>>>>>>           not have memtables flushed at all ,
> unless
> > > >>>> RocksDB
> > > >>>>>>>>>>>> decides to
> > > >>>>>>>>>>>>>>>>>>>> flush them on
> > > >>>>>>>>>>>>>>>>>>>>>           its own. Checkpointing stores and
> > > >> force-flushing
> > > >>>>>>>> their
> > > >>>>>>>>>>>>>> memtables
> > > >>>>>>>>>>>>>>>>>>>> is only
> > > >>>>>>>>>>>>>>>>>>>>>           done when a Task is being closed.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on
> at
> > > >>>> least
> > > >>>>>>>>>> *some*
> > > >>>>>>>>>>>>>>>> normal
> > > >>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB
> > > transaction
> > > >>>>>>>>>> buffers
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets are
> synced to
> > > >>>>>>>> disk to
> > > >>>>>>>>>>>>>> prevent
> > > >>>>>>>>>>>>>>>>>>>>> restores from getting out of hand. Consequently,
> my
> > > >>>>>>>> current
> > > >>>>>>>>>>>>>>>>>>>> implementation
> > > >>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit,
> which
> > > is
> > > >>>>>>>> far too
> > > >>>>>>>>>>>>>>>>>>>> frequent.
> > > >>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records,
> which
> > > is
> > > >> a
> > > >>>>>>>>>> change
> > > >>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>> flush
> > > >>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing performance
> problems
> > > for
> > > >>>>>>>> some
> > > >>>>>>>>>>>>>>>>>>>> applications.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm
> > > currently
> > > >>>>>>>> leaning
> > > >>>>>>>>>>>>>>>> towards
> > > >>>>>>>>>>>>>>>>>>>>> using the statestore.transaction.buffer.max.bytes
> > > >>>>>>>>>> configuration
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to exceed it.
> > > This
> > > >>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>> complement the
> > > >>>>>>>>>>>>>>>>>>>>> existing "early Task commit" functionality that
> this
> > > >>>>>>>>>>>> configuration
> > > >>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>           - Currently, we use
> > > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > > >>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> force an
> > > >>>>>>>>>>>>>>>>>>>>>           early Task commit if processing more
> > > records
> > > >>>> would
> > > >>>>>>>>>> cause
> > > >>>>>>>>>>>> our
> > > >>>>>>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>>>           transactions to exceed the memory
> assigned
> > > to
> > > >>>>>>>> them.
> > > >>>>>>>>>>>>>>>>>>>>>           - New functionality: when a Task *does*
> > > >> commit,
> > > >>>>>>>> we will
> > > >>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>> checkpoint
> > > >>>>>>>>>>>>>>>>>>>>>           the stores (and hence flush the
> transaction
> > > >>>>>>>> buffers)
> > > >>>>>>>>>>>> unless
> > > >>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>> expect to
> > > >>>>>>>>>>>>>>>>>>>>>           cross the
> > > >>>> statestore.transaction.buffer.max.bytes
> > > >>>>>>>>>>>> threshold
> > > >>>>>>>>>>>>>>>> before
> > > >>>>>>>>>>>>>>>>>>>> the next
> > > >>>>>>>>>>>>>>>>>>>>>           commit
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> > > >>>>>>>>>>>> nick.telford@gmail.com
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I meant in
> terms of
> > > >>>>>>>>>>>> understanding
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> performance profile under various
> circumstances. The
> > > >>>>>>>> more
> > > >>>>>>>>>>>> complex
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would be for
> users
> > > to
> > > >>>>>>>>>>>> understand
> > > >>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> performance they see. For example, spilling
> records
> > > to
> > > >>>>>>>> disk
> > > >>>>>>>>>>>> when
> > > >>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold would, I
> > > >> expect,
> > > >>>>>>>>>> reduce
> > > >>>>>>>>>>>>>> write
> > > >>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write throughput
> could
> > > >> be
> > > >>>>>>>>>>>>>> unexpected,
> > > >>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>> potentially difficult to diagnose/understand for
> > > >> users.
> > > >>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early commit"
> concept is
> > > >>>>>>>>>> relatively
> > > >>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document, and
> > > >>>> conceptually
> > > >>>>>>>>>> fairly
> > > >>>>>>>>>>>>>>>>>>>> obvious to
> > > >>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to make it
> > > >> easier
> > > >>>>>>>> to
> > > >>>>>>>>>>>>>>>> understand
> > > >>>>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>>>> it happens though.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > > >>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an indirect
> way
> > > of
> > > >>>>>>>>>> telling
> > > >>>>>>>>>>>>>>>>>>>> StateStores
> > > >>>>>>>>>>>>>>>>>>>>>> whether they should be transactional.
> READ_COMMITTED
> > > >>>>>>>>>>>> essentially
> > > >>>>>>>>>>>>>>>>>>>> requires
> > > >>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that two
> threads
> > > >>>>>>>> calling
> > > >>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes from
> the
> > > >> other
> > > >>>>>>>>>>>>>> transaction
> > > >>>>>>>>>>>>>>>>>>>> until
> > > >>>>>>>>>>>>>>>>>>>>>> they have been committed. With
> READ_UNCOMMITTED, all
> > > >>>>>>>> bets are
> > > >>>>>>>>>>>> off,
> > > >>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe written
> records
> > > at
> > > >>>>>>>> any
> > > >>>>>>>>>>>> time,
> > > >>>>>>>>>>>>>>>>>>>> which is
> > > >>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That said,
> > > StateStores
> > > >>>>>>>> are
> > > >>>>>>>>>> free
> > > >>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>> implement
> > > >>>>>>>>>>>>>>>>>>>>>> these guarantees however they can, which is a
> bit
> > > more
> > > >>>>>>>>>> relaxed
> > > >>>>>>>>>>>>>> than
> > > >>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions". For
> example,
> > > >>>> with
> > > >>>>>>>>>>>> RocksDB
> > > >>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
> > > >>>>>>>>>> "transactions",
> > > >>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the
> database.
> > > But
> > > >>>>>>>> with
> > > >>>>>>>>>>>> other
> > > >>>>>>>>>>>>>>>>>>>> storage
> > > >>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to *always* use
> > > >>>>>>>> transactions,
> > > >>>>>>>>>>>> even
> > > >>>>>>>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines
> that
> > > >> don't
> > > >>>>>>>>>> provide
> > > >>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation guarantees can
> be
> > > met
> > > >>>>>>>> using a
> > > >>>>>>>>>>>>>>>>>>>> different
> > > >>>>>>>>>>>>>>>>>>>>>> technique.
> > > >>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore
> interface
> > > as
> > > >>>>>>>>>> loosely
> > > >>>>>>>>>>>>>>>> coupled
> > > >>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > > >>>>>>>> implementers
> > > >>>>>>>>>> more
> > > >>>>>>>>>>>>>>>>>>>> freedom, and
> > > >>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge
> required.
> > > >>>>>>>>>>>>>>>>>>>>>> That said, I understand that "IsolationLevel"
> might
> > > >> not
> > > >>>>>>>> be
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>> right
> > > >>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it much more
> > > >>>>>>>> explicit if
> > > >>>>>>>>>>>>>>>>>>>> required, e.g.
> > > >>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> 7-8.
> > > >>>>>>>>>>>>>>>>>>>>>> I can make these changes either later today or
> > > >>>> tomorrow.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Small update:
> > > >>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a
> bunch of
> > > >>>>>>>> issues
> > > >>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>> needed
> > > >>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass,
> which is
> > > >>>>>>>>>> promising,
> > > >>>>>>>>>>>> but
> > > >>>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>>>>>>> need to undergo some performance testing. I
> haven't
> > > >>>>>>>> (yet)
> > > >>>>>>>>>>>> worked
> > > >>>>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I
> would
> > > >>>>>>>> expect
> > > >>>>>>>>>> that,
> > > >>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no difference. The
> > > >> branch
> > > >>>>>>>> is
> > > >>>>>>>>>>>>>> available
> > > >>>>>>>>>>>>>>>>>>>> at
> > > >>>>>>>>>>>>>>>>>>>>>>
> https://github.com/nicktelford/kafka/tree/KIP-892-c
> > > >> if
> > > >>>>>>>>>> anyone
> > > >>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> > > >>>>>>>>>>>> cadonna@apache.org>
> > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually also
> my
> > > >>>>>>>> point. I
> > > >>>>>>>>>>>>>>>> understood
> > > >>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion path as
> a way
> > > >> to
> > > >>>>>>>> avoid
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> early
> > > >>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually it is
> public
> > > >>>>>>>> API. My
> > > >>>>>>>>>>>>>> question
> > > >>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal
> processing.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the size of the
> > > >>>>>>>> transaction
> > > >>>>>>>>>>>>>> buffer
> > > >>>>>>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit affects the
> whole
> > > >>>>>>>>>> processing
> > > >>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > >>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a state store
> is
> > > not
> > > >>>>>>>>>>>> confined to
> > > >>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and
> changes the
> > > >>>>>>>> behavior
> > > >>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>> other
> > > >>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you that it
> is a
> > > >>>>>>>> decent
> > > >>>>>>>>>>>>>>>>>>>> compromise. I
> > > >>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and list
> the
> > > >>>>>>>> options to
> > > >>>>>>>>>>>>>>>> overcome
> > > >>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all options
> seem
> > > >>>> quite
> > > >>>>>>>>>> heavy
> > > >>>>>>>>>>>>>>>>>>>> compared
> > > >>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you
> mean
> > > with
> > > >>>>>>>> "less
> > > >>>>>>>>>>>>>>>>>>>> predictable
> > > >>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the alternatives
> > > really
> > > >>>>>>>>>>>>>> interesting.
> > > >>>>>>>>>>>>>>>>>>>> But I
> > > >>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good and we
> should
> > > >>>>>>>> continue
> > > >>>>>>>>>>>> with
> > > >>>>>>>>>>>>>>>> it!
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on
> June
> > > >> 20th:
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind
> putting
> > > >>>>>>>> isolation
> > > >>>>>>>>>>>>>> level
> > > >>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should that
> also
> > > be
> > > >> a
> > > >>>>>>>> way
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>> give
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to decide
> whether
> > > to
> > > >>>>>>>> turn on
> > > >>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > > >>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned about
> how do
> > > >> you
> > > >>>>>>>> know
> > > >>>>>>>>>>>> if a
> > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written under EOS,
> if
> > > you
> > > >>>>>>>> do not
> > > >>>>>>>>>>>>>> have a
> > > >>>>>>>>>>>>>>>>>>>> way
> > > >>>>>>>>>>>>>>>>>>>>>>> to know if the state store is transactional or
> not.
> > > >> If
> > > >>>>>>>> a
> > > >>>>>>>>>> state
> > > >>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be
> written
> > > >>>>>>>> during
> > > >>>>>>>>>>>> normal
> > > >>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state store is not
> > > >>>>>>>>>> transactional,
> > > >>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written under EOS.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 7.
> > > >>>>>>>>>>>>>>>>>>>>>>> My point was about not only considering the
> bytes
> > > in
> > > >>>>>>>> memory
> > > >>>>>>>>>> in
> > > >>>>>>>>>>>>>>>> config
> > > >>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also
> bytes
> > > that
> > > >>>>>>>> might
> > > >>>>>>>>>> be
> > > >>>>>>>>>>>>>>>>>>>> spilled
> > > >>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering whether you
> > > >> should
> > > >>>>>>>>>> remove
> > > >>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory bytes to
> be
> > > >> used
> > > >>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My
> > > thinking
> > > >>>>>>>> was
> > > >>>>>>>>>> that
> > > >>>>>>>>>>>>>> even
> > > >>>>>>>>>>>>>>>>>>>> if a
> > > >>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to disk,
> > > >> limiting
> > > >>>>>>>> the
> > > >>>>>>>>>>>>>> overall
> > > >>>>>>>>>>>>>>>>>>>> bytes
> > > >>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it again and
> > > >>>>>>>> considering
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> recent
> > > >>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much sense
> > > anymore.
> > > >>>>>>>>>>>>>>>>>>>>>>> I like the name
> > > >>>>>>>> statestore.transaction.buffer.max.bytes that
> > > >>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>> proposed.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> 8.
> > > >>>>>>>>>>>>>>>>>>>>>>> A high-level description (without
> implementation
> > > >>>>>>>> details) of
> > > >>>>>>>>>>>> how
> > > >>>>>>>>>>>>>>>>>>>> Kafka
> > > >>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
> > > >>>>>>>> transactions,
> > > >>>>>>>>>>>> state
> > > >>>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be great.
> > > Would
> > > >>>> be
> > > >>>>>>>>>> great
> > > >>>>>>>>>>>> if
> > > >>>>>>>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the
> behavior in
> > > >>>>>>>> case of
> > > >>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>>>>> failure.
> > > >>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional state
> store
> > > >>>>>>>> recover
> > > >>>>>>>>>>>> after a
> > > >>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the transaction
> > > buffer,
> > > >>>>>>>> etc.
> > > >>>>>>>>>>>> (that
> > > >>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>>>> what
> > > >>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> > > >>>>>>>> WriteBatchWithIndex
> > > >>>>>>>>>>>>>>>>>>>> transactions
> > > >>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
> > > >>>>>>>> configured
> > > >>>>>>>>>>>> memory
> > > >>>>>>>>>>>>>>>>>>>> needs to
> > > >>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns. Ultimately,
> any
> > > >>>>>>>> approach
> > > >>>>>>>>>>>> based
> > > >>>>>>>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>>>>>>> cracking
> > > >>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in ways
> it's
> > > not
> > > >>>>>>>> really
> > > >>>>>>>>>>>>>>>> designed
> > > >>>>>>>>>>>>>>>>>>>>>>> for is
> > > >>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen performance or
> > > >>>>>>>> consistency
> > > >>>>>>>>>>>> issues.
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing these
> early
> > > >>>>>>>> commits?
> > > >>>>>>>>>>>> While
> > > >>>>>>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > > >>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to ensure
> > > >>>>>>>> consistency
> > > >>>>>>>>>>>> whilst
> > > >>>>>>>>>>>>>>>>>>>>>>> maintaining
> > > >>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> > > >>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
> > > >>>>>>>> complicated, and
> > > >>>>>>>>>>>> might
> > > >>>>>>>>>>>>>>>>>>>>>>> actually
> > > >>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for users as a
> > > >>>>>>>> consequence.
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this
> KIP is
> > > >>>>>>>> growing a
> > > >>>>>>>>>>>> bit
> > > >>>>>>>>>>>>>>>> out
> > > >>>>>>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas for
> > > future
> > > >>>>>>>>>>>>>>>> improvements, I
> > > >>>>>>>>>>>>>>>>>>>>>>> think
> > > >>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope down to a
> > > design
> > > >>>>>>>> that
> > > >>>>>>>>>>>>>> achieves
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>>>> most
> > > >>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized
> restorations
> > > >>>>>>>> during
> > > >>>>>>>>>> dirty
> > > >>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that
> this KIP
> > > >>>>>>>> produces
> > > >>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>>>>>> ultimately
> > > >>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially if the
> bulk
> > > of
> > > >>>>>>>> it is
> > > >>>>>>>>>>>>>> internal
> > > >>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week trying
> to
> > > >>>>>>>> re-work
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>> original
> > > >>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
> > > >>>>>>>> newTransaction()
> > > >>>>>>>>>>>>>> method,
> > > >>>>>>>>>>>>>>>>>>>> such
> > > >>>>>>>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
> > > RocksDBStore.
> > > >>>>>>>> That
> > > >>>>>>>>>>>> way, if
> > > >>>>>>>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>>>>>>> want to
> > > >>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the future,
> like
> > > the
> > > >>>>>>>> SST
> > > >>>>>>>>>> file
> > > >>>>>>>>>>>>>>>>>>>> management
> > > >>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with
> little/no
> > > >>>> API
> > > >>>>>>>>>>>> changes.
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>
> > > >>>
> > > >>
> > > >
> > >
>
>

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

Posted by Guozhang Wang <gu...@gmail.com>.
Hello Nick,

First of all, thanks a lot for the great effort you've put in driving
this KIP! I really like it coming through finally, as many people in
the community have raised this. At the same time I honestly feel a bit
ashamed for not putting enough of my time supporting it and pushing it
through the finish line (you raised this KIP almost a year ago).

I briefly passed through the DISCUSS thread so far, not sure I've 100
percent digested all the bullet points. But with the goal of trying to
help take it through the finish line in mind, I'd want to throw
thoughts on top of my head only on the point #4 above which I felt may
be the main hurdle for the current KIP to drive to a consensus now.

The general question I asked myself is, whether we want to couple "IQ
reading mode" with "processing mode". While technically I tend to
agree with you that, it's feels like a bug if some single user chose
"EOS" for processing mode while choosing "read uncommitted" for IQ
reading mode, at the same time, I'm thinking if it's possible that
there could be two different persons (or even two teams) that would be
using the stream API to build the app, and the IQ API to query the
running state of the app. I know this is less of a technical thing but
rather a more design stuff, but if it could be ever the case, I'm
wondering if the personale using the IQ API knows about the risks of
using read uncommitted but still chose so for the favor of
performance, no matter if the underlying stream processing mode
configured by another personale is EOS or not. In that regard, I'm
leaning towards a "leaving the door open, and close it later if we
found it's a bad idea" aspect with a configuration 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 can still enforce
coupling them.

Again, just my 2c here. Thanks again for the great patience and
diligence on this KIP.


Guozhang



On Fri, Oct 13, 2023 at 8:48 AM Nick Telford <ni...@gmail.com> wrote:
>
> Hi Bruno,
>
> 4.
> I'll hold off on making that change until we have a consensus as to what
> configuration to use to control all of this, as it'll be affected by the
> decision on EOS isolation levels.
>
> 5.
> Done. I've chosen "committedOffsets".
>
> Regards,
> Nick
>
> On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <ca...@apache.org> wrote:
>
> > Hi Nick,
> >
> > 1.
> > Yeah, you are probably right that it does not make too much sense.
> > Thanks for the clarification!
> >
> >
> > 4.
> > Yes, sorry for the back and forth, but I think for the sake of the KIP
> > it is better to let the ALOS behavior as it is for now due to the
> > possible issues you would run into. Maybe we can find a solution in the
> > future. Now the question returns to whether we really need
> > default.state.isolation.level. Maybe the config could be the feature
> > flag Sophie requested.
> >
> >
> > 5.
> > There 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 committedOffsetFor().
> >
> >
> > Best,
> > Bruno
> >
> >
> > On 10/13/23 3:21 PM, Nick Telford wrote:
> > > Hi Bruno,
> > >
> > > Thanks for getting back to me.
> > >
> > > 1.
> > > I think this should be possible. Are you thinking of the situation where
> > a
> > > user may downgrade to a previous version of Kafka Streams? In that case,
> > > sadly, the RocksDBStore would get wiped by the older version of Kafka
> > > Streams anyway, because that version wouldn't understand the extra column
> > > family (that holds offsets), so the missing Position file would
> > > automatically get rebuilt when the store is rebuilt from the changelog.
> > > Are there other situations than downgrade where a transactional store
> > could
> > > be replaced by a non-transactional one? I can't think of any.
> > >
> > > 2.
> > > Ahh yes, the Test Plan - my Kryptonite! This section definitely needs to
> > be
> > > fleshed out. I'll work on that. How much detail do you need?
> > >
> > > 3.
> > > See my previous email discussing this.
> > >
> > > 4.
> > > Hmm, this is an interesting point. Are you suggesting that under ALOS
> > > READ_COMMITTED should not be supported?
> > >
> > > Regards,
> > > Nick
> > >
> > > On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <ca...@apache.org> wrote:
> > >
> > >> Hi Nick,
> > >>
> > >> I think the KIP is converging!
> > >>
> > >>
> > >> 1.
> > >> I am wondering whether it makes sense to write the position file during
> > >> close as we do for the checkpoint file, so that in case the state store
> > >> is replaced with a non-transactional state store the non-transactional
> > >> state store finds the position file. I think, this is not strictly
> > >> needed, but would be a nice behavior instead of just deleting the
> > >> position file.
> > >>
> > >>
> > >> 2.
> > >> The test plan does not mention integration tests. Do you not need to
> > >> extend existing ones and add new ones. Also for upgrading and
> > >> downgrading you might need integration and/or system tests.
> > >>
> > >>
> > >> 3.
> > >> I think Sophie made a point. Although, IQ reading from uncommitted data
> > >> under EOS might be considered a bug by some people. Thus, your KIP would
> > >> fix a bug rather than changing the intended behavior. However, I also
> > >> see that a feature flag would help users that rely on this buggy
> > >> behavior (at least until AK 4.0).
> > >>
> > >>
> > >> 4.
> > >> This is related to the previous point. I assume that the difference
> > >> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is that in the
> > >> former you enable transactions on the state store and in the latter you
> > >> disable them. If my assumption is correct, I think that is an issue.
> > >> Let's assume under ALOS Streams fails over a couple of times more or
> > >> less at the same step in processing after value 3 is added to an
> > >> aggregation but the offset of the corresponding input record was not
> > >> committed. Without transactions disabled, the aggregation value would
> > >> increase by 3 for each failover. With transactions enabled, value 3
> > >> would only be added to the aggregation once when the offset of the input
> > >> record is committed and the transaction finally completes. So the
> > >> content of the state store would change depending on the configuration
> > >> for IQ. IMO, the content of the state store should be independent from
> > >> IQ. Given this issue, I propose to not use transactions with ALOS at
> > >> all. I was a big proponent of using transactions with ALOS, but I
> > >> realized that transactions with ALOS is not as easy as enabling
> > >> transactions on state stores. Another aspect that is problematic is that
> > >> the changelog topic which actually replicates the state store is not
> > >> transactional under ALOS. Thus, it might happen that the state store and
> > >> the changelog differ in their content. All of this is maybe solvable
> > >> somehow, but for the sake of this KIP, I would leave it for the future.
> > >>
> > >>
> > >> Best,
> > >> Bruno
> > >>
> > >>
> > >>
> > >> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > >>> Hey Nick! First of all thanks for taking up this awesome feature, I'm
> > >> sure
> > >>> every single
> > >>> Kafka Streams user and dev would agree that it is sorely needed.
> > >>>
> > >>> I've just been catching up on the KIP and surrounding discussion, so
> > >> please
> > >>> forgive me
> > >>> for any misunderstandings or misinterpretations of the current plan and
> > >>> don't hesitate to
> > >>> correct me.
> > >>>
> > >>> Before I jump in, I just want to say that having seen this drag on for
> > so
> > >>> long, my singular
> > >>> goal in responding is to help this KIP past a perceived impasse so we
> > can
> > >>> finally move on
> > >>> to voting and implementing it. Long discussions are to be expected for
> > >>> major features like
> > >>> this but it's completely on us as the Streams devs to make sure there
> > is
> > >> an
> > >>> end in sight
> > >>> for any ongoing discussion.
> > >>>
> > >>> With that said, it's my understanding that the KIP as currently
> > proposed
> > >> is
> > >>> just not tenable
> > >>> for Kafka Streams, and would prevent some EOS users from upgrading to
> > the
> > >>> version it
> > >>> first appears in. Given that we can't predict or guarantee whether any
> > of
> > >>> the followup KIPs
> > >>> would be completed in the same release cycle as this one, we need to
> > make
> > >>> sure that the
> > >>> feature is either compatible with all current users or else
> > >> feature-flagged
> > >>> so that they may
> > >>> opt in/out.
> > >>>
> > >>> Therefore, IIUC we need to have either (or both) of these as
> > >>> fully-implemented config options:
> > >>> 1. default.state.isolation.level
> > >>> 2. enable.transactional.state.stores
> > >>>
> > >>> This way EOS users for whom read_committed semantics are not viable can
> > >>> still upgrade,
> > >>> and either use the isolation.level config to leverage the new txn state
> > >>> stores without sacrificing
> > >>> their application semantics, or else simply keep the transactional
> > state
> > >>> stores disabled until we
> > >>> are able to fully implement the isolation level configuration at either
> > >> an
> > >>> application or query level.
> > >>>
> > >>> Frankly you are the expert here and know much more about the tradeoffs
> > in
> > >>> both semantics and
> > >>> effort level of implementing one of these configs vs the other. In my
> > >>> opinion, either option would
> > >>> be fine and I would leave the decision of which one to include in this
> > >> KIP
> > >>> completely up to you.
> > >>> I just don't see a way for the KIP to proceed without some variation of
> > >> the
> > >>> above that would allow
> > >>> EOS users to opt-out of read_committed.
> > >>>
> > >>> (If it's all the same to you, I would recommend 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 that gets released.
> > >>> So personally, my choice would be to add the feature flag and leave it
> > >> off
> > >>> by default. If all 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
> > >>>
> > >>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <ni...@gmail.com>
> > >> wrote:
> > >>>
> > >>>> Hi Bruno,
> > >>>>
> > >>>> Agreed, I can live with that for now.
> > >>>>
> > >>>> In an effort to keep the scope of this KIP from expanding, I'm leaning
> > >>>> towards just providing a configurable default.state.isolation.level
> > and
> > >>>> removing IsolationLevel from the StateStoreContext. This would be
> > >>>> compatible with adding support for query-time IsolationLevels in the
> > >>>> future, whilst providing a way for users to select an isolation level
> > >> now.
> > >>>>
> > >>>> The big problem with this, however, is that if a user selects
> > >>>> processing.mode
> > >>>> = "exactly-once(-v2|-beta)", and default.state.isolation.level =
> > >>>> "READ_UNCOMMITTED", we need to guarantee that the data isn't written
> > to
> > >>>> disk until commit() is called, but we also need to permit IQ threads
> > to
> > >>>> read from the ongoing transaction.
> > >>>>
> > >>>> A simple solution would be to (temporarily) forbid this combination of
> > >>>> configuration, and have default.state.isolation.level automatically
> > >> switch
> > >>>> to READ_COMMITTED when processing.mode is anything other than
> > >>>> at-least-once. Do you think this would be acceptable?
> > >>>>
> > >>>> In a later KIP, we can add support for query-time isolation levels and
> > >>>> solve this particular problem there, which would relax this
> > restriction.
> > >>>>
> > >>>> Regards,
> > >>>> Nick
> > >>>>
> > >>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <ca...@apache.org>
> > >> 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,
> > >>>>> Bruno
> > >>>>>
> > >>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> > >>>>>> 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 <nick.telford@gmail.com
> > >
> > >>>>> wrote:
> > >>>>>>
> > >>>>>>> Hi everyone,
> > >>>>>>>
> > >>>>>>> I agree that having IsolationLevel be determined at query-time is
> > the
> > >>>>>>> ideal design, but there are a few sticking points:
> > >>>>>>>
> > >>>>>>> 1.
> > >>>>>>> There needs to be some way to communicate the IsolationLevel down
> > to
> > >>>> the
> > >>>>>>> RocksDBStore itself, so that the query can respect it. Since stores
> > >>>> are
> > >>>>>>> "layered" in functionality (i.e. ChangeLoggingStore, MeteredStore,
> > >>>>> etc.),
> > >>>>>>> we need some way to deliver that information to the bottom layer.
> > For
> > >>>>> IQv2,
> > >>>>>>> we can use the existing State#query() method, but IQv1 has no way
> > to
> > >>>> do
> > >>>>>>> this.
> > >>>>>>>
> > >>>>>>> A simple approach, which would potentially open up other options,
> > >>>> would
> > >>>>> be
> > >>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> > >>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> > ReadOnlyKeyValueStore
> > >>>>> (and
> > >>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
> > >>>>>>>
> > >>>>>>> 2.
> > >>>>>>> As mentioned above, RocksDB WriteBatches are not thread-safe, which
> > >>>>> causes
> > >>>>>>> a problem if we want to provide READ_UNCOMMITTED Iterators. I also
> > >>>> had a
> > >>>>>>> look at RocksDB Transactions[1], but they solve a very different
> > >>>>> problem,
> > >>>>>>> and have the same thread-safety issue.
> > >>>>>>>
> > >>>>>>> One possible approach that I mentioned is chaining WriteBatches:
> > >> every
> > >>>>>>> time a new Interactive Query is received (i.e. readOnlyView, see
> > >>>> above,
> > >>>>>>> is called) we "freeze" the existing WriteBatch, and start a new one
> > >>>> for
> > >>>>> new
> > >>>>>>> writes. The Interactive Query queries the "chain" of previous
> > >>>>> WriteBatches
> > >>>>>>> + the underlying database; while the StreamThread starts writing to
> > >>>> the
> > >>>>>>> *new* WriteBatch. On-commit, the StreamThread would write *all*
> > >>>>>>> WriteBatches in the chain to the database (that have not yet been
> > >>>>> written).
> > >>>>>>>
> > >>>>>>> WriteBatches would be closed/freed only when they have been both
> > >>>>>>> committed, and all open Interactive Queries on them have been
> > closed.
> > >>>>> This
> > >>>>>>> would require some reference counting.
> > >>>>>>>
> > >>>>>>> Obviously a drawback of this approach is the potential for
> > increased
> > >>>>>>> memory usage: if an Interactive Query is long-lived, for example by
> > >>>>> doing a
> > >>>>>>> full scan over a large database, or even just pausing in the middle
> > >> of
> > >>>>> an
> > >>>>>>> iteration, then the existing chain of WriteBatches could be kept
> > >>>> around
> > >>>>> for
> > >>>>>>> a long time, potentially forever.
> > >>>>>>>
> > >>>>>>> --
> > >>>>>>>
> > >>>>>>> A.
> > >>>>>>> Going off on a tangent, it looks like in addition to supporting
> > >>>>>>> READ_COMMITTED queries, we could go further and support
> > >>>> REPEATABLE_READ
> > >>>>>>> queries (i.e. where subsequent reads to the same key in the same
> > >>>>>>> Interactive Query are guaranteed to yield the same value) by making
> > >>>> use
> > >>>>> of
> > >>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so the
> > >> performance
> > >>>>>>> impact is likely to be negligible, but they do require that the
> > >>>>> Interactive
> > >>>>>>> Query session can be explicitly closed.
> > >>>>>>>
> > >>>>>>> This could be achieved if we made the above readOnlyView interface
> > >>>> look
> > >>>>>>> more like:
> > >>>>>>>
> > >>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> > >>>> ReadOnlyKeyValueStore<K,
> > >>>>>>> V>, AutoCloseable {}
> > >>>>>>>
> > >>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> > >>>>>>>        ...
> > >>>>>>>        ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel
> > >>>>> isolationLevel);
> > >>>>>>> }
> > >>>>>>>
> > >>>>>>> But this would be a breaking change, as existing IQv1 queries are
> > >>>>>>> guaranteed to never call store.close(), and therefore these would
> > >> leak
> > >>>>>>> memory under REPEATABLE_READ.
> > >>>>>>>
> > >>>>>>> B.
> > >>>>>>> One thing that's notable: MyRocks states that they support
> > >>>>> READ_COMMITTED
> > >>>>>>> and REPEATABLE_READ, but they make no mention of
> > >>>> READ_UNCOMMITTED[3][4].
> > >>>>>>> This could be because doing so is technically difficult/impossible
> > >>>> using
> > >>>>>>> the primitives available in RocksDB.
> > >>>>>>>
> > >>>>>>> --
> > >>>>>>>
> > >>>>>>> Lucas, to address your points:
> > >>>>>>>
> > >>>>>>> U1.
> > >>>>>>> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
> > >>>>>>> implementations of StateStore that do not support atomic writes.
> > >>>>> Obviously
> > >>>>>>> in those cases, the guarantees Kafka Streams provides/expects would
> > >> be
> > >>>>>>> relaxed. Do you think we should require all implementations to
> > >> support
> > >>>>>>> atomic writes?
> > >>>>>>>
> > >>>>>>> U2.
> > >>>>>>> Stores can support multiple IsolationLevels. As we've discussed
> > >> above,
> > >>>>> the
> > >>>>>>> ideal scenario would be to specify the IsolationLevel at
> > query-time.
> > >>>>>>> Failing that, I think the second-best approach is to define the
> > >>>>>>> IsolationLevel for *all* queries based on the processing.mode,
> > which
> > >>>> is
> > >>>>>>> what the default StateStoreContext#isolationLevel() achieves. Would
> > >>>> you
> > >>>>>>> prefer an alternative?
> > >>>>>>>
> > >>>>>>> While the existing implementation is equivalent to
> > READ_UNCOMMITTED,
> > >>>>> this
> > >>>>>>> can yield unexpected results/errors under EOS, if a transaction is
> > >>>>> rolled
> > >>>>>>> back. While this would be a change in behaviour for users, it would
> > >>>> look
> > >>>>>>> more like a bug fix than a breaking change. That said, we *could*
> > >> make
> > >>>>> it
> > >>>>>>> configurable, and default to the existing behaviour
> > >> (READ_UNCOMMITTED)
> > >>>>>>> instead of inferring it from the processing.mode?
> > >>>>>>>
> > >>>>>>> N1, N2.
> > >>>>>>> These were only primitives to avoid boxing costs, but since this is
> > >>>> not
> > >>>>> a
> > >>>>>>> performance sensitive area, it should be fine to change if that's
> > >>>>> desirable.
> > >>>>>>>
> > >>>>>>> N3.
> > >>>>>>> It's because the store "manages its own offsets", which includes
> > both
> > >>>>>>> committing the offset, *and providing it* via getCommittedOffset().
> > >>>>>>> Personally, I think "managesOffsets" conveys this best, but I don't
> > >>>> mind
> > >>>>>>> changing it if the nomenclature is unclear.
> > >>>>>>>
> > >>>>>>> Sorry for the massive emails/essays!
> > >>>>>>> --
> > >>>>>>> Nick
> > >>>>>>>
> > >>>>>>> 1: https://github.com/facebook/rocksdb/wiki/Transactions
> > >>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > >>>>>>> 3:
> > https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > >>>>>>> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > >>>>>>>
> > >>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > >>>>>>> <lb...@confluent.io.invalid> wrote:
> > >>>>>>>
> > >>>>>>>> Hi Nick,
> > >>>>>>>>
> > >>>>>>>> since I last read it in April, the KIP has become much cleaner and
> > >>>>>>>> easier to read. Great work!
> > >>>>>>>>
> > >>>>>>>> It feels to me the last big open point is whether we can implement
> > >>>>>>>> isolation level as a query parameter. I understand that there are
> > >>>>>>>> implementation concerns, but as Colt says, it would be a great
> > >>>>>>>> addition, and would also simplify the migration path for this
> > >> change.
> > >>>>>>>> Is the implementation problem you mentioned caused by the
> > WriteBatch
> > >>>>>>>> not having a notion of a snapshot, as the underlying DB iterator
> > >>>> does?
> > >>>>>>>> In that case, I am not sure a chain of WriteBatches as you propose
> > >>>>>>>> would fully solve the problem, but maybe I didn't dig enough into
> > >> the
> > >>>>>>>> details to fully understand it.
> > >>>>>>>>
> > >>>>>>>> If it's not possible to implement it now, would it be an option to
> > >>>>>>>> make sure in this KIP that we do not fully close the door on
> > >>>> per-query
> > >>>>>>>> isolation levels in the interface, as it may be possible to
> > >> implement
> > >>>>>>>> the missing primitives in RocksDB or Speedb in the future.
> > >>>>>>>>
> > >>>>>>>> Understanding:
> > >>>>>>>>
> > >>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to be persisted
> > >>>>>>>> atomically with the records?
> > >>>>>>>> * U2) Don't understand the default implementation of
> > >>>> `isolationLevel`.
> > >>>>>>>> The isolation level should be a property of the underlying store,
> > >> and
> > >>>>>>>> not be defined by the default config? Existing stores probably
> > don't
> > >>>>>>>> guarantee READ_COMMITTED, so the default should be to return
> > >>>>>>>> READ_UNCOMMITTED.
> > >>>>>>>>
> > >>>>>>>> Nits:
> > >>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong` return type,
> > >> to
> > >>>>>>>> avoid the `null`?
> > >>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
> > >>>>>>>> return type, to avoid the `-1`?
> > >>>>>>>> * N3) I don't understand why `managesOffsets` uses the 'manage'
> > >> verb,
> > >>>>>>>> whereas all other methods use the "commits" verb. I'd suggest
> > >>>>>>>> `commitsOffsets`.
> > >>>>>>>>
> > >>>>>>>> Either way, it feels this KIP is very close to the finish line,
> > I'm
> > >>>>>>>> looking forward to seeing this in production!
> > >>>>>>>>
> > >>>>>>>> Cheers,
> > >>>>>>>> Lucas
> > >>>>>>>>
> > >>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <colt@littlehorse.io
> > >
> > >>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> Making IsolationLevel a query-time constraint, rather than
> > linking
> > >>>> it
> > >>>>>>>> to
> > >>>>>>>>> the processing.guarantee.
> > >>>>>>>>>
> > >>>>>>>>> As I understand it, would this allow even a user of EOS to
> > control
> > >>>>>>>> whether
> > >>>>>>>>> reading committed or uncommitted records? If so, I am highly in
> > >>>> favor
> > >>>>> of
> > >>>>>>>>> this.
> > >>>>>>>>>
> > >>>>>>>>> I know that I was one of the early people to point out the
> > current
> > >>>>>>>>> shortcoming that IQ reads uncommitted records, but just this
> > >>>> morning I
> > >>>>>>>>> realized a pattern we use which means that (for certain queries)
> > >> our
> > >>>>>>>> system
> > >>>>>>>>> needs to be able to read uncommitted records, which is the
> > current
> > >>>>>>>> behavior
> > >>>>>>>>> of Kafka Streams in EOS.***
> > >>>>>>>>>
> > >>>>>>>>> If IsolationLevel being a query-time decision allows for this,
> > then
> > >>>>> that
> > >>>>>>>>> would be amazing. I would also vote that the default behavior
> > >> should
> > >>>>> be
> > >>>>>>>> for
> > >>>>>>>>> reading uncommitted records, because it is totally possible for a
> > >>>>> valid
> > >>>>>>>>> application to depend on that behavior, and breaking it in a
> > minor
> > >>>>>>>> release
> > >>>>>>>>> might be a bit strong.
> > >>>>>>>>>
> > >>>>>>>>> *** (Note, for the curious reader....) Our use-case/query pattern
> > >>>> is a
> > >>>>>>>> bit
> > >>>>>>>>> complex, but reading "uncommitted" records is actually safe in
> > our
> > >>>>> case
> > >>>>>>>>> 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 soon as the processor processes the result.
> > If
> > >>>>> they
> > >>>>>>>>> can't read uncommitted records, they may receive a "201 -
> > Created"
> > >>>>>>>>> response, immediately followed by a "404 - 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 <
> > >>>> nick.telford@gmail.com>
> > >>>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> Addendum:
> > >>>>>>>>>>
> > >>>>>>>>>> I think we would also face the same problem with the approach
> > John
> > >>>>>>>> outlined
> > >>>>>>>>>> earlier (using the record cache as a transaction buffer and
> > >>>> flushing
> > >>>>>>>> it
> > >>>>>>>>>> straight to SST files). This is because the record cache (the
> > >>>>>>>> ThreadCache
> > >>>>>>>>>> class) is not thread-safe, so every commit would invalidate open
> > >> IQ
> > >>>>>>>>>> Iterators in the same way that RocksDB WriteBatches do.
> > >>>>>>>>>> --
> > >>>>>>>>>> Nick
> > >>>>>>>>>>
> > >>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > >>>> nick.telford@gmail.com>
> > >>>>>>>>>> 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.
> > >>>>>>>>>>> 2. Making IsolationLevel a query-time constraint, rather than
> > >>>>>>>> linking it
> > >>>>>>>>>>> to the processing.guarantee.
> > >>>>>>>>>>>
> > >>>>>>>>>>> There's a wrinkle that makes this a challenge: Interactive
> > >> Queries
> > >>>>>>>> that
> > >>>>>>>>>>> open an Iterator, when using transactions and READ_UNCOMMITTED.
> > >>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries need to be
> > >>>> able
> > >>>>>>>> to
> > >>>>>>>>>>> read records from the currently uncommitted transaction buffer
> > >>>>>>>>>>> (WriteBatch). This includes for Iterators, which should iterate
> > >>>>>>>> both the
> > >>>>>>>>>>> transaction buffer and underlying database (using
> > >>>>>>>>>>> WriteBatch#iteratorWithBase()).
> > >>>>>>>>>>>
> > >>>>>>>>>>> The issue is that when the StreamThread commits, it writes the
> > >>>>>>>> current
> > >>>>>>>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*.
> > Clearing
> > >>>> the
> > >>>>>>>>>>> WriteBatch while an Interactive Query holds an open Iterator on
> > >> it
> > >>>>>>>> will
> > >>>>>>>>>>> invalidate the Iterator. Worse, it turns out that Iterators
> > over
> > >> a
> > >>>>>>>>>>> WriteBatch become invalidated not just when the WriteBatch is
> > >>>>>>>> cleared,
> > >>>>>>>>>> but
> > >>>>>>>>>>> also when the Iterators' current key receives a new write.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Now that I'm writing this, I remember that this is the major
> > >>>> reason
> > >>>>>>>> that
> > >>>>>>>>>> I
> > >>>>>>>>>>> switched the original design from having a query-time
> > >>>>>>>> IsolationLevel to
> > >>>>>>>>>>> having the IsolationLevel linked to the transactionality of the
> > >>>>>>>> stores
> > >>>>>>>>>>> themselves.
> > >>>>>>>>>>>
> > >>>>>>>>>>> It *might* be possible to resolve this, by having a "chain" of
> > >>>>>>>>>>> WriteBatches, with the StreamThread switching to a new
> > WriteBatch
> > >>>>>>>>>> whenever
> > >>>>>>>>>>> a new Interactive Query attempts to read from the database, but
> > >>>> that
> > >>>>>>>>>> could
> > >>>>>>>>>>> cause some performance problems/memory pressure when subjected
> > to
> > >>>> a
> > >>>>>>>> high
> > >>>>>>>>>>> Interactive Query load. It would also reduce the efficiency of
> > >>>>>>>>>> WriteBatches
> > >>>>>>>>>>> on-commit, as we'd have to write N WriteBatches, where N is the
> > >>>>>>>> number of
> > >>>>>>>>>>> Interactive Queries since the last commit.
> > >>>>>>>>>>>
> > >>>>>>>>>>> I realise this is getting into the weeds of the implementation,
> > >>>> and
> > >>>>>>>> you'd
> > >>>>>>>>>>> rather we focus on the API for now, but I think it's important
> > to
> > >>>>>>>>>> consider
> > >>>>>>>>>>> how to implement the desired API, in case we come up with an
> > API
> > >>>>>>>> that
> > >>>>>>>>>>> cannot be implemented efficiently, or even at all!
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thoughts?
> > >>>>>>>>>>> --
> > >>>>>>>>>>> Nick
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> > cadonna@apache.org
> > >>>
> > >>>>>>>> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>>> Hi Nick,
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> 6.
> > >>>>>>>>>>>> Of course, you are right! My bad!
> > >>>>>>>>>>>> Wiping out the state in the downgrading case is fine.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> 3a.
> > >>>>>>>>>>>> Focus on the public facing changes for the KIP. We will manage
> > >> 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.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Best,
> > >>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > >>>>>>>>>>>>> Hi Bruno,
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Thanks for getting back to me!
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>> The fact that implementations can always track estimated
> > memory
> > >>>>>>>> usage
> > >>>>>>>>>> in
> > >>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as an option,
> > and
> > >>>>>>>> I'll
> > >>>>>>>>>>>> clarify
> > >>>>>>>>>>>>> the JavaDoc that 0 is not just for non-transactional stores,
> > >>>>>>>> which is
> > >>>>>>>>>>>>> currently misleading.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> 6.
> > >>>>>>>>>>>>> The problem with catching the exception in the downgrade
> > >> process
> > >>>>>>>> is
> > >>>>>>>>>> that
> > >>>>>>>>>>>>> would require new code in the Kafka version being downgraded
> > >> to.
> > >>>>>>>> Since
> > >>>>>>>>>>>>> users could conceivably downgrade to almost *any* older
> > version
> > >>>>>>>> of
> > >>>>>>>>>> Kafka
> > >>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
> > >>>>>>>>>>>>> The only way I can think of doing it would be to provide a
> > >>>>>>>> dedicated
> > >>>>>>>>>>>>> downgrade tool, that goes through every local store and
> > removes
> > >>>>>>>> the
> > >>>>>>>>>>>>> offsets column families. But that seems like an unnecessary
> > >>>>>>>> amount of
> > >>>>>>>>>>>> extra
> > >>>>>>>>>>>>> code to maintain just to handle a somewhat niche situation,
> > >> when
> > >>>>>>>> the
> > >>>>>>>>>>>>> alternative (automatically wipe and restore stores) should be
> > >>>>>>>>>>>> acceptable.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've requested.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> 3a.
> > >>>>>>>>>>>>> I agree that IsolationLevel makes more sense at query-time,
> > and
> > >>>> I
> > >>>>>>>>>>>> actually
> > >>>>>>>>>>>>> initially attempted to place the IsolationLevel at
> > query-time,
> > >>>>>>>> but I
> > >>>>>>>>>> ran
> > >>>>>>>>>>>>> into some problems:
> > >>>>>>>>>>>>> - The key issue is that, under ALOS we're not staging writes
> > in
> > >>>>>>>>>>>>> transactions, so can't perform writes at the READ_COMMITTED
> > >>>>>>>> isolation
> > >>>>>>>>>>>>> level. However, this may be addressed if we decide to
> > *always*
> > >>>>>>>> use
> > >>>>>>>>>>>>> transactions as discussed under 3b.
> > >>>>>>>>>>>>> - IQv1 and IQv2 have quite different implementations. I
> > >> remember
> > >>>>>>>>>> having
> > >>>>>>>>>>>>> some difficulty understanding the IQv1 internals, which made
> > it
> > >>>>>>>>>>>> difficult
> > >>>>>>>>>>>>> to determine what needed to be changed. However, I *think*
> > this
> > >>>>>>>> can be
> > >>>>>>>>>>>>> addressed for both implementations by wrapping the
> > RocksDBStore
> > >>>>>>>> in an
> > >>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides read methods
> > >>>>>>>> (get,
> > >>>>>>>>>>>> etc.)
> > >>>>>>>>>>>>> to either read directly from the database or from the ongoing
> > >>>>>>>>>>>> transaction.
> > >>>>>>>>>>>>> But IQv1 might still be difficult.
> > >>>>>>>>>>>>> - If IsolationLevel becomes a query constraint, then all
> > other
> > >>>>>>>>>>>> StateStores
> > >>>>>>>>>>>>> will need to respect it, including the in-memory stores. This
> > >>>>>>>> would
> > >>>>>>>>>>>> require
> > >>>>>>>>>>>>> us to adapt in-memory stores to stage their writes so they
> > can
> > >>>> be
> > >>>>>>>>>>>> isolated
> > >>>>>>>>>>>>> from READ_COMMITTTED queries. It would also become an
> > important
> > >>>>>>>>>>>>> consideration for third-party stores on upgrade, as without
> > >>>>>>>> changes,
> > >>>>>>>>>>>> they
> > >>>>>>>>>>>>> would not support READ_COMMITTED queries correctly.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Ultimately, I may need some help making the necessary change
> > to
> > >>>>>>>> IQv1
> > >>>>>>>>>> to
> > >>>>>>>>>>>>> support this, but I don't think it's fundamentally
> > impossible,
> > >>>>>>>> if we
> > >>>>>>>>>>>> want
> > >>>>>>>>>>>>> to pursue this route.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> 3b.
> > >>>>>>>>>>>>> The main reason I chose to keep ALOS un-transactional was to
> > >>>>>>>> minimize
> > >>>>>>>>>>>>> behavioural change for most users (I believe most Streams
> > users
> > >>>>>>>> use
> > >>>>>>>>>> the
> > >>>>>>>>>>>>> default configuration, which is ALOS). That said, it's clear
> > >>>>>>>> that if
> > >>>>>>>>>>>> ALOS
> > >>>>>>>>>>>>> also used transactional stores, the only change in behaviour
> > >>>>>>>> would be
> > >>>>>>>>>>>> that
> > >>>>>>>>>>>>> it would become *more correct*, which could be considered a
> > >> "bug
> > >>>>>>>> fix"
> > >>>>>>>>>> by
> > >>>>>>>>>>>>> users, rather than a change they need to handle.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> I believe that performance using transactions (aka. RocksDB
> > >>>>>>>>>>>> WriteBatches)
> > >>>>>>>>>>>>> should actually be *better* than the un-batched write-path
> > that
> > >>>>>>>> is
> > >>>>>>>>>>>>> currently used[1]. The only "performance" consideration will
> > be
> > >>>>>>>> the
> > >>>>>>>>>>>>> increased memory usage that transactions require. Given the
> > >>>>>>>>>> mitigations
> > >>>>>>>>>>>> for
> > >>>>>>>>>>>>> this memory that we have in place, I would expect that this
> > is
> > >>>>>>>> not a
> > >>>>>>>>>>>>> problem for most users.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> If we're happy to do so, we can make ALOS also use
> > >> transactions.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Link 1:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>
> > >>>>>
> > >> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> > >>>> cadonna@apache.org
> > >>>>>>>>>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Hi Nick,
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Thanks for the updates and sorry for the delay on my side!
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>> Making the default implementation for flush() a no-op sounds
> > >>>>>>>> good to
> > >>>>>>>>>>>> me.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>> I think what was bugging me here is that a third-party state
> > >>>>>>>> store
> > >>>>>>>>>>>> needs
> > >>>>>>>>>>>>>> to implement the state store interface. That means they need
> > >> to
> > >>>>>>>>>>>>>> implement a wrapper around the actual state store as we do
> > for
> > >>>>>>>>>> RocksDB
> > >>>>>>>>>>>>>> with RocksDBStore. So, a third-party state store can always
> > >>>>>>>> estimate
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>> uncommitted bytes, if it wants, because the wrapper can
> > record
> > >>>>>>>> the
> > >>>>>>>>>>>> added
> > >>>>>>>>>>>>>> bytes.
> > >>>>>>>>>>>>>> One case I can think of where returning -1 makes sense is
> > when
> > >>>>>>>>>> Streams
> > >>>>>>>>>>>>>> does not need to estimate the size of the write batch and
> > >>>>>>>> trigger
> > >>>>>>>>>>>>>> extraordinary commits, because the third-party state store
> > >>>>>>>> takes care
> > >>>>>>>>>>>> of
> > >>>>>>>>>>>>>> memory. But in that case the method could also just return
> > 0.
> > >>>>>>>> Even
> > >>>>>>>>>> that
> > >>>>>>>>>>>>>> case would be better solved with a method that returns
> > whether
> > >>>>>>>> the
> > >>>>>>>>>>>> state
> > >>>>>>>>>>>>>> store manages itself the memory used for uncommitted bytes
> > or
> > >>>>>>>> not.
> > >>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return value, I was
> > >>>>>>>> just
> > >>>>>>>>>>>>>> wondering when and if it will be used.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Regarding returning 0 for transactional state stores when
> > the
> > >>>>>>>> batch
> > >>>>>>>>>> is
> > >>>>>>>>>>>>>> empty, I was just wondering because you explicitly stated
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> "or {@code 0} if this StateStore does not support
> > >>>> transactions."
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> So it seemed to me returning 0 could only happen for
> > >>>>>>>>>> non-transactional
> > >>>>>>>>>>>>>> state stores.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> a) What do you think if we move the isolation level to IQ
> > (v1
> > >>>>>>>> and
> > >>>>>>>>>> v2)?
> > >>>>>>>>>>>>>> In the end this is the only component that really needs to
> > >>>>>>>> specify
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>> isolation level. It is similar to the Kafka consumer that
> > can
> > >>>>>>>> choose
> > >>>>>>>>>>>>>> with what isolation level to read the input topic.
> > >>>>>>>>>>>>>> For IQv1 the isolation level should go into
> > >>>>>>>> StoreQueryParameters. For
> > >>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> b) Point a) raises the question what should happen during
> > >>>>>>>>>> at-least-once
> > >>>>>>>>>>>>>> processing when the state store does not use transactions?
> > >> John
> > >>>>>>>> in
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>> past proposed to also use transactions on state stores for
> > >>>>>>>>>>>>>> at-least-once. I like that idea, because it avoids
> > aggregating
> > >>>>>>>> the
> > >>>>>>>>>> same
> > >>>>>>>>>>>>>> records over and over again in the case of a failure. We
> > had a
> > >>>>>>>> case
> > >>>>>>>>>> in
> > >>>>>>>>>>>>>> the past where a Streams applications in at-least-once mode
> > >> was
> > >>>>>>>>>> failing
> > >>>>>>>>>>>>>> continuously for some reasons I do not remember before
> > >>>>>>>> committing the
> > >>>>>>>>>>>>>> offsets. After each failover, the app aggregated again and
> > >>>>>>>> again the
> > >>>>>>>>>>>>>> same records. Of course the aggregate increased to very
> > wrong
> > >>>>>>>> values
> > >>>>>>>>>>>>>> just because of the failover. With transactions on the state
> > >>>>>>>> stores
> > >>>>>>>>>> we
> > >>>>>>>>>>>>>> could have avoided this. The app would have output the same
> > >>>>>>>> aggregate
> > >>>>>>>>>>>>>> multiple times (i.e., after each failover) but at least the
> > >>>>>>>> value of
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>> aggregate would not depend on the number of failovers.
> > >>>>>>>> Outputting the
> > >>>>>>>>>>>>>> same aggregate multiple times would be incorrect under
> > >>>>>>>> exactly-once
> > >>>>>>>>>> but
> > >>>>>>>>>>>>>> it is OK for at-least-once.
> > >>>>>>>>>>>>>> If it makes sense to add a config to turn on and off
> > >>>>>>>> transactions on
> > >>>>>>>>>>>>>> state stores under at-least-once or just use transactions in
> > >>>>>>>> any case
> > >>>>>>>>>>>> is
> > >>>>>>>>>>>>>> a question we should also discuss in this KIP. It depends a
> > >> bit
> > >>>>>>>> on
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would add a
> > config.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> 4.
> > >>>>>>>>>>>>>> Your points are all valid. I tend to say to keep the metrics
> > >>>>>>>> around
> > >>>>>>>>>>>>>> flush() until we remove flush() completely from the
> > interface.
> > >>>>>>>> Calls
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>> flush() might still exist since existing processors might
> > >> still
> > >>>>>>>> call
> > >>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For sure, we need
> > >> to
> > >>>>>>>>>>>> document
> > >>>>>>>>>>>>>> how the metrics change due to the transactions in the
> > upgrade
> > >>>>>>>> notes.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> 5.
> > >>>>>>>>>>>>>> I see. Then you should describe how the .position files are
> > >>>>>>>> handled
> > >>>>>>>>>> in
> > >>>>>>>>>>>>>> a dedicated section of the KIP or incorporate the
> > description
> > >>>>>>>> in the
> > >>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only mentioning it
> > >> in
> > >>>>>>>> the
> > >>>>>>>>>>>>>> "Compatibility, 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:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>
> > >>>>>
> > >>>>
> > >>
> > https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> It is a bit brittle, but it works.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > >>>>>>>>>>>>>>> Hi Bruno,
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thanks for taking the time to review the KIP. I'm back from
> > >>>>>>>> leave
> > >>>>>>>>>> now
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>> intend to move this forwards as quickly as I can.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Addressing your points:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>> Because flush() is part of the StateStore API, it's exposed
> > >> to
> > >>>>>>>>>> custom
> > >>>>>>>>>>>>>>> Processors, which might be making calls to flush(). This
> > was
> > >>>>>>>>>> actually
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>> case in a few integration tests.
> > >>>>>>>>>>>>>>> To maintain as much compatibility as possible, I'd prefer
> > not
> > >>>>>>>> to
> > >>>>>>>>>> make
> > >>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>> an UnsupportedOperationException, as it will cause
> > previously
> > >>>>>>>>>> working
> > >>>>>>>>>>>>>>> Processors to start throwing exceptions at runtime.
> > >>>>>>>>>>>>>>> I agree that it doesn't make sense for it to proxy
> > commit(),
> > >>>>>>>> though,
> > >>>>>>>>>>>> as
> > >>>>>>>>>>>>>>> that would cause it to violate the "StateStores commit only
> > >>>>>>>> when the
> > >>>>>>>>>>>> Task
> > >>>>>>>>>>>>>>> commits" rule.
> > >>>>>>>>>>>>>>> Instead, I think we should make this a no-op. That way,
> > >>>>>>>> existing
> > >>>>>>>>>> user
> > >>>>>>>>>>>>>>> Processors will continue to work as-before, without
> > violation
> > >>>>>>>> of
> > >>>>>>>>>> store
> > >>>>>>>>>>>>>>> consistency that would be caused by premature flush/commit
> > of
> > >>>>>>>>>>>> StateStore
> > >>>>>>>>>>>>>>> data to disk.
> > >>>>>>>>>>>>>>> What do you think?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore implementation
> > is
> > >>>>>>>>>>>>>>> transactional, but is unable to estimate the uncommitted
> > >>>> memory
> > >>>>>>>>>> usage,
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>> method will return -1.
> > >>>>>>>>>>>>>>> The intention here is to permit third-party implementations
> > >>>>>>>> that may
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>> able to estimate memory usage.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been written to the
> > store
> > >>>>>>>> yet. I
> > >>>>>>>>>>>>>> thought
> > >>>>>>>>>>>>>>> that was implied by "This method will return an
> > approximation
> > >>>>>>>> of the
> > >>>>>>>>>>>>>> memory
> > >>>>>>>>>>>>>>> would be freed by the next call to {@link #commit(Map)}"
> > and
> > >>>>>>>>>> "@return
> > >>>>>>>>>>>> The
> > >>>>>>>>>>>>>>> approximate size of all records awaiting {@link
> > >>>> #commit(Map)}",
> > >>>>>>>>>>>> however,
> > >>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you think this is
> > >>>>>>>> unclear?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>> I realise this is probably the most contentious point in my
> > >>>>>>>> design,
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>> I'm
> > >>>>>>>>>>>>>>> open to changing it if I'm unable to convince you of the
> > >>>>>>>> benefits.
> > >>>>>>>>>>>>>>> Nevertheless, here's my argument:
> > >>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly provided
> > >>>>>>>> StateStores
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>> query, and it may be important for users to
> > programmatically
> > >>>>>>>> know
> > >>>>>>>>>>>> which
> > >>>>>>>>>>>>>>> mode the StateStore is operating under. If we simply
> > provide
> > >>>> an
> > >>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the internal
> > streams
> > >>>>>>>>>>>> engine), or
> > >>>>>>>>>>>>>>> similar, then users will need to understand the operation
> > and
> > >>>>>>>>>>>>>> consequences
> > >>>>>>>>>>>>>>> of each available processing mode and how it pertains to
> > >> their
> > >>>>>>>>>>>>>> StateStore.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Interactive Query users aren't the only people that care
> > >> about
> > >>>>>>>> the
> > >>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
> > implementers
> > >>>> of
> > >>>>>>>>>> custom
> > >>>>>>>>>>>>>>> StateStores also need to understand the behaviour expected
> > of
> > >>>>>>>> their
> > >>>>>>>>>>>>>>> implementation. KIP-892 introduces some assumptions into
> > the
> > >>>>>>>> Streams
> > >>>>>>>>>>>>>> Engine
> > >>>>>>>>>>>>>>> about how StateStores operate under each processing mode,
> > and
> > >>>>>>>> it's
> > >>>>>>>>>>>>>>> important that custom implementations adhere to those
> > >>>>>>>> assumptions in
> > >>>>>>>>>>>>>> order
> > >>>>>>>>>>>>>>> to maintain the consistency guarantees.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> IsolationLevels provide a high-level contract on the
> > >> behaviour
> > >>>>>>>> of
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>> StateStore: a user knows that under READ_COMMITTED, they
> > will
> > >>>>>>>> see
> > >>>>>>>>>>>> writes
> > >>>>>>>>>>>>>>> only after the Task has committed, and under
> > READ_UNCOMMITTED
> > >>>>>>>> they
> > >>>>>>>>>>>> will
> > >>>>>>>>>>>>>> see
> > >>>>>>>>>>>>>>> writes immediately. No understanding of the details of each
> > >>>>>>>>>>>>>> processing.mode
> > >>>>>>>>>>>>>>> is required, either for IQ users or StateStore
> > implementers.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> An argument can be made that these contractual guarantees
> > can
> > >>>>>>>> simply
> > >>>>>>>>>>>> be
> > >>>>>>>>>>>>>>> documented for the processing.mode (i.e. that exactly-once
> > >> and
> > >>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
> > at-least-once
> > >>>>>>>> behaves
> > >>>>>>>>>>>> like
> > >>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small issues with
> > >>>>>>>> this I'd
> > >>>>>>>>>>>>>> prefer
> > >>>>>>>>>>>>>>> to avoid:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>         - Where would we document these contracts, in a way
> > >> that
> > >>>>>>>> is
> > >>>>>>>>>>>> difficult
> > >>>>>>>>>>>>>>>         for users/implementers to miss/ignore?
> > >>>>>>>>>>>>>>>         - It's not clear to users that the processing mode
> > is
> > >>>>>>>>>>>> communicating
> > >>>>>>>>>>>>>>>         an expectation of read isolation, unless they read
> > the
> > >>>>>>>>>>>>>> documentation. Users
> > >>>>>>>>>>>>>>>         rarely consult documentation unless they feel they
> > >> need
> > >>>>>>>> to, so
> > >>>>>>>>>>>> it's
> > >>>>>>>>>>>>>> likely
> > >>>>>>>>>>>>>>>         this detail would get missed by many users.
> > >>>>>>>>>>>>>>>         - It tightly couples processing modes to read
> > >> isolation.
> > >>>>>>>> Adding
> > >>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>         processing modes, or changing the read isolation of
> > >>>>>>>> existing
> > >>>>>>>>>>>>>> processing
> > >>>>>>>>>>>>>>>         modes would be difficult/impossible.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Ultimately, the cost of introducing IsolationLevels is
> > just a
> > >>>>>>>> single
> > >>>>>>>>>>>>>>> method, since we re-use the existing IsolationLevel enum
> > from
> > >>>>>>>> Kafka.
> > >>>>>>>>>>>> This
> > >>>>>>>>>>>>>>> gives us a clear place to document the contractual
> > guarantees
> > >>>>>>>>>> expected
> > >>>>>>>>>>>>>>> of/provided by StateStores, that is accessible both by the
> > >>>>>>>>>> StateStore
> > >>>>>>>>>>>>>>> itself, and by IQ users.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> (Writing this I've just realised that the StateStore and IQ
> > >>>>>>>> APIs
> > >>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>> don't provide access to StateStoreContext that IQ users
> > would
> > >>>>>>>> have
> > >>>>>>>>>>>> direct
> > >>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> > >> isolationLevel()
> > >>>>>>>>>> itself
> > >>>>>>>>>>>>>> too?)
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> 4.
> > >>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics in-place
> > >>>>>>>> either, as
> > >>>>>>>>>>>> it's a
> > >>>>>>>>>>>>>>> backwards incompatible change. My concern is that, if we
> > >> leave
> > >>>>>>>> the
> > >>>>>>>>>>>>>> existing
> > >>>>>>>>>>>>>>> "flush" metrics in place, they will be confusing to users.
> > >>>>>>>> Right
> > >>>>>>>>>> now,
> > >>>>>>>>>>>>>>> "flush" metrics record explicit flushes to disk, but under
> > >>>>>>>> KIP-892,
> > >>>>>>>>>>>> even
> > >>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>> commit() will not explicitly flush data to disk - RocksDB
> > >> will
> > >>>>>>>>>> decide
> > >>>>>>>>>>>> on
> > >>>>>>>>>>>>>>> when to flush memtables to disk itself.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd have two
> > >>>> options,
> > >>>>>>>>>> which
> > >>>>>>>>>>>>>> both
> > >>>>>>>>>>>>>>> seem pretty bad to me:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>         1. Have them record calls to commit(), which would
> > be
> > >>>>>>>>>>>> misleading, as
> > >>>>>>>>>>>>>>>         data is no longer explicitly "flushed" to disk by
> > this
> > >>>>>>>> call.
> > >>>>>>>>>>>>>>>         2. Have them record nothing at all, which is
> > >> equivalent
> > >>>> to
> > >>>>>>>>>>>> removing
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>         metrics, except that users will see the metric
> > still
> > >>>>>>>> exists and
> > >>>>>>>>>>>> so
> > >>>>>>>>>>>>>> assume
> > >>>>>>>>>>>>>>>         that the metric is correct, and that there's a
> > problem
> > >>>>>>>> with
> > >>>>>>>>>> their
> > >>>>>>>>>>>>>> system
> > >>>>>>>>>>>>>>>         when there isn't.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> I agree that removing them is also a bad solution, and I'd
> > >>>>>>>> like some
> > >>>>>>>>>>>>>>> guidance on the best path forward here.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> 5.
> > >>>>>>>>>>>>>>> Position files are updated on every write to a StateStore.
> > >>>>>>>> Since our
> > >>>>>>>>>>>>>> writes
> > >>>>>>>>>>>>>>> are now buffered until commit(), we can't update the
> > Position
> > >>>>>>>> file
> > >>>>>>>>>>>> until
> > >>>>>>>>>>>>>>> commit() has been called, otherwise it would be
> > inconsistent
> > >>>>>>>> with
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>> data
> > >>>>>>>>>>>>>>> in the event of a rollback. Consequently, we need to manage
> > >>>>>>>> these
> > >>>>>>>>>>>> offsets
> > >>>>>>>>>>>>>>> the same way we manage the checkpoint offsets, and ensure
> > >>>>>>>> they're
> > >>>>>>>>>> only
> > >>>>>>>>>>>>>>> written on commit().
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> 6.
> > >>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what tests to
> > >> write.
> > >>>>>>>> How
> > >>>>>>>>>>>>>> explicit
> > >>>>>>>>>>>>>>> do we need to be here in the KIP?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be
> > seamless,
> > >>>>>>>> and we
> > >>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>> definitely add some tests around that. Downgrade, it
> > >>>>>>>> transpires,
> > >>>>>>>>>> isn't
> > >>>>>>>>>>>>>>> currently possible, as the extra column family for offset
> > >>>>>>>> storage is
> > >>>>>>>>>>>>>>> incompatible with the pre-KIP-892 implementation: when you
> > >>>>>>>> open a
> > >>>>>>>>>>>> RocksDB
> > >>>>>>>>>>>>>>> database, you must open all available column families or
> > >>>>>>>> receive an
> > >>>>>>>>>>>>>> error.
> > >>>>>>>>>>>>>>> What currently happens on downgrade is that it attempts to
> > >>>>>>>> open the
> > >>>>>>>>>>>>>> store,
> > >>>>>>>>>>>>>>> throws an error about the offsets column family not being
> > >>>>>>>> opened,
> > >>>>>>>>>>>> which
> > >>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given that
> > >> downgrades
> > >>>>>>>>>> should
> > >>>>>>>>>>>> be
> > >>>>>>>>>>>>>>> uncommon, I think this is acceptable behaviour, as the
> > >>>>>>>> end-state is
> > >>>>>>>>>>>>>>> consistent, even if it results in an undesirable state
> > >>>> restore.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Should I document the upgrade/downgrade behaviour
> > explicitly
> > >>>>>>>> in the
> > >>>>>>>>>>>> KIP?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > >>>>>>>> cadonna@apache.org>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Hi Nick!
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Thanks for the updates!
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> > >>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > >>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore after this
> > KIP
> > >>>>>>>> is
> > >>>>>>>>>>>>>>>> released, I would rather throw an unsupported operation
> > >>>>>>>> exception
> > >>>>>>>>>> by
> > >>>>>>>>>>>>>>>> default.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>>> When would a state store return -1 from
> > >>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while being
> > >>>>>>>>>>>> transactional?
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Wouldn't StateStore#approximateNumUncommittedBytes() also
> > >>>>>>>> return 0
> > >>>>>>>>>> if
> > >>>>>>>>>>>>>>>> the state store is transactional but nothing has been
> > >> written
> > >>>>>>>> to
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>> state store yet?
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this KIP really
> > need
> > >>>> to
> > >>>>>>>>>>>> introduce
> > >>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()? StateStoreContext has
> > >>>>>>>> already
> > >>>>>>>>>>>>>>>> appConfigs() which basically exposes the same information,
> > >>>>>>>> i.e., if
> > >>>>>>>>>>>> EOS
> > >>>>>>>>>>>>>>>> is enabled or not.
> > >>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore interface as
> > >>>>>>>> loosely
> > >>>>>>>>>>>> coupled
> > >>>>>>>>>>>>>>>> from the Streams engine as possible, to give implementers
> > >>>> more
> > >>>>>>>>>>>> freedom,
> > >>>>>>>>>>>>>>>> and reduce the amount of internal knowledge required."
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> While I understand the intent, I doubt that it decreases
> > the
> > >>>>>>>>>>>> coupling of
> > >>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> > >> READ_COMMITTED
> > >>>>>>>> only
> > >>>>>>>>>>>>>>>> applies to IQ but not to reads by processors. Thus,
> > >>>>>>>> implementers
> > >>>>>>>>>>>> need to
> > >>>>>>>>>>>>>>>> understand how Streams accesses the state stores.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I would like to hear what others think about this.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> 4.
> > >>>>>>>>>>>>>>>> Great exposing new metrics for transactional state stores!
> > >>>>>>>>>> However, I
> > >>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate (in the
> > docs)
> > >>>>>>>> the 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 also need to adapt/add integration tests
> > >>>>>>>> besides
> > >>>>>>>>>> unit
> > >>>>>>>>>>>>>>>> tests. Additionally, we probably need integration or
> > system
> > >>>>>>>> tests
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>>> verify that upgrades and downgrades between transactional
> > >> and
> > >>>>>>>>>>>>>>>> non-transactional state stores work as expected.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > >>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in the KIP,
> > under
> > >>>>>>>>>>>> "Rejected
> > >>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea worth pursuing,
> > >>>>>>>> but I
> > >>>>>>>>>>>> believe
> > >>>>>>>>>>>>>>>>> that it's out of the scope of this KIP, because it
> > solves a
> > >>>>>>>>>>>> different
> > >>>>>>>>>>>>>> set
> > >>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of this one has
> > >>>>>>>> already
> > >>>>>>>>>> grown
> > >>>>>>>>>>>>>>>> quite
> > >>>>>>>>>>>>>>>>> large!
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > >>>>>>>>>> nick.telford@gmail.com>
> > >>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Hi everyone,
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I've updated the KIP (
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>
> > >>>>>
> > >>>>
> > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > >>>>>>>>>>>>>>>> )
> > >>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing back "Atomic
> > >>>>>>>>>>>> Checkpointing"
> > >>>>>>>>>>>>>>>> (for
> > >>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think the one thing
> > >>>>>>>> missing is
> > >>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>> changes to metrics (notably the store "flush" metrics
> > will
> > >>>>>>>> need
> > >>>>>>>>>> to
> > >>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>> renamed to "commit").
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> The reason I brought back Atomic Checkpointing was to
> > >>>>>>>> decouple
> > >>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>> flush
> > >>>>>>>>>>>>>>>>>> from store commit. This is important, because with
> > >>>>>>>> Transactional
> > >>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on *every* Task
> > >>>>>>>> commit,
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>> just when the StateStore is closing, otherwise our
> > >>>>>>>> transaction
> > >>>>>>>>>>>> buffer
> > >>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>> never be written and persisted, instead growing
> > unbounded!
> > >>>> I
> > >>>>>>>>>>>>>>>> experimented
> > >>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a store flush
> > >>>>>>>> whenever
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its configured
> > >>>>>>>> size, but
> > >>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>> was
> > >>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer from being
> > >>>>>>>>>> configured
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>> unbounded, and it still would have required explicit
> > >>>>>>>> flushes of
> > >>>>>>>>>>>>>> RocksDB,
> > >>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory utilization.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right" way to
> > >>>>>>>> resolve
> > >>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog offsets that
> > >>>>>>>> correspond
> > >>>>>>>>>> to
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>> most
> > >>>>>>>>>>>>>>>>>> recently written records are always atomically written
> > to
> > >>>>>>>> the
> > >>>>>>>>>>>>>> StateStore
> > >>>>>>>>>>>>>>>>>> (by writing them to the same transaction buffer), we can
> > >>>>>>>> avoid
> > >>>>>>>>>>>>>> forcibly
> > >>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting RocksDB
> > >>>>>>>> flush
> > >>>>>>>>>> them
> > >>>>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>> when necessary, without losing any of our consistency
> > >>>>>>>> guarantees.
> > >>>>>>>>>>>> See
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> updated KIP for more info.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I have fully implemented these changes, although I'm
> > still
> > >>>>>>>> not
> > >>>>>>>>>>>>>> entirely
> > >>>>>>>>>>>>>>>>>> happy with the implementation for segmented StateStores,
> > >> so
> > >>>>>>>> I
> > >>>>>>>>>> plan
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests pass. If you'd
> > like
> > >>>>>>>> to try
> > >>>>>>>>>>>> out
> > >>>>>>>>>>>>>> or
> > >>>>>>>>>>>>>>>>>> review this highly experimental and incomplete branch,
> > >> it's
> > >>>>>>>>>>>> available
> > >>>>>>>>>>>>>>>> here:
> > >>>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> > .
> > >>>>>>>> Note:
> > >>>>>>>>>>>> it's
> > >>>>>>>>>>>>>>>> built
> > >>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base to build
> > >>>>>>>> and test
> > >>>>>>>>>>>> it
> > >>>>>>>>>>>>>> on,
> > >>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples comparisons in a
> > live
> > >>>>>>>>>>>>>> environment. I
> > >>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's nearer
> > >> completion
> > >>>>>>>> and
> > >>>>>>>>>> has
> > >>>>>>>>>>>>>> been
> > >>>>>>>>>>>>>>>>>> proven on our main application.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing and testing:
> > >>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window) stores
> > >>>>>>>>>>>>>>>>>> - Global stores
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> As I do not currently use either of these, so my primary
> > >>>>>>>> test
> > >>>>>>>>>>>>>>>> environment
> > >>>>>>>>>>>>>>>>>> doesn't test these areas.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next week for a few
> > >>>>>>>> weeks,
> > >>>>>>>>>> so
> > >>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>> not have time to move this forward until late August.
> > That
> > >>>>>>>> said,
> > >>>>>>>>>>>> your
> > >>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just won't be
> > able
> > >>>> to
> > >>>>>>>>>>>> respond
> > >>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>> quickly as usual.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > >>>>>>>>>> nick.telford@gmail.com>
> > >>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Hi Bruno
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ is not
> > >>>>>>>> something
> > >>>>>>>>>> I
> > >>>>>>>>>>>> had
> > >>>>>>>>>>>>>>>>>>> considered.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> What about atomically updating the state store from the
> > >>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> > checkpoint
> > >>>>>>>> (thus,
> > >>>>>>>>>>>>>>>> flushing
> > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> > >>>>>>>> number of
> > >>>>>>>>>>>>>> commit
> > >>>>>>>>>>>>>>>>>>>> intervals?
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting that we
> > >>>>>>>> add an
> > >>>>>>>>>>>>>>>> additional
> > >>>>>>>>>>>>>>>>>>> config for the max number of commit intervals between
> > >>>>>>>>>> checkpoints?
> > >>>>>>>>>>>>>> That
> > >>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the transaction
> > >>>>>>>> buffers
> > >>>>>>>>>> are
> > >>>>>>>>>>>>>>>> nearly
> > >>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of commit
> > intervals
> > >>>>>>>> have
> > >>>>>>>>>>>>>> elapsed,
> > >>>>>>>>>>>>>>>>>>> whichever comes first?
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although this
> > re-ignites
> > >>>>>>>> an
> > >>>>>>>>>>>> earlier
> > >>>>>>>>>>>>>>>>>>> debate about whether a config should be measured in
> > >>>>>>>> "number of
> > >>>>>>>>>>>> commit
> > >>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute time.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the reason I was
> > >>>>>>>> pursuing
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>> Atomic
> > >>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush from
> > >>>>>>>> checkpointing,
> > >>>>>>>>>>>>>> which
> > >>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every commit without
> > any
> > >>>>>>>>>>>> performance
> > >>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the "best"
> > >>>>>>>> solution,
> > >>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>> I'm not
> > >>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back into this KIP.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> I'm currently working on moving all the transactional
> > >>>> logic
> > >>>>>>>>>>>> directly
> > >>>>>>>>>>>>>>>> into
> > >>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with the
> > >>>>>>>>>>>>>> StateStore#newTransaction
> > >>>>>>>>>>>>>>>>>>> method, and reduces the number of new classes
> > introduced,
> > >>>>>>>>>>>>>> significantly
> > >>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and the
> > complexity
> > >>>> is
> > >>>>>>>>>>>>>> drastically
> > >>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic Checkpoints
> > into
> > >>>>>>>> this
> > >>>>>>>>>> KIP.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
> > >>>>>>>> cadonna@apache.org>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Hi Nick,
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> As far as I understand, you want to atomically update
> > >> the
> > >>>>>>>> state
> > >>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the memtable of a
> > >>>> state
> > >>>>>>>>>> store
> > >>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit time elapsed
> > >>>> but
> > >>>>>>>>>> after
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that would lead to
> > >>>>>>>> exceeding
> > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes before the
> > next
> > >>>>>>>> commit
> > >>>>>>>>>>>>>>>> interval
> > >>>>>>>>>>>>>>>>>>>> ends.
> > >>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would commit every
> > >>>>>>>> commit
> > >>>>>>>>>>>> interval
> > >>>>>>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>>>>>> the state store will only be atomically updated
> > roughly
> > >>>>>>>> every
> > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of data. Also
> > IQ
> > >>>>>>>> would
> > >>>>>>>>>>>> then
> > >>>>>>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>>>> see new data roughly every
> > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > >>>>>>>>>>>>>>>>>>>> After a failure the state store needs to restore up to
> > >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Is this correct?
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> What about atomically updating the state store from
> > the
> > >>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> > checkpoint
> > >>>>>>>> (thus,
> > >>>>>>>>>>>>>>>> flushing
> > >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> > >>>>>>>> number of
> > >>>>>>>>>>>>>> commit
> > >>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have the same delay
> > >>>> for
> > >>>>>>>>>>>> records
> > >>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because both would
> > >>>>>>>> appear
> > >>>>>>>>>> when
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However, after a
> > failure
> > >>>>>>>> the
> > >>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>> still needs to restore up to
> > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > >>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>> it might restore data that is already in the state
> > store
> > >>>>>>>>>> because
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable offset (i.e.
> > the
> > >>>>>>>> last
> > >>>>>>>>>>>>>> committed
> > >>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring data that
> > is
> > >>>>>>>> already
> > >>>>>>>>>>>> in
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should not violated.
> > >>>>>>>>>>>>>>>>>>>> This solution needs at least one new config to specify
> > >>>>>>>> when a
> > >>>>>>>>>>>>>>>> checkpoint
> > >>>>>>>>>>>>>>>>>>>> should be written.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail that does
> > not
> > >>>>>>>> change
> > >>>>>>>>>>>>>>>> anything
> > >>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit interval is 30
> > >>>>>>>> seconds,
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>> five
> > >>>>>>>>>>>>>>>>>>>> seconds.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > >>>>>>>>>>>>>>>>>>>>> Hi everyone,
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch on our
> > staging
> > >>>>>>>>>>>>>> environment,
> > >>>>>>>>>>>>>>>>>>>>> putting it through its paces in our non-trivial
> > >>>>>>>> application.
> > >>>>>>>>>> I'm
> > >>>>>>>>>>>>>>>>>>>> already
> > >>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate that we saw
> > the
> > >>>>>>>> last
> > >>>>>>>>>>>> time
> > >>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP, but this
> > time,
> > >> I
> > >>>>>>>>>> think I
> > >>>>>>>>>>>>>> know
> > >>>>>>>>>>>>>>>>>>>> why.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called
> > at
> > >>>>>>>> the end
> > >>>>>>>>>>>> of
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> Task
> > >>>>>>>>>>>>>>>>>>>>> commit process, has the following behaviour:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>           - Under ALOS: checkpoint the state stores.
> > >> This
> > >>>>>>>>>> includes
> > >>>>>>>>>>>>>>>>>>>>>           flushing memtables in RocksDB. This is
> > >>>> acceptable
> > >>>>>>>>>>>> because the
> > >>>>>>>>>>>>>>>>>>>> default
> > >>>>>>>>>>>>>>>>>>>>>           commit.interval.ms is 5 seconds, so
> > forcibly
> > >>>>>>>> flushing
> > >>>>>>>>>>>>>> memtables
> > >>>>>>>>>>>>>>>>>>>> every 5
> > >>>>>>>>>>>>>>>>>>>>>           seconds is acceptable for most
> > applications.
> > >>>>>>>>>>>>>>>>>>>>>           - Under EOS: checkpointing is not done,
> > >> *unless*
> > >>>>>>>> it's
> > >>>>>>>>>>>> being
> > >>>>>>>>>>>>>>>>>>>> forced, due
> > >>>>>>>>>>>>>>>>>>>>>           to e.g. the Task closing or being revoked.
> > >> This
> > >>>>>>>> means
> > >>>>>>>>>>>> that
> > >>>>>>>>>>>>>> under
> > >>>>>>>>>>>>>>>>>>>> normal
> > >>>>>>>>>>>>>>>>>>>>>           processing conditions, the state stores
> > will
> > >> not
> > >>>>>>>> be
> > >>>>>>>>>>>>>>>> checkpointed,
> > >>>>>>>>>>>>>>>>>>>> and will
> > >>>>>>>>>>>>>>>>>>>>>           not have memtables flushed at all , unless
> > >>>> RocksDB
> > >>>>>>>>>>>> decides to
> > >>>>>>>>>>>>>>>>>>>> flush them on
> > >>>>>>>>>>>>>>>>>>>>>           its own. Checkpointing stores and
> > >> force-flushing
> > >>>>>>>> their
> > >>>>>>>>>>>>>> memtables
> > >>>>>>>>>>>>>>>>>>>> is only
> > >>>>>>>>>>>>>>>>>>>>>           done when a Task is being closed.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at
> > >>>> least
> > >>>>>>>>>> *some*
> > >>>>>>>>>>>>>>>> normal
> > >>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB
> > transaction
> > >>>>>>>>>> buffers
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets are synced to
> > >>>>>>>> disk to
> > >>>>>>>>>>>>>> prevent
> > >>>>>>>>>>>>>>>>>>>>> restores from getting out of hand. Consequently, my
> > >>>>>>>> current
> > >>>>>>>>>>>>>>>>>>>> implementation
> > >>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit, which
> > is
> > >>>>>>>> far too
> > >>>>>>>>>>>>>>>>>>>> frequent.
> > >>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records, which
> > is
> > >> a
> > >>>>>>>>>> change
> > >>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>> flush
> > >>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing performance problems
> > for
> > >>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>> applications.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm
> > currently
> > >>>>>>>> leaning
> > >>>>>>>>>>>>>>>> towards
> > >>>>>>>>>>>>>>>>>>>>> using the statestore.transaction.buffer.max.bytes
> > >>>>>>>>>> configuration
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to exceed it.
> > This
> > >>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>> complement the
> > >>>>>>>>>>>>>>>>>>>>> existing "early Task commit" functionality that this
> > >>>>>>>>>>>> configuration
> > >>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>           - Currently, we use
> > >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> > >>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> force an
> > >>>>>>>>>>>>>>>>>>>>>           early Task commit if processing more
> > records
> > >>>> would
> > >>>>>>>>>> cause
> > >>>>>>>>>>>> our
> > >>>>>>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>>           transactions to exceed the memory assigned
> > to
> > >>>>>>>> them.
> > >>>>>>>>>>>>>>>>>>>>>           - New functionality: when a Task *does*
> > >> commit,
> > >>>>>>>> we will
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>> checkpoint
> > >>>>>>>>>>>>>>>>>>>>>           the stores (and hence flush the transaction
> > >>>>>>>> buffers)
> > >>>>>>>>>>>> unless
> > >>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>> expect to
> > >>>>>>>>>>>>>>>>>>>>>           cross the
> > >>>> statestore.transaction.buffer.max.bytes
> > >>>>>>>>>>>> threshold
> > >>>>>>>>>>>>>>>> before
> > >>>>>>>>>>>>>>>>>>>> the next
> > >>>>>>>>>>>>>>>>>>>>>           commit
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> > >>>>>>>>>>>> nick.telford@gmail.com
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I meant in terms of
> > >>>>>>>>>>>> understanding
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> performance profile under various circumstances. The
> > >>>>>>>> more
> > >>>>>>>>>>>> complex
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would be for users
> > to
> > >>>>>>>>>>>> understand
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> performance they see. For example, spilling records
> > to
> > >>>>>>>> disk
> > >>>>>>>>>>>> when
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold would, I
> > >> expect,
> > >>>>>>>>>> reduce
> > >>>>>>>>>>>>>> write
> > >>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write throughput could
> > >> be
> > >>>>>>>>>>>>>> unexpected,
> > >>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>> potentially difficult to diagnose/understand for
> > >> users.
> > >>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early commit" concept is
> > >>>>>>>>>> relatively
> > >>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document, and
> > >>>> conceptually
> > >>>>>>>>>> fairly
> > >>>>>>>>>>>>>>>>>>>> obvious to
> > >>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to make it
> > >> easier
> > >>>>>>>> to
> > >>>>>>>>>>>>>>>> understand
> > >>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>> it happens though.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> > >>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an indirect way
> > of
> > >>>>>>>>>> telling
> > >>>>>>>>>>>>>>>>>>>> StateStores
> > >>>>>>>>>>>>>>>>>>>>>> whether they should be transactional. READ_COMMITTED
> > >>>>>>>>>>>> essentially
> > >>>>>>>>>>>>>>>>>>>> requires
> > >>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that two threads
> > >>>>>>>> calling
> > >>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes from the
> > >> other
> > >>>>>>>>>>>>>> transaction
> > >>>>>>>>>>>>>>>>>>>> until
> > >>>>>>>>>>>>>>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all
> > >>>>>>>> bets are
> > >>>>>>>>>>>> off,
> > >>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe written records
> > at
> > >>>>>>>> any
> > >>>>>>>>>>>> time,
> > >>>>>>>>>>>>>>>>>>>> which is
> > >>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That said,
> > StateStores
> > >>>>>>>> are
> > >>>>>>>>>> free
> > >>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>> implement
> > >>>>>>>>>>>>>>>>>>>>>> these guarantees however they can, which is a bit
> > more
> > >>>>>>>>>> relaxed
> > >>>>>>>>>>>>>> than
> > >>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions". For example,
> > >>>> with
> > >>>>>>>>>>>> RocksDB
> > >>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
> > >>>>>>>>>> "transactions",
> > >>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the database.
> > But
> > >>>>>>>> with
> > >>>>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>>>> storage
> > >>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to *always* use
> > >>>>>>>> transactions,
> > >>>>>>>>>>>> even
> > >>>>>>>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines that
> > >> don't
> > >>>>>>>>>> provide
> > >>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation guarantees can be
> > met
> > >>>>>>>> using a
> > >>>>>>>>>>>>>>>>>>>> different
> > >>>>>>>>>>>>>>>>>>>>>> technique.
> > >>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore interface
> > as
> > >>>>>>>>>> loosely
> > >>>>>>>>>>>>>>>> coupled
> > >>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > >>>>>>>> implementers
> > >>>>>>>>>> more
> > >>>>>>>>>>>>>>>>>>>> freedom, and
> > >>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge required.
> > >>>>>>>>>>>>>>>>>>>>>> That said, I understand that "IsolationLevel" might
> > >> not
> > >>>>>>>> be
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>> right
> > >>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it much more
> > >>>>>>>> explicit if
> > >>>>>>>>>>>>>>>>>>>> required, e.g.
> > >>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> 7-8.
> > >>>>>>>>>>>>>>>>>>>>>> I can make these changes either later today or
> > >>>> tomorrow.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Small update:
> > >>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of
> > >>>>>>>> issues
> > >>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>> needed
> > >>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass, which is
> > >>>>>>>>>> promising,
> > >>>>>>>>>>>> but
> > >>>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>>>>>>> need to undergo some performance testing. I haven't
> > >>>>>>>> (yet)
> > >>>>>>>>>>>> worked
> > >>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I would
> > >>>>>>>> expect
> > >>>>>>>>>> that,
> > >>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no difference. The
> > >> branch
> > >>>>>>>> is
> > >>>>>>>>>>>>>> available
> > >>>>>>>>>>>>>>>>>>>> at
> > >>>>>>>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> > >> if
> > >>>>>>>>>> anyone
> > >>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> > >>>>>>>>>>>> cadonna@apache.org>
> > >>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually also my
> > >>>>>>>> point. I
> > >>>>>>>>>>>>>>>> understood
> > >>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion path as a way
> > >> to
> > >>>>>>>> avoid
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> early
> > >>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually it is public
> > >>>>>>>> API. My
> > >>>>>>>>>>>>>> question
> > >>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal processing.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the size of the
> > >>>>>>>> transaction
> > >>>>>>>>>>>>>> buffer
> > >>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit affects the whole
> > >>>>>>>>>> processing
> > >>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>> Kafka
> > >>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a state store is
> > not
> > >>>>>>>>>>>> confined to
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and changes the
> > >>>>>>>> behavior
> > >>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you that it is a
> > >>>>>>>> decent
> > >>>>>>>>>>>>>>>>>>>> compromise. I
> > >>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and list the
> > >>>>>>>> options to
> > >>>>>>>>>>>>>>>> overcome
> > >>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all options seem
> > >>>> quite
> > >>>>>>>>>> heavy
> > >>>>>>>>>>>>>>>>>>>> compared
> > >>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you mean
> > with
> > >>>>>>>> "less
> > >>>>>>>>>>>>>>>>>>>> predictable
> > >>>>>>>>>>>>>>>>>>>>>>> for users", though.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the alternatives
> > really
> > >>>>>>>>>>>>>> interesting.
> > >>>>>>>>>>>>>>>>>>>> But I
> > >>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good and we should
> > >>>>>>>> continue
> > >>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>> it!
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on June
> > >> 20th:
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind putting
> > >>>>>>>> isolation
> > >>>>>>>>>>>>>> level
> > >>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should that also
> > be
> > >> a
> > >>>>>>>> way
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>> give
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to decide whether
> > to
> > >>>>>>>> turn on
> > >>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> > >>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned about how do
> > >> you
> > >>>>>>>> know
> > >>>>>>>>>>>> if a
> > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written under EOS, if
> > you
> > >>>>>>>> do not
> > >>>>>>>>>>>>>> have a
> > >>>>>>>>>>>>>>>>>>>> way
> > >>>>>>>>>>>>>>>>>>>>>>> to know if the state store is transactional or not.
> > >> If
> > >>>>>>>> a
> > >>>>>>>>>> state
> > >>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be written
> > >>>>>>>> during
> > >>>>>>>>>>>> normal
> > >>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state store is not
> > >>>>>>>>>> transactional,
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written under EOS.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 7.
> > >>>>>>>>>>>>>>>>>>>>>>> My point was about not only considering the bytes
> > in
> > >>>>>>>> memory
> > >>>>>>>>>> in
> > >>>>>>>>>>>>>>>> config
> > >>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes
> > that
> > >>>>>>>> might
> > >>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>> spilled
> > >>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering whether you
> > >> should
> > >>>>>>>>>> remove
> > >>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory bytes to be
> > >> used
> > >>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My
> > thinking
> > >>>>>>>> was
> > >>>>>>>>>> that
> > >>>>>>>>>>>>>> even
> > >>>>>>>>>>>>>>>>>>>> if a
> > >>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to disk,
> > >> limiting
> > >>>>>>>> the
> > >>>>>>>>>>>>>> overall
> > >>>>>>>>>>>>>>>>>>>> bytes
> > >>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it again and
> > >>>>>>>> considering
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>> recent
> > >>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much sense
> > anymore.
> > >>>>>>>>>>>>>>>>>>>>>>> I like the name
> > >>>>>>>> statestore.transaction.buffer.max.bytes that
> > >>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>> proposed.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> 8.
> > >>>>>>>>>>>>>>>>>>>>>>> A high-level description (without implementation
> > >>>>>>>> details) of
> > >>>>>>>>>>>> how
> > >>>>>>>>>>>>>>>>>>>> Kafka
> > >>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
> > >>>>>>>> transactions,
> > >>>>>>>>>>>> state
> > >>>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be great.
> > Would
> > >>>> be
> > >>>>>>>>>> great
> > >>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the behavior in
> > >>>>>>>> case of
> > >>>>>>>>>> a
> > >>>>>>>>>>>>>>>>>>>> failure.
> > >>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional state store
> > >>>>>>>> recover
> > >>>>>>>>>>>> after a
> > >>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the transaction
> > buffer,
> > >>>>>>>> etc.
> > >>>>>>>>>>>> (that
> > >>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>> what
> > >>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> > >>>>>>>> WriteBatchWithIndex
> > >>>>>>>>>>>>>>>>>>>> transactions
> > >>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
> > >>>>>>>> configured
> > >>>>>>>>>>>> memory
> > >>>>>>>>>>>>>>>>>>>> needs to
> > >>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns. Ultimately, any
> > >>>>>>>> approach
> > >>>>>>>>>>>> based
> > >>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>>>> cracking
> > >>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in ways it's
> > not
> > >>>>>>>> really
> > >>>>>>>>>>>>>>>> designed
> > >>>>>>>>>>>>>>>>>>>>>>> for is
> > >>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen performance or
> > >>>>>>>> consistency
> > >>>>>>>>>>>> issues.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing these early
> > >>>>>>>> commits?
> > >>>>>>>>>>>> While
> > >>>>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>>>>>>> ideal, I
> > >>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to ensure
> > >>>>>>>> consistency
> > >>>>>>>>>>>> whilst
> > >>>>>>>>>>>>>>>>>>>>>>> maintaining
> > >>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> > >>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
> > >>>>>>>> complicated, and
> > >>>>>>>>>>>> might
> > >>>>>>>>>>>>>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for users as a
> > >>>>>>>> consequence.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is
> > >>>>>>>> growing a
> > >>>>>>>>>>>> bit
> > >>>>>>>>>>>>>>>> out
> > >>>>>>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas for
> > future
> > >>>>>>>>>>>>>>>> improvements, I
> > >>>>>>>>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope down to a
> > design
> > >>>>>>>> that
> > >>>>>>>>>>>>>> achieves
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>> most
> > >>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized restorations
> > >>>>>>>> during
> > >>>>>>>>>> dirty
> > >>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that this KIP
> > >>>>>>>> produces
> > >>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>>>>>> ultimately
> > >>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially if the bulk
> > of
> > >>>>>>>> it is
> > >>>>>>>>>>>>>> internal
> > >>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week trying to
> > >>>>>>>> re-work
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> original
> > >>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
> > >>>>>>>> newTransaction()
> > >>>>>>>>>>>>>> method,
> > >>>>>>>>>>>>>>>>>>>> such
> > >>>>>>>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
> > RocksDBStore.
> > >>>>>>>> That
> > >>>>>>>>>>>> way, if
> > >>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>> want to
> > >>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the future, like
> > the
> > >>>>>>>> SST
> > >>>>>>>>>> file
> > >>>>>>>>>>>>>>>>>>>> management
> > >>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with little/no
> > >>>> API
> > >>>>>>>>>>>> changes.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >
> >


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

Posted by Nick Telford <ni...@gmail.com>.
Hi Bruno,

4.
I'll hold off on making that change until we have a consensus as to what
configuration to use to control all of this, as it'll be affected by the
decision on EOS isolation levels.

5.
Done. I've chosen "committedOffsets".

Regards,
Nick

On Fri, 13 Oct 2023 at 16:23, Bruno Cadonna <ca...@apache.org> wrote:

> Hi Nick,
>
> 1.
> Yeah, you are probably right that it does not make too much sense.
> Thanks for the clarification!
>
>
> 4.
> Yes, sorry for the back and forth, but I think for the sake of the KIP
> it is better to let the ALOS behavior as it is for now due to the
> possible issues you would run into. Maybe we can find a solution in the
> future. Now the question returns to whether we really need
> default.state.isolation.level. Maybe the config could be the feature
> flag Sophie requested.
>
>
> 5.
> There 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 committedOffsetFor().
>
>
> Best,
> Bruno
>
>
> On 10/13/23 3:21 PM, Nick Telford wrote:
> > Hi Bruno,
> >
> > Thanks for getting back to me.
> >
> > 1.
> > I think this should be possible. Are you thinking of the situation where
> a
> > user may downgrade to a previous version of Kafka Streams? In that case,
> > sadly, the RocksDBStore would get wiped by the older version of Kafka
> > Streams anyway, because that version wouldn't understand the extra column
> > family (that holds offsets), so the missing Position file would
> > automatically get rebuilt when the store is rebuilt from the changelog.
> > Are there other situations than downgrade where a transactional store
> could
> > be replaced by a non-transactional one? I can't think of any.
> >
> > 2.
> > Ahh yes, the Test Plan - my Kryptonite! This section definitely needs to
> be
> > fleshed out. I'll work on that. How much detail do you need?
> >
> > 3.
> > See my previous email discussing this.
> >
> > 4.
> > Hmm, this is an interesting point. Are you suggesting that under ALOS
> > READ_COMMITTED should not be supported?
> >
> > Regards,
> > Nick
> >
> > On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <ca...@apache.org> wrote:
> >
> >> Hi Nick,
> >>
> >> I think the KIP is converging!
> >>
> >>
> >> 1.
> >> I am wondering whether it makes sense to write the position file during
> >> close as we do for the checkpoint file, so that in case the state store
> >> is replaced with a non-transactional state store the non-transactional
> >> state store finds the position file. I think, this is not strictly
> >> needed, but would be a nice behavior instead of just deleting the
> >> position file.
> >>
> >>
> >> 2.
> >> The test plan does not mention integration tests. Do you not need to
> >> extend existing ones and add new ones. Also for upgrading and
> >> downgrading you might need integration and/or system tests.
> >>
> >>
> >> 3.
> >> I think Sophie made a point. Although, IQ reading from uncommitted data
> >> under EOS might be considered a bug by some people. Thus, your KIP would
> >> fix a bug rather than changing the intended behavior. However, I also
> >> see that a feature flag would help users that rely on this buggy
> >> behavior (at least until AK 4.0).
> >>
> >>
> >> 4.
> >> This is related to the previous point. I assume that the difference
> >> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is that in the
> >> former you enable transactions on the state store and in the latter you
> >> disable them. If my assumption is correct, I think that is an issue.
> >> Let's assume under ALOS Streams fails over a couple of times more or
> >> less at the same step in processing after value 3 is added to an
> >> aggregation but the offset of the corresponding input record was not
> >> committed. Without transactions disabled, the aggregation value would
> >> increase by 3 for each failover. With transactions enabled, value 3
> >> would only be added to the aggregation once when the offset of the input
> >> record is committed and the transaction finally completes. So the
> >> content of the state store would change depending on the configuration
> >> for IQ. IMO, the content of the state store should be independent from
> >> IQ. Given this issue, I propose to not use transactions with ALOS at
> >> all. I was a big proponent of using transactions with ALOS, but I
> >> realized that transactions with ALOS is not as easy as enabling
> >> transactions on state stores. Another aspect that is problematic is that
> >> the changelog topic which actually replicates the state store is not
> >> transactional under ALOS. Thus, it might happen that the state store and
> >> the changelog differ in their content. All of this is maybe solvable
> >> somehow, but for the sake of this KIP, I would leave it for the future.
> >>
> >>
> >> Best,
> >> Bruno
> >>
> >>
> >>
> >> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> >>> Hey Nick! First of all thanks for taking up this awesome feature, I'm
> >> sure
> >>> every single
> >>> Kafka Streams user and dev would agree that it is sorely needed.
> >>>
> >>> I've just been catching up on the KIP and surrounding discussion, so
> >> please
> >>> forgive me
> >>> for any misunderstandings or misinterpretations of the current plan and
> >>> don't hesitate to
> >>> correct me.
> >>>
> >>> Before I jump in, I just want to say that having seen this drag on for
> so
> >>> long, my singular
> >>> goal in responding is to help this KIP past a perceived impasse so we
> can
> >>> finally move on
> >>> to voting and implementing it. Long discussions are to be expected for
> >>> major features like
> >>> this but it's completely on us as the Streams devs to make sure there
> is
> >> an
> >>> end in sight
> >>> for any ongoing discussion.
> >>>
> >>> With that said, it's my understanding that the KIP as currently
> proposed
> >> is
> >>> just not tenable
> >>> for Kafka Streams, and would prevent some EOS users from upgrading to
> the
> >>> version it
> >>> first appears in. Given that we can't predict or guarantee whether any
> of
> >>> the followup KIPs
> >>> would be completed in the same release cycle as this one, we need to
> make
> >>> sure that the
> >>> feature is either compatible with all current users or else
> >> feature-flagged
> >>> so that they may
> >>> opt in/out.
> >>>
> >>> Therefore, IIUC we need to have either (or both) of these as
> >>> fully-implemented config options:
> >>> 1. default.state.isolation.level
> >>> 2. enable.transactional.state.stores
> >>>
> >>> This way EOS users for whom read_committed semantics are not viable can
> >>> still upgrade,
> >>> and either use the isolation.level config to leverage the new txn state
> >>> stores without sacrificing
> >>> their application semantics, or else simply keep the transactional
> state
> >>> stores disabled until we
> >>> are able to fully implement the isolation level configuration at either
> >> an
> >>> application or query level.
> >>>
> >>> Frankly you are the expert here and know much more about the tradeoffs
> in
> >>> both semantics and
> >>> effort level of implementing one of these configs vs the other. In my
> >>> opinion, either option would
> >>> be fine and I would leave the decision of which one to include in this
> >> KIP
> >>> completely up to you.
> >>> I just don't see a way for the KIP to proceed without some variation of
> >> the
> >>> above that would allow
> >>> EOS users to opt-out of read_committed.
> >>>
> >>> (If it's all the same to you, I would recommend 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 that gets released.
> >>> So personally, my choice would be to add the feature flag and leave it
> >> off
> >>> by default. If all 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
> >>>
> >>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <ni...@gmail.com>
> >> wrote:
> >>>
> >>>> Hi Bruno,
> >>>>
> >>>> Agreed, I can live with that for now.
> >>>>
> >>>> In an effort to keep the scope of this KIP from expanding, I'm leaning
> >>>> towards just providing a configurable default.state.isolation.level
> and
> >>>> removing IsolationLevel from the StateStoreContext. This would be
> >>>> compatible with adding support for query-time IsolationLevels in the
> >>>> future, whilst providing a way for users to select an isolation level
> >> now.
> >>>>
> >>>> The big problem with this, however, is that if a user selects
> >>>> processing.mode
> >>>> = "exactly-once(-v2|-beta)", and default.state.isolation.level =
> >>>> "READ_UNCOMMITTED", we need to guarantee that the data isn't written
> to
> >>>> disk until commit() is called, but we also need to permit IQ threads
> to
> >>>> read from the ongoing transaction.
> >>>>
> >>>> A simple solution would be to (temporarily) forbid this combination of
> >>>> configuration, and have default.state.isolation.level automatically
> >> switch
> >>>> to READ_COMMITTED when processing.mode is anything other than
> >>>> at-least-once. Do you think this would be acceptable?
> >>>>
> >>>> In a later KIP, we can add support for query-time isolation levels and
> >>>> solve this particular problem there, which would relax this
> restriction.
> >>>>
> >>>> Regards,
> >>>> Nick
> >>>>
> >>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <ca...@apache.org>
> >> 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,
> >>>>> Bruno
> >>>>>
> >>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
> >>>>>> 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 <nick.telford@gmail.com
> >
> >>>>> wrote:
> >>>>>>
> >>>>>>> Hi everyone,
> >>>>>>>
> >>>>>>> I agree that having IsolationLevel be determined at query-time is
> the
> >>>>>>> ideal design, but there are a few sticking points:
> >>>>>>>
> >>>>>>> 1.
> >>>>>>> There needs to be some way to communicate the IsolationLevel down
> to
> >>>> the
> >>>>>>> RocksDBStore itself, so that the query can respect it. Since stores
> >>>> are
> >>>>>>> "layered" in functionality (i.e. ChangeLoggingStore, MeteredStore,
> >>>>> etc.),
> >>>>>>> we need some way to deliver that information to the bottom layer.
> For
> >>>>> IQv2,
> >>>>>>> we can use the existing State#query() method, but IQv1 has no way
> to
> >>>> do
> >>>>>>> this.
> >>>>>>>
> >>>>>>> A simple approach, which would potentially open up other options,
> >>>> would
> >>>>> be
> >>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> >>>>>>> readOnlyView(IsolationLevel isolationLevel) to
> ReadOnlyKeyValueStore
> >>>>> (and
> >>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
> >>>>>>>
> >>>>>>> 2.
> >>>>>>> As mentioned above, RocksDB WriteBatches are not thread-safe, which
> >>>>> causes
> >>>>>>> a problem if we want to provide READ_UNCOMMITTED Iterators. I also
> >>>> had a
> >>>>>>> look at RocksDB Transactions[1], but they solve a very different
> >>>>> problem,
> >>>>>>> and have the same thread-safety issue.
> >>>>>>>
> >>>>>>> One possible approach that I mentioned is chaining WriteBatches:
> >> every
> >>>>>>> time a new Interactive Query is received (i.e. readOnlyView, see
> >>>> above,
> >>>>>>> is called) we "freeze" the existing WriteBatch, and start a new one
> >>>> for
> >>>>> new
> >>>>>>> writes. The Interactive Query queries the "chain" of previous
> >>>>> WriteBatches
> >>>>>>> + the underlying database; while the StreamThread starts writing to
> >>>> the
> >>>>>>> *new* WriteBatch. On-commit, the StreamThread would write *all*
> >>>>>>> WriteBatches in the chain to the database (that have not yet been
> >>>>> written).
> >>>>>>>
> >>>>>>> WriteBatches would be closed/freed only when they have been both
> >>>>>>> committed, and all open Interactive Queries on them have been
> closed.
> >>>>> This
> >>>>>>> would require some reference counting.
> >>>>>>>
> >>>>>>> Obviously a drawback of this approach is the potential for
> increased
> >>>>>>> memory usage: if an Interactive Query is long-lived, for example by
> >>>>> doing a
> >>>>>>> full scan over a large database, or even just pausing in the middle
> >> of
> >>>>> an
> >>>>>>> iteration, then the existing chain of WriteBatches could be kept
> >>>> around
> >>>>> for
> >>>>>>> a long time, potentially forever.
> >>>>>>>
> >>>>>>> --
> >>>>>>>
> >>>>>>> A.
> >>>>>>> Going off on a tangent, it looks like in addition to supporting
> >>>>>>> READ_COMMITTED queries, we could go further and support
> >>>> REPEATABLE_READ
> >>>>>>> queries (i.e. where subsequent reads to the same key in the same
> >>>>>>> Interactive Query are guaranteed to yield the same value) by making
> >>>> use
> >>>>> of
> >>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so the
> >> performance
> >>>>>>> impact is likely to be negligible, but they do require that the
> >>>>> Interactive
> >>>>>>> Query session can be explicitly closed.
> >>>>>>>
> >>>>>>> This could be achieved if we made the above readOnlyView interface
> >>>> look
> >>>>>>> more like:
> >>>>>>>
> >>>>>>> interface ReadOnlyKeyValueView<K, V> implements
> >>>> ReadOnlyKeyValueStore<K,
> >>>>>>> V>, AutoCloseable {}
> >>>>>>>
> >>>>>>> interface ReadOnlyKeyValueStore<K, V> {
> >>>>>>>        ...
> >>>>>>>        ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel
> >>>>> isolationLevel);
> >>>>>>> }
> >>>>>>>
> >>>>>>> But this would be a breaking change, as existing IQv1 queries are
> >>>>>>> guaranteed to never call store.close(), and therefore these would
> >> leak
> >>>>>>> memory under REPEATABLE_READ.
> >>>>>>>
> >>>>>>> B.
> >>>>>>> One thing that's notable: MyRocks states that they support
> >>>>> READ_COMMITTED
> >>>>>>> and REPEATABLE_READ, but they make no mention of
> >>>> READ_UNCOMMITTED[3][4].
> >>>>>>> This could be because doing so is technically difficult/impossible
> >>>> using
> >>>>>>> the primitives available in RocksDB.
> >>>>>>>
> >>>>>>> --
> >>>>>>>
> >>>>>>> Lucas, to address your points:
> >>>>>>>
> >>>>>>> U1.
> >>>>>>> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
> >>>>>>> implementations of StateStore that do not support atomic writes.
> >>>>> Obviously
> >>>>>>> in those cases, the guarantees Kafka Streams provides/expects would
> >> be
> >>>>>>> relaxed. Do you think we should require all implementations to
> >> support
> >>>>>>> atomic writes?
> >>>>>>>
> >>>>>>> U2.
> >>>>>>> Stores can support multiple IsolationLevels. As we've discussed
> >> above,
> >>>>> the
> >>>>>>> ideal scenario would be to specify the IsolationLevel at
> query-time.
> >>>>>>> Failing that, I think the second-best approach is to define the
> >>>>>>> IsolationLevel for *all* queries based on the processing.mode,
> which
> >>>> is
> >>>>>>> what the default StateStoreContext#isolationLevel() achieves. Would
> >>>> you
> >>>>>>> prefer an alternative?
> >>>>>>>
> >>>>>>> While the existing implementation is equivalent to
> READ_UNCOMMITTED,
> >>>>> this
> >>>>>>> can yield unexpected results/errors under EOS, if a transaction is
> >>>>> rolled
> >>>>>>> back. While this would be a change in behaviour for users, it would
> >>>> look
> >>>>>>> more like a bug fix than a breaking change. That said, we *could*
> >> make
> >>>>> it
> >>>>>>> configurable, and default to the existing behaviour
> >> (READ_UNCOMMITTED)
> >>>>>>> instead of inferring it from the processing.mode?
> >>>>>>>
> >>>>>>> N1, N2.
> >>>>>>> These were only primitives to avoid boxing costs, but since this is
> >>>> not
> >>>>> a
> >>>>>>> performance sensitive area, it should be fine to change if that's
> >>>>> desirable.
> >>>>>>>
> >>>>>>> N3.
> >>>>>>> It's because the store "manages its own offsets", which includes
> both
> >>>>>>> committing the offset, *and providing it* via getCommittedOffset().
> >>>>>>> Personally, I think "managesOffsets" conveys this best, but I don't
> >>>> mind
> >>>>>>> changing it if the nomenclature is unclear.
> >>>>>>>
> >>>>>>> Sorry for the massive emails/essays!
> >>>>>>> --
> >>>>>>> Nick
> >>>>>>>
> >>>>>>> 1: https://github.com/facebook/rocksdb/wiki/Transactions
> >>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> >>>>>>> 3:
> https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> >>>>>>> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
> >>>>>>>
> >>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> >>>>>>> <lb...@confluent.io.invalid> wrote:
> >>>>>>>
> >>>>>>>> Hi Nick,
> >>>>>>>>
> >>>>>>>> since I last read it in April, the KIP has become much cleaner and
> >>>>>>>> easier to read. Great work!
> >>>>>>>>
> >>>>>>>> It feels to me the last big open point is whether we can implement
> >>>>>>>> isolation level as a query parameter. I understand that there are
> >>>>>>>> implementation concerns, but as Colt says, it would be a great
> >>>>>>>> addition, and would also simplify the migration path for this
> >> change.
> >>>>>>>> Is the implementation problem you mentioned caused by the
> WriteBatch
> >>>>>>>> not having a notion of a snapshot, as the underlying DB iterator
> >>>> does?
> >>>>>>>> In that case, I am not sure a chain of WriteBatches as you propose
> >>>>>>>> would fully solve the problem, but maybe I didn't dig enough into
> >> the
> >>>>>>>> details to fully understand it.
> >>>>>>>>
> >>>>>>>> If it's not possible to implement it now, would it be an option to
> >>>>>>>> make sure in this KIP that we do not fully close the door on
> >>>> per-query
> >>>>>>>> isolation levels in the interface, as it may be possible to
> >> implement
> >>>>>>>> the missing primitives in RocksDB or Speedb in the future.
> >>>>>>>>
> >>>>>>>> Understanding:
> >>>>>>>>
> >>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to be persisted
> >>>>>>>> atomically with the records?
> >>>>>>>> * U2) Don't understand the default implementation of
> >>>> `isolationLevel`.
> >>>>>>>> The isolation level should be a property of the underlying store,
> >> and
> >>>>>>>> not be defined by the default config? Existing stores probably
> don't
> >>>>>>>> guarantee READ_COMMITTED, so the default should be to return
> >>>>>>>> READ_UNCOMMITTED.
> >>>>>>>>
> >>>>>>>> Nits:
> >>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong` return type,
> >> to
> >>>>>>>> avoid the `null`?
> >>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
> >>>>>>>> return type, to avoid the `-1`?
> >>>>>>>> * N3) I don't understand why `managesOffsets` uses the 'manage'
> >> verb,
> >>>>>>>> whereas all other methods use the "commits" verb. I'd suggest
> >>>>>>>> `commitsOffsets`.
> >>>>>>>>
> >>>>>>>> Either way, it feels this KIP is very close to the finish line,
> I'm
> >>>>>>>> looking forward to seeing this in production!
> >>>>>>>>
> >>>>>>>> Cheers,
> >>>>>>>> Lucas
> >>>>>>>>
> >>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <colt@littlehorse.io
> >
> >>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Making IsolationLevel a query-time constraint, rather than
> linking
> >>>> it
> >>>>>>>> to
> >>>>>>>>> the processing.guarantee.
> >>>>>>>>>
> >>>>>>>>> As I understand it, would this allow even a user of EOS to
> control
> >>>>>>>> whether
> >>>>>>>>> reading committed or uncommitted records? If so, I am highly in
> >>>> favor
> >>>>> of
> >>>>>>>>> this.
> >>>>>>>>>
> >>>>>>>>> I know that I was one of the early people to point out the
> current
> >>>>>>>>> shortcoming that IQ reads uncommitted records, but just this
> >>>> morning I
> >>>>>>>>> realized a pattern we use which means that (for certain queries)
> >> our
> >>>>>>>> system
> >>>>>>>>> needs to be able to read uncommitted records, which is the
> current
> >>>>>>>> behavior
> >>>>>>>>> of Kafka Streams in EOS.***
> >>>>>>>>>
> >>>>>>>>> If IsolationLevel being a query-time decision allows for this,
> then
> >>>>> that
> >>>>>>>>> would be amazing. I would also vote that the default behavior
> >> should
> >>>>> be
> >>>>>>>> for
> >>>>>>>>> reading uncommitted records, because it is totally possible for a
> >>>>> valid
> >>>>>>>>> application to depend on that behavior, and breaking it in a
> minor
> >>>>>>>> release
> >>>>>>>>> might be a bit strong.
> >>>>>>>>>
> >>>>>>>>> *** (Note, for the curious reader....) Our use-case/query pattern
> >>>> is a
> >>>>>>>> bit
> >>>>>>>>> complex, but reading "uncommitted" records is actually safe in
> our
> >>>>> case
> >>>>>>>>> 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 soon as the processor processes the result.
> If
> >>>>> they
> >>>>>>>>> can't read uncommitted records, they may receive a "201 -
> Created"
> >>>>>>>>> response, immediately followed by a "404 - 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 <
> >>>> nick.telford@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Addendum:
> >>>>>>>>>>
> >>>>>>>>>> I think we would also face the same problem with the approach
> John
> >>>>>>>> outlined
> >>>>>>>>>> earlier (using the record cache as a transaction buffer and
> >>>> flushing
> >>>>>>>> it
> >>>>>>>>>> straight to SST files). This is because the record cache (the
> >>>>>>>> ThreadCache
> >>>>>>>>>> class) is not thread-safe, so every commit would invalidate open
> >> IQ
> >>>>>>>>>> Iterators in the same way that RocksDB WriteBatches do.
> >>>>>>>>>> --
> >>>>>>>>>> Nick
> >>>>>>>>>>
> >>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> >>>> nick.telford@gmail.com>
> >>>>>>>>>> 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.
> >>>>>>>>>>> 2. Making IsolationLevel a query-time constraint, rather than
> >>>>>>>> linking it
> >>>>>>>>>>> to the processing.guarantee.
> >>>>>>>>>>>
> >>>>>>>>>>> There's a wrinkle that makes this a challenge: Interactive
> >> Queries
> >>>>>>>> that
> >>>>>>>>>>> open an Iterator, when using transactions and READ_UNCOMMITTED.
> >>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries need to be
> >>>> able
> >>>>>>>> to
> >>>>>>>>>>> read records from the currently uncommitted transaction buffer
> >>>>>>>>>>> (WriteBatch). This includes for Iterators, which should iterate
> >>>>>>>> both the
> >>>>>>>>>>> transaction buffer and underlying database (using
> >>>>>>>>>>> WriteBatch#iteratorWithBase()).
> >>>>>>>>>>>
> >>>>>>>>>>> The issue is that when the StreamThread commits, it writes the
> >>>>>>>> current
> >>>>>>>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*.
> Clearing
> >>>> the
> >>>>>>>>>>> WriteBatch while an Interactive Query holds an open Iterator on
> >> it
> >>>>>>>> will
> >>>>>>>>>>> invalidate the Iterator. Worse, it turns out that Iterators
> over
> >> a
> >>>>>>>>>>> WriteBatch become invalidated not just when the WriteBatch is
> >>>>>>>> cleared,
> >>>>>>>>>> but
> >>>>>>>>>>> also when the Iterators' current key receives a new write.
> >>>>>>>>>>>
> >>>>>>>>>>> Now that I'm writing this, I remember that this is the major
> >>>> reason
> >>>>>>>> that
> >>>>>>>>>> I
> >>>>>>>>>>> switched the original design from having a query-time
> >>>>>>>> IsolationLevel to
> >>>>>>>>>>> having the IsolationLevel linked to the transactionality of the
> >>>>>>>> stores
> >>>>>>>>>>> themselves.
> >>>>>>>>>>>
> >>>>>>>>>>> It *might* be possible to resolve this, by having a "chain" of
> >>>>>>>>>>> WriteBatches, with the StreamThread switching to a new
> WriteBatch
> >>>>>>>>>> whenever
> >>>>>>>>>>> a new Interactive Query attempts to read from the database, but
> >>>> that
> >>>>>>>>>> could
> >>>>>>>>>>> cause some performance problems/memory pressure when subjected
> to
> >>>> a
> >>>>>>>> high
> >>>>>>>>>>> Interactive Query load. It would also reduce the efficiency of
> >>>>>>>>>> WriteBatches
> >>>>>>>>>>> on-commit, as we'd have to write N WriteBatches, where N is the
> >>>>>>>> number of
> >>>>>>>>>>> Interactive Queries since the last commit.
> >>>>>>>>>>>
> >>>>>>>>>>> I realise this is getting into the weeds of the implementation,
> >>>> and
> >>>>>>>> you'd
> >>>>>>>>>>> rather we focus on the API for now, but I think it's important
> to
> >>>>>>>>>> consider
> >>>>>>>>>>> how to implement the desired API, in case we come up with an
> API
> >>>>>>>> that
> >>>>>>>>>>> cannot be implemented efficiently, or even at all!
> >>>>>>>>>>>
> >>>>>>>>>>> Thoughts?
> >>>>>>>>>>> --
> >>>>>>>>>>> Nick
> >>>>>>>>>>>
> >>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> cadonna@apache.org
> >>>
> >>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>
> >>>>>>>>>>>> 6.
> >>>>>>>>>>>> Of course, you are right! My bad!
> >>>>>>>>>>>> Wiping out the state in the downgrading case is fine.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> 3a.
> >>>>>>>>>>>> Focus on the public facing changes for the KIP. We will manage
> >> 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.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Best,
> >>>>>>>>>>>> Bruno
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> >>>>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks for getting back to me!
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 2.
> >>>>>>>>>>>>> The fact that implementations can always track estimated
> memory
> >>>>>>>> usage
> >>>>>>>>>> in
> >>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as an option,
> and
> >>>>>>>> I'll
> >>>>>>>>>>>> clarify
> >>>>>>>>>>>>> the JavaDoc that 0 is not just for non-transactional stores,
> >>>>>>>> which is
> >>>>>>>>>>>>> currently misleading.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 6.
> >>>>>>>>>>>>> The problem with catching the exception in the downgrade
> >> process
> >>>>>>>> is
> >>>>>>>>>> that
> >>>>>>>>>>>>> would require new code in the Kafka version being downgraded
> >> to.
> >>>>>>>> Since
> >>>>>>>>>>>>> users could conceivably downgrade to almost *any* older
> version
> >>>>>>>> of
> >>>>>>>>>> Kafka
> >>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
> >>>>>>>>>>>>> The only way I can think of doing it would be to provide a
> >>>>>>>> dedicated
> >>>>>>>>>>>>> downgrade tool, that goes through every local store and
> removes
> >>>>>>>> the
> >>>>>>>>>>>>> offsets column families. But that seems like an unnecessary
> >>>>>>>> amount of
> >>>>>>>>>>>> extra
> >>>>>>>>>>>>> code to maintain just to handle a somewhat niche situation,
> >> when
> >>>>>>>> the
> >>>>>>>>>>>>> alternative (automatically wipe and restore stores) should be
> >>>>>>>>>>>> acceptable.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've requested.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 3a.
> >>>>>>>>>>>>> I agree that IsolationLevel makes more sense at query-time,
> and
> >>>> I
> >>>>>>>>>>>> actually
> >>>>>>>>>>>>> initially attempted to place the IsolationLevel at
> query-time,
> >>>>>>>> but I
> >>>>>>>>>> ran
> >>>>>>>>>>>>> into some problems:
> >>>>>>>>>>>>> - The key issue is that, under ALOS we're not staging writes
> in
> >>>>>>>>>>>>> transactions, so can't perform writes at the READ_COMMITTED
> >>>>>>>> isolation
> >>>>>>>>>>>>> level. However, this may be addressed if we decide to
> *always*
> >>>>>>>> use
> >>>>>>>>>>>>> transactions as discussed under 3b.
> >>>>>>>>>>>>> - IQv1 and IQv2 have quite different implementations. I
> >> remember
> >>>>>>>>>> having
> >>>>>>>>>>>>> some difficulty understanding the IQv1 internals, which made
> it
> >>>>>>>>>>>> difficult
> >>>>>>>>>>>>> to determine what needed to be changed. However, I *think*
> this
> >>>>>>>> can be
> >>>>>>>>>>>>> addressed for both implementations by wrapping the
> RocksDBStore
> >>>>>>>> in an
> >>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides read methods
> >>>>>>>> (get,
> >>>>>>>>>>>> etc.)
> >>>>>>>>>>>>> to either read directly from the database or from the ongoing
> >>>>>>>>>>>> transaction.
> >>>>>>>>>>>>> But IQv1 might still be difficult.
> >>>>>>>>>>>>> - If IsolationLevel becomes a query constraint, then all
> other
> >>>>>>>>>>>> StateStores
> >>>>>>>>>>>>> will need to respect it, including the in-memory stores. This
> >>>>>>>> would
> >>>>>>>>>>>> require
> >>>>>>>>>>>>> us to adapt in-memory stores to stage their writes so they
> can
> >>>> be
> >>>>>>>>>>>> isolated
> >>>>>>>>>>>>> from READ_COMMITTTED queries. It would also become an
> important
> >>>>>>>>>>>>> consideration for third-party stores on upgrade, as without
> >>>>>>>> changes,
> >>>>>>>>>>>> they
> >>>>>>>>>>>>> would not support READ_COMMITTED queries correctly.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Ultimately, I may need some help making the necessary change
> to
> >>>>>>>> IQv1
> >>>>>>>>>> to
> >>>>>>>>>>>>> support this, but I don't think it's fundamentally
> impossible,
> >>>>>>>> if we
> >>>>>>>>>>>> want
> >>>>>>>>>>>>> to pursue this route.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 3b.
> >>>>>>>>>>>>> The main reason I chose to keep ALOS un-transactional was to
> >>>>>>>> minimize
> >>>>>>>>>>>>> behavioural change for most users (I believe most Streams
> users
> >>>>>>>> use
> >>>>>>>>>> the
> >>>>>>>>>>>>> default configuration, which is ALOS). That said, it's clear
> >>>>>>>> that if
> >>>>>>>>>>>> ALOS
> >>>>>>>>>>>>> also used transactional stores, the only change in behaviour
> >>>>>>>> would be
> >>>>>>>>>>>> that
> >>>>>>>>>>>>> it would become *more correct*, which could be considered a
> >> "bug
> >>>>>>>> fix"
> >>>>>>>>>> by
> >>>>>>>>>>>>> users, rather than a change they need to handle.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I believe that performance using transactions (aka. RocksDB
> >>>>>>>>>>>> WriteBatches)
> >>>>>>>>>>>>> should actually be *better* than the un-batched write-path
> that
> >>>>>>>> is
> >>>>>>>>>>>>> currently used[1]. The only "performance" consideration will
> be
> >>>>>>>> the
> >>>>>>>>>>>>> increased memory usage that transactions require. Given the
> >>>>>>>>>> mitigations
> >>>>>>>>>>>> for
> >>>>>>>>>>>>> this memory that we have in place, I would expect that this
> is
> >>>>>>>> not a
> >>>>>>>>>>>>> problem for most users.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> If we're happy to do so, we can make ALOS also use
> >> transactions.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Link 1:
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>
> >> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> >>>> cadonna@apache.org
> >>>>>>>>>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks for the updates and sorry for the delay on my side!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>> Making the default implementation for flush() a no-op sounds
> >>>>>>>> good to
> >>>>>>>>>>>> me.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>> I think what was bugging me here is that a third-party state
> >>>>>>>> store
> >>>>>>>>>>>> needs
> >>>>>>>>>>>>>> to implement the state store interface. That means they need
> >> to
> >>>>>>>>>>>>>> implement a wrapper around the actual state store as we do
> for
> >>>>>>>>>> RocksDB
> >>>>>>>>>>>>>> with RocksDBStore. So, a third-party state store can always
> >>>>>>>> estimate
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>> uncommitted bytes, if it wants, because the wrapper can
> record
> >>>>>>>> the
> >>>>>>>>>>>> added
> >>>>>>>>>>>>>> bytes.
> >>>>>>>>>>>>>> One case I can think of where returning -1 makes sense is
> when
> >>>>>>>>>> Streams
> >>>>>>>>>>>>>> does not need to estimate the size of the write batch and
> >>>>>>>> trigger
> >>>>>>>>>>>>>> extraordinary commits, because the third-party state store
> >>>>>>>> takes care
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>> memory. But in that case the method could also just return
> 0.
> >>>>>>>> Even
> >>>>>>>>>> that
> >>>>>>>>>>>>>> case would be better solved with a method that returns
> whether
> >>>>>>>> the
> >>>>>>>>>>>> state
> >>>>>>>>>>>>>> store manages itself the memory used for uncommitted bytes
> or
> >>>>>>>> not.
> >>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return value, I was
> >>>>>>>> just
> >>>>>>>>>>>>>> wondering when and if it will be used.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regarding returning 0 for transactional state stores when
> the
> >>>>>>>> batch
> >>>>>>>>>> is
> >>>>>>>>>>>>>> empty, I was just wondering because you explicitly stated
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> "or {@code 0} if this StateStore does not support
> >>>> transactions."
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> So it seemed to me returning 0 could only happen for
> >>>>>>>>>> non-transactional
> >>>>>>>>>>>>>> state stores.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> a) What do you think if we move the isolation level to IQ
> (v1
> >>>>>>>> and
> >>>>>>>>>> v2)?
> >>>>>>>>>>>>>> In the end this is the only component that really needs to
> >>>>>>>> specify
> >>>>>>>>>> the
> >>>>>>>>>>>>>> isolation level. It is similar to the Kafka consumer that
> can
> >>>>>>>> choose
> >>>>>>>>>>>>>> with what isolation level to read the input topic.
> >>>>>>>>>>>>>> For IQv1 the isolation level should go into
> >>>>>>>> StoreQueryParameters. For
> >>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> b) Point a) raises the question what should happen during
> >>>>>>>>>> at-least-once
> >>>>>>>>>>>>>> processing when the state store does not use transactions?
> >> John
> >>>>>>>> in
> >>>>>>>>>> the
> >>>>>>>>>>>>>> past proposed to also use transactions on state stores for
> >>>>>>>>>>>>>> at-least-once. I like that idea, because it avoids
> aggregating
> >>>>>>>> the
> >>>>>>>>>> same
> >>>>>>>>>>>>>> records over and over again in the case of a failure. We
> had a
> >>>>>>>> case
> >>>>>>>>>> in
> >>>>>>>>>>>>>> the past where a Streams applications in at-least-once mode
> >> was
> >>>>>>>>>> failing
> >>>>>>>>>>>>>> continuously for some reasons I do not remember before
> >>>>>>>> committing the
> >>>>>>>>>>>>>> offsets. After each failover, the app aggregated again and
> >>>>>>>> again the
> >>>>>>>>>>>>>> same records. Of course the aggregate increased to very
> wrong
> >>>>>>>> values
> >>>>>>>>>>>>>> just because of the failover. With transactions on the state
> >>>>>>>> stores
> >>>>>>>>>> we
> >>>>>>>>>>>>>> could have avoided this. The app would have output the same
> >>>>>>>> aggregate
> >>>>>>>>>>>>>> multiple times (i.e., after each failover) but at least the
> >>>>>>>> value of
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>> aggregate would not depend on the number of failovers.
> >>>>>>>> Outputting the
> >>>>>>>>>>>>>> same aggregate multiple times would be incorrect under
> >>>>>>>> exactly-once
> >>>>>>>>>> but
> >>>>>>>>>>>>>> it is OK for at-least-once.
> >>>>>>>>>>>>>> If it makes sense to add a config to turn on and off
> >>>>>>>> transactions on
> >>>>>>>>>>>>>> state stores under at-least-once or just use transactions in
> >>>>>>>> any case
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>> a question we should also discuss in this KIP. It depends a
> >> bit
> >>>>>>>> on
> >>>>>>>>>> the
> >>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would add a
> config.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 4.
> >>>>>>>>>>>>>> Your points are all valid. I tend to say to keep the metrics
> >>>>>>>> around
> >>>>>>>>>>>>>> flush() until we remove flush() completely from the
> interface.
> >>>>>>>> Calls
> >>>>>>>>>> to
> >>>>>>>>>>>>>> flush() might still exist since existing processors might
> >> still
> >>>>>>>> call
> >>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For sure, we need
> >> to
> >>>>>>>>>>>> document
> >>>>>>>>>>>>>> how the metrics change due to the transactions in the
> upgrade
> >>>>>>>> notes.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 5.
> >>>>>>>>>>>>>> I see. Then you should describe how the .position files are
> >>>>>>>> handled
> >>>>>>>>>> in
> >>>>>>>>>>>>>> a dedicated section of the KIP or incorporate the
> description
> >>>>>>>> in the
> >>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only mentioning it
> >> in
> >>>>>>>> the
> >>>>>>>>>>>>>> "Compatibility, 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:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>
> >>>>
> >>
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> It is a bit brittle, but it works.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> >>>>>>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks for taking the time to review the KIP. I'm back from
> >>>>>>>> leave
> >>>>>>>>>> now
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>> intend to move this forwards as quickly as I can.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Addressing your points:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>> Because flush() is part of the StateStore API, it's exposed
> >> to
> >>>>>>>>>> custom
> >>>>>>>>>>>>>>> Processors, which might be making calls to flush(). This
> was
> >>>>>>>>>> actually
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>> case in a few integration tests.
> >>>>>>>>>>>>>>> To maintain as much compatibility as possible, I'd prefer
> not
> >>>>>>>> to
> >>>>>>>>>> make
> >>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>> an UnsupportedOperationException, as it will cause
> previously
> >>>>>>>>>> working
> >>>>>>>>>>>>>>> Processors to start throwing exceptions at runtime.
> >>>>>>>>>>>>>>> I agree that it doesn't make sense for it to proxy
> commit(),
> >>>>>>>> though,
> >>>>>>>>>>>> as
> >>>>>>>>>>>>>>> that would cause it to violate the "StateStores commit only
> >>>>>>>> when the
> >>>>>>>>>>>> Task
> >>>>>>>>>>>>>>> commits" rule.
> >>>>>>>>>>>>>>> Instead, I think we should make this a no-op. That way,
> >>>>>>>> existing
> >>>>>>>>>> user
> >>>>>>>>>>>>>>> Processors will continue to work as-before, without
> violation
> >>>>>>>> of
> >>>>>>>>>> store
> >>>>>>>>>>>>>>> consistency that would be caused by premature flush/commit
> of
> >>>>>>>>>>>> StateStore
> >>>>>>>>>>>>>>> data to disk.
> >>>>>>>>>>>>>>> What do you think?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore implementation
> is
> >>>>>>>>>>>>>>> transactional, but is unable to estimate the uncommitted
> >>>> memory
> >>>>>>>>>> usage,
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> method will return -1.
> >>>>>>>>>>>>>>> The intention here is to permit third-party implementations
> >>>>>>>> that may
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>> able to estimate memory usage.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been written to the
> store
> >>>>>>>> yet. I
> >>>>>>>>>>>>>> thought
> >>>>>>>>>>>>>>> that was implied by "This method will return an
> approximation
> >>>>>>>> of the
> >>>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>> would be freed by the next call to {@link #commit(Map)}"
> and
> >>>>>>>>>> "@return
> >>>>>>>>>>>> The
> >>>>>>>>>>>>>>> approximate size of all records awaiting {@link
> >>>> #commit(Map)}",
> >>>>>>>>>>>> however,
> >>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you think this is
> >>>>>>>> unclear?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>> I realise this is probably the most contentious point in my
> >>>>>>>> design,
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>> open to changing it if I'm unable to convince you of the
> >>>>>>>> benefits.
> >>>>>>>>>>>>>>> Nevertheless, here's my argument:
> >>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly provided
> >>>>>>>> StateStores
> >>>>>>>>>> to
> >>>>>>>>>>>>>>> query, and it may be important for users to
> programmatically
> >>>>>>>> know
> >>>>>>>>>>>> which
> >>>>>>>>>>>>>>> mode the StateStore is operating under. If we simply
> provide
> >>>> an
> >>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the internal
> streams
> >>>>>>>>>>>> engine), or
> >>>>>>>>>>>>>>> similar, then users will need to understand the operation
> and
> >>>>>>>>>>>>>> consequences
> >>>>>>>>>>>>>>> of each available processing mode and how it pertains to
> >> their
> >>>>>>>>>>>>>> StateStore.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Interactive Query users aren't the only people that care
> >> about
> >>>>>>>> the
> >>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore:
> implementers
> >>>> of
> >>>>>>>>>> custom
> >>>>>>>>>>>>>>> StateStores also need to understand the behaviour expected
> of
> >>>>>>>> their
> >>>>>>>>>>>>>>> implementation. KIP-892 introduces some assumptions into
> the
> >>>>>>>> Streams
> >>>>>>>>>>>>>> Engine
> >>>>>>>>>>>>>>> about how StateStores operate under each processing mode,
> and
> >>>>>>>> it's
> >>>>>>>>>>>>>>> important that custom implementations adhere to those
> >>>>>>>> assumptions in
> >>>>>>>>>>>>>> order
> >>>>>>>>>>>>>>> to maintain the consistency guarantees.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> IsolationLevels provide a high-level contract on the
> >> behaviour
> >>>>>>>> of
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> StateStore: a user knows that under READ_COMMITTED, they
> will
> >>>>>>>> see
> >>>>>>>>>>>> writes
> >>>>>>>>>>>>>>> only after the Task has committed, and under
> READ_UNCOMMITTED
> >>>>>>>> they
> >>>>>>>>>>>> will
> >>>>>>>>>>>>>> see
> >>>>>>>>>>>>>>> writes immediately. No understanding of the details of each
> >>>>>>>>>>>>>> processing.mode
> >>>>>>>>>>>>>>> is required, either for IQ users or StateStore
> implementers.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> An argument can be made that these contractual guarantees
> can
> >>>>>>>> simply
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>> documented for the processing.mode (i.e. that exactly-once
> >> and
> >>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and
> at-least-once
> >>>>>>>> behaves
> >>>>>>>>>>>> like
> >>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small issues with
> >>>>>>>> this I'd
> >>>>>>>>>>>>>> prefer
> >>>>>>>>>>>>>>> to avoid:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>         - Where would we document these contracts, in a way
> >> that
> >>>>>>>> is
> >>>>>>>>>>>> difficult
> >>>>>>>>>>>>>>>         for users/implementers to miss/ignore?
> >>>>>>>>>>>>>>>         - It's not clear to users that the processing mode
> is
> >>>>>>>>>>>> communicating
> >>>>>>>>>>>>>>>         an expectation of read isolation, unless they read
> the
> >>>>>>>>>>>>>> documentation. Users
> >>>>>>>>>>>>>>>         rarely consult documentation unless they feel they
> >> need
> >>>>>>>> to, so
> >>>>>>>>>>>> it's
> >>>>>>>>>>>>>> likely
> >>>>>>>>>>>>>>>         this detail would get missed by many users.
> >>>>>>>>>>>>>>>         - It tightly couples processing modes to read
> >> isolation.
> >>>>>>>> Adding
> >>>>>>>>>>>> new
> >>>>>>>>>>>>>>>         processing modes, or changing the read isolation of
> >>>>>>>> existing
> >>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>         modes would be difficult/impossible.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Ultimately, the cost of introducing IsolationLevels is
> just a
> >>>>>>>> single
> >>>>>>>>>>>>>>> method, since we re-use the existing IsolationLevel enum
> from
> >>>>>>>> Kafka.
> >>>>>>>>>>>> This
> >>>>>>>>>>>>>>> gives us a clear place to document the contractual
> guarantees
> >>>>>>>>>> expected
> >>>>>>>>>>>>>>> of/provided by StateStores, that is accessible both by the
> >>>>>>>>>> StateStore
> >>>>>>>>>>>>>>> itself, and by IQ users.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> (Writing this I've just realised that the StateStore and IQ
> >>>>>>>> APIs
> >>>>>>>>>>>> actually
> >>>>>>>>>>>>>>> don't provide access to StateStoreContext that IQ users
> would
> >>>>>>>> have
> >>>>>>>>>>>> direct
> >>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
> >> isolationLevel()
> >>>>>>>>>> itself
> >>>>>>>>>>>>>> too?)
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 4.
> >>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics in-place
> >>>>>>>> either, as
> >>>>>>>>>>>> it's a
> >>>>>>>>>>>>>>> backwards incompatible change. My concern is that, if we
> >> leave
> >>>>>>>> the
> >>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>> "flush" metrics in place, they will be confusing to users.
> >>>>>>>> Right
> >>>>>>>>>> now,
> >>>>>>>>>>>>>>> "flush" metrics record explicit flushes to disk, but under
> >>>>>>>> KIP-892,
> >>>>>>>>>>>> even
> >>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>> commit() will not explicitly flush data to disk - RocksDB
> >> will
> >>>>>>>>>> decide
> >>>>>>>>>>>> on
> >>>>>>>>>>>>>>> when to flush memtables to disk itself.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd have two
> >>>> options,
> >>>>>>>>>> which
> >>>>>>>>>>>>>> both
> >>>>>>>>>>>>>>> seem pretty bad to me:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>         1. Have them record calls to commit(), which would
> be
> >>>>>>>>>>>> misleading, as
> >>>>>>>>>>>>>>>         data is no longer explicitly "flushed" to disk by
> this
> >>>>>>>> call.
> >>>>>>>>>>>>>>>         2. Have them record nothing at all, which is
> >> equivalent
> >>>> to
> >>>>>>>>>>>> removing
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>         metrics, except that users will see the metric
> still
> >>>>>>>> exists and
> >>>>>>>>>>>> so
> >>>>>>>>>>>>>> assume
> >>>>>>>>>>>>>>>         that the metric is correct, and that there's a
> problem
> >>>>>>>> with
> >>>>>>>>>> their
> >>>>>>>>>>>>>> system
> >>>>>>>>>>>>>>>         when there isn't.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I agree that removing them is also a bad solution, and I'd
> >>>>>>>> like some
> >>>>>>>>>>>>>>> guidance on the best path forward here.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 5.
> >>>>>>>>>>>>>>> Position files are updated on every write to a StateStore.
> >>>>>>>> Since our
> >>>>>>>>>>>>>> writes
> >>>>>>>>>>>>>>> are now buffered until commit(), we can't update the
> Position
> >>>>>>>> file
> >>>>>>>>>>>> until
> >>>>>>>>>>>>>>> commit() has been called, otherwise it would be
> inconsistent
> >>>>>>>> with
> >>>>>>>>>> the
> >>>>>>>>>>>>>> data
> >>>>>>>>>>>>>>> in the event of a rollback. Consequently, we need to manage
> >>>>>>>> these
> >>>>>>>>>>>> offsets
> >>>>>>>>>>>>>>> the same way we manage the checkpoint offsets, and ensure
> >>>>>>>> they're
> >>>>>>>>>> only
> >>>>>>>>>>>>>>> written on commit().
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 6.
> >>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what tests to
> >> write.
> >>>>>>>> How
> >>>>>>>>>>>>>> explicit
> >>>>>>>>>>>>>>> do we need to be here in the KIP?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be
> seamless,
> >>>>>>>> and we
> >>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>> definitely add some tests around that. Downgrade, it
> >>>>>>>> transpires,
> >>>>>>>>>> isn't
> >>>>>>>>>>>>>>> currently possible, as the extra column family for offset
> >>>>>>>> storage is
> >>>>>>>>>>>>>>> incompatible with the pre-KIP-892 implementation: when you
> >>>>>>>> open a
> >>>>>>>>>>>> RocksDB
> >>>>>>>>>>>>>>> database, you must open all available column families or
> >>>>>>>> receive an
> >>>>>>>>>>>>>> error.
> >>>>>>>>>>>>>>> What currently happens on downgrade is that it attempts to
> >>>>>>>> open the
> >>>>>>>>>>>>>> store,
> >>>>>>>>>>>>>>> throws an error about the offsets column family not being
> >>>>>>>> opened,
> >>>>>>>>>>>> which
> >>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given that
> >> downgrades
> >>>>>>>>>> should
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>> uncommon, I think this is acceptable behaviour, as the
> >>>>>>>> end-state is
> >>>>>>>>>>>>>>> consistent, even if it results in an undesirable state
> >>>> restore.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Should I document the upgrade/downgrade behaviour
> explicitly
> >>>>>>>> in the
> >>>>>>>>>>>> KIP?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> --
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> >>>>>>>> cadonna@apache.org>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi Nick!
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks for the updates!
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>> Why does StateStore#flush() default to
> >>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> >>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore after this
> KIP
> >>>>>>>> is
> >>>>>>>>>>>>>>>> released, I would rather throw an unsupported operation
> >>>>>>>> exception
> >>>>>>>>>> by
> >>>>>>>>>>>>>>>> default.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>> When would a state store return -1 from
> >>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while being
> >>>>>>>>>>>> transactional?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Wouldn't StateStore#approximateNumUncommittedBytes() also
> >>>>>>>> return 0
> >>>>>>>>>> if
> >>>>>>>>>>>>>>>> the state store is transactional but nothing has been
> >> written
> >>>>>>>> to
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>> state store yet?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this KIP really
> need
> >>>> to
> >>>>>>>>>>>> introduce
> >>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()? StateStoreContext has
> >>>>>>>> already
> >>>>>>>>>>>>>>>> appConfigs() which basically exposes the same information,
> >>>>>>>> i.e., if
> >>>>>>>>>>>> EOS
> >>>>>>>>>>>>>>>> is enabled or not.
> >>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore interface as
> >>>>>>>> loosely
> >>>>>>>>>>>> coupled
> >>>>>>>>>>>>>>>> from the Streams engine as possible, to give implementers
> >>>> more
> >>>>>>>>>>>> freedom,
> >>>>>>>>>>>>>>>> and reduce the amount of internal knowledge required."
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> While I understand the intent, I doubt that it decreases
> the
> >>>>>>>>>>>> coupling of
> >>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> >> READ_COMMITTED
> >>>>>>>> only
> >>>>>>>>>>>>>>>> applies to IQ but not to reads by processors. Thus,
> >>>>>>>> implementers
> >>>>>>>>>>>> need to
> >>>>>>>>>>>>>>>> understand how Streams accesses the state stores.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I would like to hear what others think about this.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 4.
> >>>>>>>>>>>>>>>> Great exposing new metrics for transactional state stores!
> >>>>>>>>>> However, I
> >>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate (in the
> docs)
> >>>>>>>> the 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 also need to adapt/add integration tests
> >>>>>>>> besides
> >>>>>>>>>> unit
> >>>>>>>>>>>>>>>> tests. Additionally, we probably need integration or
> system
> >>>>>>>> tests
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>> verify that upgrades and downgrades between transactional
> >> and
> >>>>>>>>>>>>>>>> non-transactional state stores work as expected.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> >>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in the KIP,
> under
> >>>>>>>>>>>> "Rejected
> >>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea worth pursuing,
> >>>>>>>> but I
> >>>>>>>>>>>> believe
> >>>>>>>>>>>>>>>>> that it's out of the scope of this KIP, because it
> solves a
> >>>>>>>>>>>> different
> >>>>>>>>>>>>>> set
> >>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of this one has
> >>>>>>>> already
> >>>>>>>>>> grown
> >>>>>>>>>>>>>>>> quite
> >>>>>>>>>>>>>>>>> large!
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> >>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I've updated the KIP (
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>>>> )
> >>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing back "Atomic
> >>>>>>>>>>>> Checkpointing"
> >>>>>>>>>>>>>>>> (for
> >>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think the one thing
> >>>>>>>> missing is
> >>>>>>>>>>>> some
> >>>>>>>>>>>>>>>>>> changes to metrics (notably the store "flush" metrics
> will
> >>>>>>>> need
> >>>>>>>>>> to
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> renamed to "commit").
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The reason I brought back Atomic Checkpointing was to
> >>>>>>>> decouple
> >>>>>>>>>>>> store
> >>>>>>>>>>>>>>>> flush
> >>>>>>>>>>>>>>>>>> from store commit. This is important, because with
> >>>>>>>> Transactional
> >>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on *every* Task
> >>>>>>>> commit,
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>> just when the StateStore is closing, otherwise our
> >>>>>>>> transaction
> >>>>>>>>>>>> buffer
> >>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>> never be written and persisted, instead growing
> unbounded!
> >>>> I
> >>>>>>>>>>>>>>>> experimented
> >>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a store flush
> >>>>>>>> whenever
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its configured
> >>>>>>>> size, but
> >>>>>>>>>>>> this
> >>>>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer from being
> >>>>>>>>>> configured
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> unbounded, and it still would have required explicit
> >>>>>>>> flushes of
> >>>>>>>>>>>>>> RocksDB,
> >>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory utilization.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right" way to
> >>>>>>>> resolve
> >>>>>>>>>> this
> >>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog offsets that
> >>>>>>>> correspond
> >>>>>>>>>> to
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> most
> >>>>>>>>>>>>>>>>>> recently written records are always atomically written
> to
> >>>>>>>> the
> >>>>>>>>>>>>>> StateStore
> >>>>>>>>>>>>>>>>>> (by writing them to the same transaction buffer), we can
> >>>>>>>> avoid
> >>>>>>>>>>>>>> forcibly
> >>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting RocksDB
> >>>>>>>> flush
> >>>>>>>>>> them
> >>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>> when necessary, without losing any of our consistency
> >>>>>>>> guarantees.
> >>>>>>>>>>>> See
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> updated KIP for more info.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I have fully implemented these changes, although I'm
> still
> >>>>>>>> not
> >>>>>>>>>>>>>> entirely
> >>>>>>>>>>>>>>>>>> happy with the implementation for segmented StateStores,
> >> so
> >>>>>>>> I
> >>>>>>>>>> plan
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests pass. If you'd
> like
> >>>>>>>> to try
> >>>>>>>>>>>> out
> >>>>>>>>>>>>>> or
> >>>>>>>>>>>>>>>>>> review this highly experimental and incomplete branch,
> >> it's
> >>>>>>>>>>>> available
> >>>>>>>>>>>>>>>> here:
> >>>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0
> .
> >>>>>>>> Note:
> >>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>> built
> >>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base to build
> >>>>>>>> and test
> >>>>>>>>>>>> it
> >>>>>>>>>>>>>> on,
> >>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples comparisons in a
> live
> >>>>>>>>>>>>>> environment. I
> >>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's nearer
> >> completion
> >>>>>>>> and
> >>>>>>>>>> has
> >>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>> proven on our main application.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing and testing:
> >>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window) stores
> >>>>>>>>>>>>>>>>>> - Global stores
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> As I do not currently use either of these, so my primary
> >>>>>>>> test
> >>>>>>>>>>>>>>>> environment
> >>>>>>>>>>>>>>>>>> doesn't test these areas.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next week for a few
> >>>>>>>> weeks,
> >>>>>>>>>> so
> >>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>> not have time to move this forward until late August.
> That
> >>>>>>>> said,
> >>>>>>>>>>>> your
> >>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just won't be
> able
> >>>> to
> >>>>>>>>>>>> respond
> >>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>> quickly as usual.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> >>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hi Bruno
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ is not
> >>>>>>>> something
> >>>>>>>>>> I
> >>>>>>>>>>>> had
> >>>>>>>>>>>>>>>>>>> considered.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> What about atomically updating the state store from the
> >>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> checkpoint
> >>>>>>>> (thus,
> >>>>>>>>>>>>>>>> flushing
> >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> >>>>>>>> number of
> >>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>> intervals?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting that we
> >>>>>>>> add an
> >>>>>>>>>>>>>>>> additional
> >>>>>>>>>>>>>>>>>>> config for the max number of commit intervals between
> >>>>>>>>>> checkpoints?
> >>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the transaction
> >>>>>>>> buffers
> >>>>>>>>>> are
> >>>>>>>>>>>>>>>> nearly
> >>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of commit
> intervals
> >>>>>>>> have
> >>>>>>>>>>>>>> elapsed,
> >>>>>>>>>>>>>>>>>>> whichever comes first?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although this
> re-ignites
> >>>>>>>> an
> >>>>>>>>>>>> earlier
> >>>>>>>>>>>>>>>>>>> debate about whether a config should be measured in
> >>>>>>>> "number of
> >>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute time.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the reason I was
> >>>>>>>> pursuing
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>> Atomic
> >>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush from
> >>>>>>>> checkpointing,
> >>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every commit without
> any
> >>>>>>>>>>>> performance
> >>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the "best"
> >>>>>>>> solution,
> >>>>>>>>>>>> but
> >>>>>>>>>>>>>>>> I'm not
> >>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back into this KIP.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I'm currently working on moving all the transactional
> >>>> logic
> >>>>>>>>>>>> directly
> >>>>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with the
> >>>>>>>>>>>>>> StateStore#newTransaction
> >>>>>>>>>>>>>>>>>>> method, and reduces the number of new classes
> introduced,
> >>>>>>>>>>>>>> significantly
> >>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and the
> complexity
> >>>> is
> >>>>>>>>>>>>>> drastically
> >>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic Checkpoints
> into
> >>>>>>>> this
> >>>>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
> >>>>>>>> cadonna@apache.org>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> As far as I understand, you want to atomically update
> >> the
> >>>>>>>> state
> >>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the memtable of a
> >>>> state
> >>>>>>>>>> store
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit time elapsed
> >>>> but
> >>>>>>>>>> after
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that would lead to
> >>>>>>>> exceeding
> >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes before the
> next
> >>>>>>>> commit
> >>>>>>>>>>>>>>>> interval
> >>>>>>>>>>>>>>>>>>>> ends.
> >>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would commit every
> >>>>>>>> commit
> >>>>>>>>>>>> interval
> >>>>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>>>> the state store will only be atomically updated
> roughly
> >>>>>>>> every
> >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of data. Also
> IQ
> >>>>>>>> would
> >>>>>>>>>>>> then
> >>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>> see new data roughly every
> >>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> >>>>>>>>>>>>>>>>>>>> After a failure the state store needs to restore up to
> >>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Is this correct?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> What about atomically updating the state store from
> the
> >>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the
> checkpoint
> >>>>>>>> (thus,
> >>>>>>>>>>>>>>>> flushing
> >>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> >>>>>>>> number of
> >>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have the same delay
> >>>> for
> >>>>>>>>>>>> records
> >>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because both would
> >>>>>>>> appear
> >>>>>>>>>> when
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However, after a
> failure
> >>>>>>>> the
> >>>>>>>>>>>> state
> >>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>> still needs to restore up to
> >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> it might restore data that is already in the state
> store
> >>>>>>>>>> because
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable offset (i.e.
> the
> >>>>>>>> last
> >>>>>>>>>>>>>> committed
> >>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring data that
> is
> >>>>>>>> already
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should not violated.
> >>>>>>>>>>>>>>>>>>>> This solution needs at least one new config to specify
> >>>>>>>> when a
> >>>>>>>>>>>>>>>> checkpoint
> >>>>>>>>>>>>>>>>>>>> should be written.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail that does
> not
> >>>>>>>> change
> >>>>>>>>>>>>>>>> anything
> >>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit interval is 30
> >>>>>>>> seconds,
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>>>> five
> >>>>>>>>>>>>>>>>>>>> seconds.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch on our
> staging
> >>>>>>>>>>>>>> environment,
> >>>>>>>>>>>>>>>>>>>>> putting it through its paces in our non-trivial
> >>>>>>>> application.
> >>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate that we saw
> the
> >>>>>>>> last
> >>>>>>>>>>>> time
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP, but this
> time,
> >> I
> >>>>>>>>>> think I
> >>>>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>>>> why.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called
> at
> >>>>>>>> the end
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> Task
> >>>>>>>>>>>>>>>>>>>>> commit process, has the following behaviour:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>           - Under ALOS: checkpoint the state stores.
> >> This
> >>>>>>>>>> includes
> >>>>>>>>>>>>>>>>>>>>>           flushing memtables in RocksDB. This is
> >>>> acceptable
> >>>>>>>>>>>> because the
> >>>>>>>>>>>>>>>>>>>> default
> >>>>>>>>>>>>>>>>>>>>>           commit.interval.ms is 5 seconds, so
> forcibly
> >>>>>>>> flushing
> >>>>>>>>>>>>>> memtables
> >>>>>>>>>>>>>>>>>>>> every 5
> >>>>>>>>>>>>>>>>>>>>>           seconds is acceptable for most
> applications.
> >>>>>>>>>>>>>>>>>>>>>           - Under EOS: checkpointing is not done,
> >> *unless*
> >>>>>>>> it's
> >>>>>>>>>>>> being
> >>>>>>>>>>>>>>>>>>>> forced, due
> >>>>>>>>>>>>>>>>>>>>>           to e.g. the Task closing or being revoked.
> >> This
> >>>>>>>> means
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>> under
> >>>>>>>>>>>>>>>>>>>> normal
> >>>>>>>>>>>>>>>>>>>>>           processing conditions, the state stores
> will
> >> not
> >>>>>>>> be
> >>>>>>>>>>>>>>>> checkpointed,
> >>>>>>>>>>>>>>>>>>>> and will
> >>>>>>>>>>>>>>>>>>>>>           not have memtables flushed at all , unless
> >>>> RocksDB
> >>>>>>>>>>>> decides to
> >>>>>>>>>>>>>>>>>>>> flush them on
> >>>>>>>>>>>>>>>>>>>>>           its own. Checkpointing stores and
> >> force-flushing
> >>>>>>>> their
> >>>>>>>>>>>>>> memtables
> >>>>>>>>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>>>>>>>>>           done when a Task is being closed.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at
> >>>> least
> >>>>>>>>>> *some*
> >>>>>>>>>>>>>>>> normal
> >>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB
> transaction
> >>>>>>>>>> buffers
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets are synced to
> >>>>>>>> disk to
> >>>>>>>>>>>>>> prevent
> >>>>>>>>>>>>>>>>>>>>> restores from getting out of hand. Consequently, my
> >>>>>>>> current
> >>>>>>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit, which
> is
> >>>>>>>> far too
> >>>>>>>>>>>>>>>>>>>> frequent.
> >>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records, which
> is
> >> a
> >>>>>>>>>> change
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>> flush
> >>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing performance problems
> for
> >>>>>>>> some
> >>>>>>>>>>>>>>>>>>>> applications.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm
> currently
> >>>>>>>> leaning
> >>>>>>>>>>>>>>>> towards
> >>>>>>>>>>>>>>>>>>>>> using the statestore.transaction.buffer.max.bytes
> >>>>>>>>>> configuration
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to exceed it.
> This
> >>>>>>>> would
> >>>>>>>>>>>>>>>>>>>> complement the
> >>>>>>>>>>>>>>>>>>>>> existing "early Task commit" functionality that this
> >>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>>>>>>>> provides, in the following way:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>           - Currently, we use
> >>>>>>>>>>>> statestore.transaction.buffer.max.bytes
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> force an
> >>>>>>>>>>>>>>>>>>>>>           early Task commit if processing more
> records
> >>>> would
> >>>>>>>>>> cause
> >>>>>>>>>>>> our
> >>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>           transactions to exceed the memory assigned
> to
> >>>>>>>> them.
> >>>>>>>>>>>>>>>>>>>>>           - New functionality: when a Task *does*
> >> commit,
> >>>>>>>> we will
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>> checkpoint
> >>>>>>>>>>>>>>>>>>>>>           the stores (and hence flush the transaction
> >>>>>>>> buffers)
> >>>>>>>>>>>> unless
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> expect to
> >>>>>>>>>>>>>>>>>>>>>           cross the
> >>>> statestore.transaction.buffer.max.bytes
> >>>>>>>>>>>> threshold
> >>>>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>>>> the next
> >>>>>>>>>>>>>>>>>>>>>           commit
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> >>>>>>>>>>>> nick.telford@gmail.com
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I meant in terms of
> >>>>>>>>>>>> understanding
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> performance profile under various circumstances. The
> >>>>>>>> more
> >>>>>>>>>>>> complex
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would be for users
> to
> >>>>>>>>>>>> understand
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> performance they see. For example, spilling records
> to
> >>>>>>>> disk
> >>>>>>>>>>>> when
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold would, I
> >> expect,
> >>>>>>>>>> reduce
> >>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write throughput could
> >> be
> >>>>>>>>>>>>>> unexpected,
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>> potentially difficult to diagnose/understand for
> >> users.
> >>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early commit" concept is
> >>>>>>>>>> relatively
> >>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document, and
> >>>> conceptually
> >>>>>>>>>> fairly
> >>>>>>>>>>>>>>>>>>>> obvious to
> >>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to make it
> >> easier
> >>>>>>>> to
> >>>>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>> it happens though.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
> >>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an indirect way
> of
> >>>>>>>>>> telling
> >>>>>>>>>>>>>>>>>>>> StateStores
> >>>>>>>>>>>>>>>>>>>>>> whether they should be transactional. READ_COMMITTED
> >>>>>>>>>>>> essentially
> >>>>>>>>>>>>>>>>>>>> requires
> >>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that two threads
> >>>>>>>> calling
> >>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes from the
> >> other
> >>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all
> >>>>>>>> bets are
> >>>>>>>>>>>> off,
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe written records
> at
> >>>>>>>> any
> >>>>>>>>>>>> time,
> >>>>>>>>>>>>>>>>>>>> which is
> >>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That said,
> StateStores
> >>>>>>>> are
> >>>>>>>>>> free
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> implement
> >>>>>>>>>>>>>>>>>>>>>> these guarantees however they can, which is a bit
> more
> >>>>>>>>>> relaxed
> >>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions". For example,
> >>>> with
> >>>>>>>>>>>> RocksDB
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
> >>>>>>>>>> "transactions",
> >>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the database.
> But
> >>>>>>>> with
> >>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>> storage
> >>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to *always* use
> >>>>>>>> transactions,
> >>>>>>>>>>>> even
> >>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines that
> >> don't
> >>>>>>>>>> provide
> >>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation guarantees can be
> met
> >>>>>>>> using a
> >>>>>>>>>>>>>>>>>>>> different
> >>>>>>>>>>>>>>>>>>>>>> technique.
> >>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore interface
> as
> >>>>>>>>>> loosely
> >>>>>>>>>>>>>>>> coupled
> >>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> >>>>>>>> implementers
> >>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>> freedom, and
> >>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge required.
> >>>>>>>>>>>>>>>>>>>>>> That said, I understand that "IsolationLevel" might
> >> not
> >>>>>>>> be
> >>>>>>>>>> the
> >>>>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it much more
> >>>>>>>> explicit if
> >>>>>>>>>>>>>>>>>>>> required, e.g.
> >>>>>>>>>>>>>>>>>>>>>> boolean transactional()
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 7-8.
> >>>>>>>>>>>>>>>>>>>>>> I can make these changes either later today or
> >>>> tomorrow.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Small update:
> >>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of
> >>>>>>>> issues
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> needed
> >>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass, which is
> >>>>>>>>>> promising,
> >>>>>>>>>>>> but
> >>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>> need to undergo some performance testing. I haven't
> >>>>>>>> (yet)
> >>>>>>>>>>>> worked
> >>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I would
> >>>>>>>> expect
> >>>>>>>>>> that,
> >>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no difference. The
> >> branch
> >>>>>>>> is
> >>>>>>>>>>>>>> available
> >>>>>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> >> if
> >>>>>>>>>> anyone
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> >>>>>>>>>>>> cadonna@apache.org>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually also my
> >>>>>>>> point. I
> >>>>>>>>>>>>>>>> understood
> >>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion path as a way
> >> to
> >>>>>>>> avoid
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> early
> >>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually it is public
> >>>>>>>> API. My
> >>>>>>>>>>>>>> question
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal processing.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the size of the
> >>>>>>>> transaction
> >>>>>>>>>>>>>> buffer
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit affects the whole
> >>>>>>>>>> processing
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a state store is
> not
> >>>>>>>>>>>> confined to
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and changes the
> >>>>>>>> behavior
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you that it is a
> >>>>>>>> decent
> >>>>>>>>>>>>>>>>>>>> compromise. I
> >>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and list the
> >>>>>>>> options to
> >>>>>>>>>>>>>>>> overcome
> >>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all options seem
> >>>> quite
> >>>>>>>>>> heavy
> >>>>>>>>>>>>>>>>>>>> compared
> >>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you mean
> with
> >>>>>>>> "less
> >>>>>>>>>>>>>>>>>>>> predictable
> >>>>>>>>>>>>>>>>>>>>>>> for users", though.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the alternatives
> really
> >>>>>>>>>>>>>> interesting.
> >>>>>>>>>>>>>>>>>>>> But I
> >>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good and we should
> >>>>>>>> continue
> >>>>>>>>>>>> with
> >>>>>>>>>>>>>>>> it!
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on June
> >> 20th:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind putting
> >>>>>>>> isolation
> >>>>>>>>>>>>>> level
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should that also
> be
> >> a
> >>>>>>>> way
> >>>>>>>>>> to
> >>>>>>>>>>>>>> give
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to decide whether
> to
> >>>>>>>> turn on
> >>>>>>>>>>>>>>>>>>>>>>> transactions or not?
> >>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned about how do
> >> you
> >>>>>>>> know
> >>>>>>>>>>>> if a
> >>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written under EOS, if
> you
> >>>>>>>> do not
> >>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>>>> to know if the state store is transactional or not.
> >> If
> >>>>>>>> a
> >>>>>>>>>> state
> >>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be written
> >>>>>>>> during
> >>>>>>>>>>>> normal
> >>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state store is not
> >>>>>>>>>> transactional,
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written under EOS.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 7.
> >>>>>>>>>>>>>>>>>>>>>>> My point was about not only considering the bytes
> in
> >>>>>>>> memory
> >>>>>>>>>> in
> >>>>>>>>>>>>>>>> config
> >>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes
> that
> >>>>>>>> might
> >>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>> spilled
> >>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering whether you
> >> should
> >>>>>>>>>> remove
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory bytes to be
> >> used
> >>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My
> thinking
> >>>>>>>> was
> >>>>>>>>>> that
> >>>>>>>>>>>>>> even
> >>>>>>>>>>>>>>>>>>>> if a
> >>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to disk,
> >> limiting
> >>>>>>>> the
> >>>>>>>>>>>>>> overall
> >>>>>>>>>>>>>>>>>>>> bytes
> >>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it again and
> >>>>>>>> considering
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>> recent
> >>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much sense
> anymore.
> >>>>>>>>>>>>>>>>>>>>>>> I like the name
> >>>>>>>> statestore.transaction.buffer.max.bytes that
> >>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>> proposed.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 8.
> >>>>>>>>>>>>>>>>>>>>>>> A high-level description (without implementation
> >>>>>>>> details) of
> >>>>>>>>>>>> how
> >>>>>>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
> >>>>>>>> transactions,
> >>>>>>>>>>>> state
> >>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be great.
> Would
> >>>> be
> >>>>>>>>>> great
> >>>>>>>>>>>> if
> >>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the behavior in
> >>>>>>>> case of
> >>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional state store
> >>>>>>>> recover
> >>>>>>>>>>>> after a
> >>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the transaction
> buffer,
> >>>>>>>> etc.
> >>>>>>>>>>>> (that
> >>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> >>>>>>>> WriteBatchWithIndex
> >>>>>>>>>>>>>>>>>>>> transactions
> >>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
> >>>>>>>> configured
> >>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>>>> needs to
> >>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns. Ultimately, any
> >>>>>>>> approach
> >>>>>>>>>>>> based
> >>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>>> cracking
> >>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in ways it's
> not
> >>>>>>>> really
> >>>>>>>>>>>>>>>> designed
> >>>>>>>>>>>>>>>>>>>>>>> for is
> >>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen performance or
> >>>>>>>> consistency
> >>>>>>>>>>>> issues.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing these early
> >>>>>>>> commits?
> >>>>>>>>>>>> While
> >>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>> ideal, I
> >>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to ensure
> >>>>>>>> consistency
> >>>>>>>>>>>> whilst
> >>>>>>>>>>>>>>>>>>>>>>> maintaining
> >>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> >>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
> >>>>>>>> complicated, and
> >>>>>>>>>>>> might
> >>>>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for users as a
> >>>>>>>> consequence.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is
> >>>>>>>> growing a
> >>>>>>>>>>>> bit
> >>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas for
> future
> >>>>>>>>>>>>>>>> improvements, I
> >>>>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope down to a
> design
> >>>>>>>> that
> >>>>>>>>>>>>>> achieves
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> most
> >>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized restorations
> >>>>>>>> during
> >>>>>>>>>> dirty
> >>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that this KIP
> >>>>>>>> produces
> >>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>> ultimately
> >>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially if the bulk
> of
> >>>>>>>> it is
> >>>>>>>>>>>>>> internal
> >>>>>>>>>>>>>>>>>>>>>>>> behaviour.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week trying to
> >>>>>>>> re-work
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
> >>>>>>>> newTransaction()
> >>>>>>>>>>>>>> method,
> >>>>>>>>>>>>>>>>>>>> such
> >>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of
> RocksDBStore.
> >>>>>>>> That
> >>>>>>>>>>>> way, if
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>>>> want to
> >>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the future, like
> the
> >>>>>>>> SST
> >>>>>>>>>> file
> >>>>>>>>>>>>>>>>>>>> management
> >>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with little/no
> >>>> API
> >>>>>>>>>>>> changes.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>

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

Posted by Bruno Cadonna <ca...@apache.org>.
Hi Nick,

1.
Yeah, you are probably right that it does not make too much sense. 
Thanks for the clarification!


4.
Yes, sorry for the back and forth, but I think for the sake of the KIP 
it is better to let the ALOS behavior as it is for now due to the 
possible issues you would run into. Maybe we can find a solution in the 
future. Now the question returns to whether we really need 
default.state.isolation.level. Maybe the config could be the feature 
flag Sophie requested.


5.
There 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 committedOffsetFor().


Best,
Bruno


On 10/13/23 3:21 PM, Nick Telford wrote:
> Hi Bruno,
> 
> Thanks for getting back to me.
> 
> 1.
> I think this should be possible. Are you thinking of the situation where a
> user may downgrade to a previous version of Kafka Streams? In that case,
> sadly, the RocksDBStore would get wiped by the older version of Kafka
> Streams anyway, because that version wouldn't understand the extra column
> family (that holds offsets), so the missing Position file would
> automatically get rebuilt when the store is rebuilt from the changelog.
> Are there other situations than downgrade where a transactional store could
> be replaced by a non-transactional one? I can't think of any.
> 
> 2.
> Ahh yes, the Test Plan - my Kryptonite! This section definitely needs to be
> fleshed out. I'll work on that. How much detail do you need?
> 
> 3.
> See my previous email discussing this.
> 
> 4.
> Hmm, this is an interesting point. Are you suggesting that under ALOS
> READ_COMMITTED should not be supported?
> 
> Regards,
> Nick
> 
> On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <ca...@apache.org> wrote:
> 
>> Hi Nick,
>>
>> I think the KIP is converging!
>>
>>
>> 1.
>> I am wondering whether it makes sense to write the position file during
>> close as we do for the checkpoint file, so that in case the state store
>> is replaced with a non-transactional state store the non-transactional
>> state store finds the position file. I think, this is not strictly
>> needed, but would be a nice behavior instead of just deleting the
>> position file.
>>
>>
>> 2.
>> The test plan does not mention integration tests. Do you not need to
>> extend existing ones and add new ones. Also for upgrading and
>> downgrading you might need integration and/or system tests.
>>
>>
>> 3.
>> I think Sophie made a point. Although, IQ reading from uncommitted data
>> under EOS might be considered a bug by some people. Thus, your KIP would
>> fix a bug rather than changing the intended behavior. However, I also
>> see that a feature flag would help users that rely on this buggy
>> behavior (at least until AK 4.0).
>>
>>
>> 4.
>> This is related to the previous point. I assume that the difference
>> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is that in the
>> former you enable transactions on the state store and in the latter you
>> disable them. If my assumption is correct, I think that is an issue.
>> Let's assume under ALOS Streams fails over a couple of times more or
>> less at the same step in processing after value 3 is added to an
>> aggregation but the offset of the corresponding input record was not
>> committed. Without transactions disabled, the aggregation value would
>> increase by 3 for each failover. With transactions enabled, value 3
>> would only be added to the aggregation once when the offset of the input
>> record is committed and the transaction finally completes. So the
>> content of the state store would change depending on the configuration
>> for IQ. IMO, the content of the state store should be independent from
>> IQ. Given this issue, I propose to not use transactions with ALOS at
>> all. I was a big proponent of using transactions with ALOS, but I
>> realized that transactions with ALOS is not as easy as enabling
>> transactions on state stores. Another aspect that is problematic is that
>> the changelog topic which actually replicates the state store is not
>> transactional under ALOS. Thus, it might happen that the state store and
>> the changelog differ in their content. All of this is maybe solvable
>> somehow, but for the sake of this KIP, I would leave it for the future.
>>
>>
>> Best,
>> Bruno
>>
>>
>>
>> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
>>> Hey Nick! First of all thanks for taking up this awesome feature, I'm
>> sure
>>> every single
>>> Kafka Streams user and dev would agree that it is sorely needed.
>>>
>>> I've just been catching up on the KIP and surrounding discussion, so
>> please
>>> forgive me
>>> for any misunderstandings or misinterpretations of the current plan and
>>> don't hesitate to
>>> correct me.
>>>
>>> Before I jump in, I just want to say that having seen this drag on for so
>>> long, my singular
>>> goal in responding is to help this KIP past a perceived impasse so we can
>>> finally move on
>>> to voting and implementing it. Long discussions are to be expected for
>>> major features like
>>> this but it's completely on us as the Streams devs to make sure there is
>> an
>>> end in sight
>>> for any ongoing discussion.
>>>
>>> With that said, it's my understanding that the KIP as currently proposed
>> is
>>> just not tenable
>>> for Kafka Streams, and would prevent some EOS users from upgrading to the
>>> version it
>>> first appears in. Given that we can't predict or guarantee whether any of
>>> the followup KIPs
>>> would be completed in the same release cycle as this one, we need to make
>>> sure that the
>>> feature is either compatible with all current users or else
>> feature-flagged
>>> so that they may
>>> opt in/out.
>>>
>>> Therefore, IIUC we need to have either (or both) of these as
>>> fully-implemented config options:
>>> 1. default.state.isolation.level
>>> 2. enable.transactional.state.stores
>>>
>>> This way EOS users for whom read_committed semantics are not viable can
>>> still upgrade,
>>> and either use the isolation.level config to leverage the new txn state
>>> stores without sacrificing
>>> their application semantics, or else simply keep the transactional state
>>> stores disabled until we
>>> are able to fully implement the isolation level configuration at either
>> an
>>> application or query level.
>>>
>>> Frankly you are the expert here and know much more about the tradeoffs in
>>> both semantics and
>>> effort level of implementing one of these configs vs the other. In my
>>> opinion, either option would
>>> be fine and I would leave the decision of which one to include in this
>> KIP
>>> completely up to you.
>>> I just don't see a way for the KIP to proceed without some variation of
>> the
>>> above that would allow
>>> EOS users to opt-out of read_committed.
>>>
>>> (If it's all the same to you, I would recommend 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 that gets released.
>>> So personally, my choice would be to add the feature flag and leave it
>> off
>>> by default. If all 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
>>>
>>> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <ni...@gmail.com>
>> wrote:
>>>
>>>> Hi Bruno,
>>>>
>>>> Agreed, I can live with that for now.
>>>>
>>>> In an effort to keep the scope of this KIP from expanding, I'm leaning
>>>> towards just providing a configurable default.state.isolation.level and
>>>> removing IsolationLevel from the StateStoreContext. This would be
>>>> compatible with adding support for query-time IsolationLevels in the
>>>> future, whilst providing a way for users to select an isolation level
>> now.
>>>>
>>>> The big problem with this, however, is that if a user selects
>>>> processing.mode
>>>> = "exactly-once(-v2|-beta)", and default.state.isolation.level =
>>>> "READ_UNCOMMITTED", we need to guarantee that the data isn't written to
>>>> disk until commit() is called, but we also need to permit IQ threads to
>>>> read from the ongoing transaction.
>>>>
>>>> A simple solution would be to (temporarily) forbid this combination of
>>>> configuration, and have default.state.isolation.level automatically
>> switch
>>>> to READ_COMMITTED when processing.mode is anything other than
>>>> at-least-once. Do you think this would be acceptable?
>>>>
>>>> In a later KIP, we can add support for query-time isolation levels and
>>>> solve this particular problem there, which would relax this restriction.
>>>>
>>>> Regards,
>>>> Nick
>>>>
>>>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <ca...@apache.org>
>> 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,
>>>>> Bruno
>>>>>
>>>>> On 9/18/23 7:12 PM, Nick Telford wrote:
>>>>>> 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 <ni...@gmail.com>
>>>>> wrote:
>>>>>>
>>>>>>> Hi everyone,
>>>>>>>
>>>>>>> I agree that having IsolationLevel be determined at query-time is the
>>>>>>> ideal design, but there are a few sticking points:
>>>>>>>
>>>>>>> 1.
>>>>>>> There needs to be some way to communicate the IsolationLevel down to
>>>> the
>>>>>>> RocksDBStore itself, so that the query can respect it. Since stores
>>>> are
>>>>>>> "layered" in functionality (i.e. ChangeLoggingStore, MeteredStore,
>>>>> etc.),
>>>>>>> we need some way to deliver that information to the bottom layer. For
>>>>> IQv2,
>>>>>>> we can use the existing State#query() method, but IQv1 has no way to
>>>> do
>>>>>>> this.
>>>>>>>
>>>>>>> A simple approach, which would potentially open up other options,
>>>> would
>>>>> be
>>>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
>>>>>>> readOnlyView(IsolationLevel isolationLevel) to ReadOnlyKeyValueStore
>>>>> (and
>>>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
>>>>>>>
>>>>>>> 2.
>>>>>>> As mentioned above, RocksDB WriteBatches are not thread-safe, which
>>>>> causes
>>>>>>> a problem if we want to provide READ_UNCOMMITTED Iterators. I also
>>>> had a
>>>>>>> look at RocksDB Transactions[1], but they solve a very different
>>>>> problem,
>>>>>>> and have the same thread-safety issue.
>>>>>>>
>>>>>>> One possible approach that I mentioned is chaining WriteBatches:
>> every
>>>>>>> time a new Interactive Query is received (i.e. readOnlyView, see
>>>> above,
>>>>>>> is called) we "freeze" the existing WriteBatch, and start a new one
>>>> for
>>>>> new
>>>>>>> writes. The Interactive Query queries the "chain" of previous
>>>>> WriteBatches
>>>>>>> + the underlying database; while the StreamThread starts writing to
>>>> the
>>>>>>> *new* WriteBatch. On-commit, the StreamThread would write *all*
>>>>>>> WriteBatches in the chain to the database (that have not yet been
>>>>> written).
>>>>>>>
>>>>>>> WriteBatches would be closed/freed only when they have been both
>>>>>>> committed, and all open Interactive Queries on them have been closed.
>>>>> This
>>>>>>> would require some reference counting.
>>>>>>>
>>>>>>> Obviously a drawback of this approach is the potential for increased
>>>>>>> memory usage: if an Interactive Query is long-lived, for example by
>>>>> doing a
>>>>>>> full scan over a large database, or even just pausing in the middle
>> of
>>>>> an
>>>>>>> iteration, then the existing chain of WriteBatches could be kept
>>>> around
>>>>> for
>>>>>>> a long time, potentially forever.
>>>>>>>
>>>>>>> --
>>>>>>>
>>>>>>> A.
>>>>>>> Going off on a tangent, it looks like in addition to supporting
>>>>>>> READ_COMMITTED queries, we could go further and support
>>>> REPEATABLE_READ
>>>>>>> queries (i.e. where subsequent reads to the same key in the same
>>>>>>> Interactive Query are guaranteed to yield the same value) by making
>>>> use
>>>>> of
>>>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so the
>> performance
>>>>>>> impact is likely to be negligible, but they do require that the
>>>>> Interactive
>>>>>>> Query session can be explicitly closed.
>>>>>>>
>>>>>>> This could be achieved if we made the above readOnlyView interface
>>>> look
>>>>>>> more like:
>>>>>>>
>>>>>>> interface ReadOnlyKeyValueView<K, V> implements
>>>> ReadOnlyKeyValueStore<K,
>>>>>>> V>, AutoCloseable {}
>>>>>>>
>>>>>>> interface ReadOnlyKeyValueStore<K, V> {
>>>>>>>        ...
>>>>>>>        ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel
>>>>> isolationLevel);
>>>>>>> }
>>>>>>>
>>>>>>> But this would be a breaking change, as existing IQv1 queries are
>>>>>>> guaranteed to never call store.close(), and therefore these would
>> leak
>>>>>>> memory under REPEATABLE_READ.
>>>>>>>
>>>>>>> B.
>>>>>>> One thing that's notable: MyRocks states that they support
>>>>> READ_COMMITTED
>>>>>>> and REPEATABLE_READ, but they make no mention of
>>>> READ_UNCOMMITTED[3][4].
>>>>>>> This could be because doing so is technically difficult/impossible
>>>> using
>>>>>>> the primitives available in RocksDB.
>>>>>>>
>>>>>>> --
>>>>>>>
>>>>>>> Lucas, to address your points:
>>>>>>>
>>>>>>> U1.
>>>>>>> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
>>>>>>> implementations of StateStore that do not support atomic writes.
>>>>> Obviously
>>>>>>> in those cases, the guarantees Kafka Streams provides/expects would
>> be
>>>>>>> relaxed. Do you think we should require all implementations to
>> support
>>>>>>> atomic writes?
>>>>>>>
>>>>>>> U2.
>>>>>>> Stores can support multiple IsolationLevels. As we've discussed
>> above,
>>>>> the
>>>>>>> ideal scenario would be to specify the IsolationLevel at query-time.
>>>>>>> Failing that, I think the second-best approach is to define the
>>>>>>> IsolationLevel for *all* queries based on the processing.mode, which
>>>> is
>>>>>>> what the default StateStoreContext#isolationLevel() achieves. Would
>>>> you
>>>>>>> prefer an alternative?
>>>>>>>
>>>>>>> While the existing implementation is equivalent to READ_UNCOMMITTED,
>>>>> this
>>>>>>> can yield unexpected results/errors under EOS, if a transaction is
>>>>> rolled
>>>>>>> back. While this would be a change in behaviour for users, it would
>>>> look
>>>>>>> more like a bug fix than a breaking change. That said, we *could*
>> make
>>>>> it
>>>>>>> configurable, and default to the existing behaviour
>> (READ_UNCOMMITTED)
>>>>>>> instead of inferring it from the processing.mode?
>>>>>>>
>>>>>>> N1, N2.
>>>>>>> These were only primitives to avoid boxing costs, but since this is
>>>> not
>>>>> a
>>>>>>> performance sensitive area, it should be fine to change if that's
>>>>> desirable.
>>>>>>>
>>>>>>> N3.
>>>>>>> It's because the store "manages its own offsets", which includes both
>>>>>>> committing the offset, *and providing it* via getCommittedOffset().
>>>>>>> Personally, I think "managesOffsets" conveys this best, but I don't
>>>> mind
>>>>>>> changing it if the nomenclature is unclear.
>>>>>>>
>>>>>>> Sorry for the massive emails/essays!
>>>>>>> --
>>>>>>> Nick
>>>>>>>
>>>>>>> 1: https://github.com/facebook/rocksdb/wiki/Transactions
>>>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
>>>>>>> 3: https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
>>>>>>> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
>>>>>>>
>>>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
>>>>>>> <lb...@confluent.io.invalid> wrote:
>>>>>>>
>>>>>>>> Hi Nick,
>>>>>>>>
>>>>>>>> since I last read it in April, the KIP has become much cleaner and
>>>>>>>> easier to read. Great work!
>>>>>>>>
>>>>>>>> It feels to me the last big open point is whether we can implement
>>>>>>>> isolation level as a query parameter. I understand that there are
>>>>>>>> implementation concerns, but as Colt says, it would be a great
>>>>>>>> addition, and would also simplify the migration path for this
>> change.
>>>>>>>> Is the implementation problem you mentioned caused by the WriteBatch
>>>>>>>> not having a notion of a snapshot, as the underlying DB iterator
>>>> does?
>>>>>>>> In that case, I am not sure a chain of WriteBatches as you propose
>>>>>>>> would fully solve the problem, but maybe I didn't dig enough into
>> the
>>>>>>>> details to fully understand it.
>>>>>>>>
>>>>>>>> If it's not possible to implement it now, would it be an option to
>>>>>>>> make sure in this KIP that we do not fully close the door on
>>>> per-query
>>>>>>>> isolation levels in the interface, as it may be possible to
>> implement
>>>>>>>> the missing primitives in RocksDB or Speedb in the future.
>>>>>>>>
>>>>>>>> Understanding:
>>>>>>>>
>>>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to be persisted
>>>>>>>> atomically with the records?
>>>>>>>> * U2) Don't understand the default implementation of
>>>> `isolationLevel`.
>>>>>>>> The isolation level should be a property of the underlying store,
>> and
>>>>>>>> not be defined by the default config? Existing stores probably don't
>>>>>>>> guarantee READ_COMMITTED, so the default should be to return
>>>>>>>> READ_UNCOMMITTED.
>>>>>>>>
>>>>>>>> Nits:
>>>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong` return type,
>> to
>>>>>>>> avoid the `null`?
>>>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
>>>>>>>> return type, to avoid the `-1`?
>>>>>>>> * N3) I don't understand why `managesOffsets` uses the 'manage'
>> verb,
>>>>>>>> whereas all other methods use the "commits" verb. I'd suggest
>>>>>>>> `commitsOffsets`.
>>>>>>>>
>>>>>>>> Either way, it feels this KIP is very close to the finish line, I'm
>>>>>>>> looking forward to seeing this in production!
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Lucas
>>>>>>>>
>>>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <co...@littlehorse.io>
>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Making IsolationLevel a query-time constraint, rather than linking
>>>> it
>>>>>>>> to
>>>>>>>>> the processing.guarantee.
>>>>>>>>>
>>>>>>>>> As I understand it, would this allow even a user of EOS to control
>>>>>>>> whether
>>>>>>>>> reading committed or uncommitted records? If so, I am highly in
>>>> favor
>>>>> of
>>>>>>>>> this.
>>>>>>>>>
>>>>>>>>> I know that I was one of the early people to point out the current
>>>>>>>>> shortcoming that IQ reads uncommitted records, but just this
>>>> morning I
>>>>>>>>> realized a pattern we use which means that (for certain queries)
>> our
>>>>>>>> system
>>>>>>>>> needs to be able to read uncommitted records, which is the current
>>>>>>>> behavior
>>>>>>>>> of Kafka Streams in EOS.***
>>>>>>>>>
>>>>>>>>> If IsolationLevel being a query-time decision allows for this, then
>>>>> that
>>>>>>>>> would be amazing. I would also vote that the default behavior
>> should
>>>>> be
>>>>>>>> for
>>>>>>>>> reading uncommitted records, because it is totally possible for a
>>>>> valid
>>>>>>>>> application to depend on that behavior, and breaking it in a minor
>>>>>>>> release
>>>>>>>>> might be a bit strong.
>>>>>>>>>
>>>>>>>>> *** (Note, for the curious reader....) Our use-case/query pattern
>>>> is a
>>>>>>>> bit
>>>>>>>>> complex, but reading "uncommitted" records is actually safe in our
>>>>> case
>>>>>>>>> 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 soon as the processor processes the result. If
>>>>> they
>>>>>>>>> can't read uncommitted records, they may receive a "201 - Created"
>>>>>>>>> response, immediately followed by a "404 - 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 <
>>>> nick.telford@gmail.com>
>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Addendum:
>>>>>>>>>>
>>>>>>>>>> I think we would also face the same problem with the approach John
>>>>>>>> outlined
>>>>>>>>>> earlier (using the record cache as a transaction buffer and
>>>> flushing
>>>>>>>> it
>>>>>>>>>> straight to SST files). This is because the record cache (the
>>>>>>>> ThreadCache
>>>>>>>>>> class) is not thread-safe, so every commit would invalidate open
>> IQ
>>>>>>>>>> Iterators in the same way that RocksDB WriteBatches do.
>>>>>>>>>> --
>>>>>>>>>> Nick
>>>>>>>>>>
>>>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
>>>> nick.telford@gmail.com>
>>>>>>>>>> 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.
>>>>>>>>>>> 2. Making IsolationLevel a query-time constraint, rather than
>>>>>>>> linking it
>>>>>>>>>>> to the processing.guarantee.
>>>>>>>>>>>
>>>>>>>>>>> There's a wrinkle that makes this a challenge: Interactive
>> Queries
>>>>>>>> that
>>>>>>>>>>> open an Iterator, when using transactions and READ_UNCOMMITTED.
>>>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries need to be
>>>> able
>>>>>>>> to
>>>>>>>>>>> read records from the currently uncommitted transaction buffer
>>>>>>>>>>> (WriteBatch). This includes for Iterators, which should iterate
>>>>>>>> both the
>>>>>>>>>>> transaction buffer and underlying database (using
>>>>>>>>>>> WriteBatch#iteratorWithBase()).
>>>>>>>>>>>
>>>>>>>>>>> The issue is that when the StreamThread commits, it writes the
>>>>>>>> current
>>>>>>>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing
>>>> the
>>>>>>>>>>> WriteBatch while an Interactive Query holds an open Iterator on
>> it
>>>>>>>> will
>>>>>>>>>>> invalidate the Iterator. Worse, it turns out that Iterators over
>> a
>>>>>>>>>>> WriteBatch become invalidated not just when the WriteBatch is
>>>>>>>> cleared,
>>>>>>>>>> but
>>>>>>>>>>> also when the Iterators' current key receives a new write.
>>>>>>>>>>>
>>>>>>>>>>> Now that I'm writing this, I remember that this is the major
>>>> reason
>>>>>>>> that
>>>>>>>>>> I
>>>>>>>>>>> switched the original design from having a query-time
>>>>>>>> IsolationLevel to
>>>>>>>>>>> having the IsolationLevel linked to the transactionality of the
>>>>>>>> stores
>>>>>>>>>>> themselves.
>>>>>>>>>>>
>>>>>>>>>>> It *might* be possible to resolve this, by having a "chain" of
>>>>>>>>>>> WriteBatches, with the StreamThread switching to a new WriteBatch
>>>>>>>>>> whenever
>>>>>>>>>>> a new Interactive Query attempts to read from the database, but
>>>> that
>>>>>>>>>> could
>>>>>>>>>>> cause some performance problems/memory pressure when subjected to
>>>> a
>>>>>>>> high
>>>>>>>>>>> Interactive Query load. It would also reduce the efficiency of
>>>>>>>>>> WriteBatches
>>>>>>>>>>> on-commit, as we'd have to write N WriteBatches, where N is the
>>>>>>>> number of
>>>>>>>>>>> Interactive Queries since the last commit.
>>>>>>>>>>>
>>>>>>>>>>> I realise this is getting into the weeds of the implementation,
>>>> and
>>>>>>>> you'd
>>>>>>>>>>> rather we focus on the API for now, but I think it's important to
>>>>>>>>>> consider
>>>>>>>>>>> how to implement the desired API, in case we come up with an API
>>>>>>>> that
>>>>>>>>>>> cannot be implemented efficiently, or even at all!
>>>>>>>>>>>
>>>>>>>>>>> Thoughts?
>>>>>>>>>>> --
>>>>>>>>>>> Nick
>>>>>>>>>>>
>>>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <cadonna@apache.org
>>>
>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>
>>>>>>>>>>>> 6.
>>>>>>>>>>>> Of course, you are right! My bad!
>>>>>>>>>>>> Wiping out the state in the downgrading case is fine.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 3a.
>>>>>>>>>>>> Focus on the public facing changes for the KIP. We will manage
>> 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.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Best,
>>>>>>>>>>>> Bruno
>>>>>>>>>>>>
>>>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
>>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for getting back to me!
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2.
>>>>>>>>>>>>> The fact that implementations can always track estimated memory
>>>>>>>> usage
>>>>>>>>>> in
>>>>>>>>>>>>> the wrapper is a good point. I can remove -1 as an option, and
>>>>>>>> I'll
>>>>>>>>>>>> clarify
>>>>>>>>>>>>> the JavaDoc that 0 is not just for non-transactional stores,
>>>>>>>> which is
>>>>>>>>>>>>> currently misleading.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 6.
>>>>>>>>>>>>> The problem with catching the exception in the downgrade
>> process
>>>>>>>> is
>>>>>>>>>> that
>>>>>>>>>>>>> would require new code in the Kafka version being downgraded
>> to.
>>>>>>>> Since
>>>>>>>>>>>>> users could conceivably downgrade to almost *any* older version
>>>>>>>> of
>>>>>>>>>> Kafka
>>>>>>>>>>>>> Streams, I'm not sure how we could add that code?
>>>>>>>>>>>>> The only way I can think of doing it would be to provide a
>>>>>>>> dedicated
>>>>>>>>>>>>> downgrade tool, that goes through every local store and removes
>>>>>>>> the
>>>>>>>>>>>>> offsets column families. But that seems like an unnecessary
>>>>>>>> amount of
>>>>>>>>>>>> extra
>>>>>>>>>>>>> code to maintain just to handle a somewhat niche situation,
>> when
>>>>>>>> the
>>>>>>>>>>>>> alternative (automatically wipe and restore stores) should be
>>>>>>>>>>>> acceptable.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've requested.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 3a.
>>>>>>>>>>>>> I agree that IsolationLevel makes more sense at query-time, and
>>>> I
>>>>>>>>>>>> actually
>>>>>>>>>>>>> initially attempted to place the IsolationLevel at query-time,
>>>>>>>> but I
>>>>>>>>>> ran
>>>>>>>>>>>>> into some problems:
>>>>>>>>>>>>> - The key issue is that, under ALOS we're not staging writes in
>>>>>>>>>>>>> transactions, so can't perform writes at the READ_COMMITTED
>>>>>>>> isolation
>>>>>>>>>>>>> level. However, this may be addressed if we decide to *always*
>>>>>>>> use
>>>>>>>>>>>>> transactions as discussed under 3b.
>>>>>>>>>>>>> - IQv1 and IQv2 have quite different implementations. I
>> remember
>>>>>>>>>> having
>>>>>>>>>>>>> some difficulty understanding the IQv1 internals, which made it
>>>>>>>>>>>> difficult
>>>>>>>>>>>>> to determine what needed to be changed. However, I *think* this
>>>>>>>> can be
>>>>>>>>>>>>> addressed for both implementations by wrapping the RocksDBStore
>>>>>>>> in an
>>>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides read methods
>>>>>>>> (get,
>>>>>>>>>>>> etc.)
>>>>>>>>>>>>> to either read directly from the database or from the ongoing
>>>>>>>>>>>> transaction.
>>>>>>>>>>>>> But IQv1 might still be difficult.
>>>>>>>>>>>>> - If IsolationLevel becomes a query constraint, then all other
>>>>>>>>>>>> StateStores
>>>>>>>>>>>>> will need to respect it, including the in-memory stores. This
>>>>>>>> would
>>>>>>>>>>>> require
>>>>>>>>>>>>> us to adapt in-memory stores to stage their writes so they can
>>>> be
>>>>>>>>>>>> isolated
>>>>>>>>>>>>> from READ_COMMITTTED queries. It would also become an important
>>>>>>>>>>>>> consideration for third-party stores on upgrade, as without
>>>>>>>> changes,
>>>>>>>>>>>> they
>>>>>>>>>>>>> would not support READ_COMMITTED queries correctly.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Ultimately, I may need some help making the necessary change to
>>>>>>>> IQv1
>>>>>>>>>> to
>>>>>>>>>>>>> support this, but I don't think it's fundamentally impossible,
>>>>>>>> if we
>>>>>>>>>>>> want
>>>>>>>>>>>>> to pursue this route.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 3b.
>>>>>>>>>>>>> The main reason I chose to keep ALOS un-transactional was to
>>>>>>>> minimize
>>>>>>>>>>>>> behavioural change for most users (I believe most Streams users
>>>>>>>> use
>>>>>>>>>> the
>>>>>>>>>>>>> default configuration, which is ALOS). That said, it's clear
>>>>>>>> that if
>>>>>>>>>>>> ALOS
>>>>>>>>>>>>> also used transactional stores, the only change in behaviour
>>>>>>>> would be
>>>>>>>>>>>> that
>>>>>>>>>>>>> it would become *more correct*, which could be considered a
>> "bug
>>>>>>>> fix"
>>>>>>>>>> by
>>>>>>>>>>>>> users, rather than a change they need to handle.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I believe that performance using transactions (aka. RocksDB
>>>>>>>>>>>> WriteBatches)
>>>>>>>>>>>>> should actually be *better* than the un-batched write-path that
>>>>>>>> is
>>>>>>>>>>>>> currently used[1]. The only "performance" consideration will be
>>>>>>>> the
>>>>>>>>>>>>> increased memory usage that transactions require. Given the
>>>>>>>>>> mitigations
>>>>>>>>>>>> for
>>>>>>>>>>>>> this memory that we have in place, I would expect that this is
>>>>>>>> not a
>>>>>>>>>>>>> problem for most users.
>>>>>>>>>>>>>
>>>>>>>>>>>>> If we're happy to do so, we can make ALOS also use
>> transactions.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>
>>>>>>>>>>>>> Link 1:
>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
>>>> cadonna@apache.org
>>>>>>>>>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for the updates and sorry for the delay on my side!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>> Making the default implementation for flush() a no-op sounds
>>>>>>>> good to
>>>>>>>>>>>> me.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>> I think what was bugging me here is that a third-party state
>>>>>>>> store
>>>>>>>>>>>> needs
>>>>>>>>>>>>>> to implement the state store interface. That means they need
>> to
>>>>>>>>>>>>>> implement a wrapper around the actual state store as we do for
>>>>>>>>>> RocksDB
>>>>>>>>>>>>>> with RocksDBStore. So, a third-party state store can always
>>>>>>>> estimate
>>>>>>>>>>>> the
>>>>>>>>>>>>>> uncommitted bytes, if it wants, because the wrapper can record
>>>>>>>> the
>>>>>>>>>>>> added
>>>>>>>>>>>>>> bytes.
>>>>>>>>>>>>>> One case I can think of where returning -1 makes sense is when
>>>>>>>>>> Streams
>>>>>>>>>>>>>> does not need to estimate the size of the write batch and
>>>>>>>> trigger
>>>>>>>>>>>>>> extraordinary commits, because the third-party state store
>>>>>>>> takes care
>>>>>>>>>>>> of
>>>>>>>>>>>>>> memory. But in that case the method could also just return 0.
>>>>>>>> Even
>>>>>>>>>> that
>>>>>>>>>>>>>> case would be better solved with a method that returns whether
>>>>>>>> the
>>>>>>>>>>>> state
>>>>>>>>>>>>>> store manages itself the memory used for uncommitted bytes or
>>>>>>>> not.
>>>>>>>>>>>>>> Said that, I am fine with keeping the -1 return value, I was
>>>>>>>> just
>>>>>>>>>>>>>> wondering when and if it will be used.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regarding returning 0 for transactional state stores when the
>>>>>>>> batch
>>>>>>>>>> is
>>>>>>>>>>>>>> empty, I was just wondering because you explicitly stated
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> "or {@code 0} if this StateStore does not support
>>>> transactions."
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> So it seemed to me returning 0 could only happen for
>>>>>>>>>> non-transactional
>>>>>>>>>>>>>> state stores.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> a) What do you think if we move the isolation level to IQ (v1
>>>>>>>> and
>>>>>>>>>> v2)?
>>>>>>>>>>>>>> In the end this is the only component that really needs to
>>>>>>>> specify
>>>>>>>>>> the
>>>>>>>>>>>>>> isolation level. It is similar to the Kafka consumer that can
>>>>>>>> choose
>>>>>>>>>>>>>> with what isolation level to read the input topic.
>>>>>>>>>>>>>> For IQv1 the isolation level should go into
>>>>>>>> StoreQueryParameters. For
>>>>>>>>>>>>>> IQv2, I would add it to the Query interface.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> b) Point a) raises the question what should happen during
>>>>>>>>>> at-least-once
>>>>>>>>>>>>>> processing when the state store does not use transactions?
>> John
>>>>>>>> in
>>>>>>>>>> the
>>>>>>>>>>>>>> past proposed to also use transactions on state stores for
>>>>>>>>>>>>>> at-least-once. I like that idea, because it avoids aggregating
>>>>>>>> the
>>>>>>>>>> same
>>>>>>>>>>>>>> records over and over again in the case of a failure. We had a
>>>>>>>> case
>>>>>>>>>> in
>>>>>>>>>>>>>> the past where a Streams applications in at-least-once mode
>> was
>>>>>>>>>> failing
>>>>>>>>>>>>>> continuously for some reasons I do not remember before
>>>>>>>> committing the
>>>>>>>>>>>>>> offsets. After each failover, the app aggregated again and
>>>>>>>> again the
>>>>>>>>>>>>>> same records. Of course the aggregate increased to very wrong
>>>>>>>> values
>>>>>>>>>>>>>> just because of the failover. With transactions on the state
>>>>>>>> stores
>>>>>>>>>> we
>>>>>>>>>>>>>> could have avoided this. The app would have output the same
>>>>>>>> aggregate
>>>>>>>>>>>>>> multiple times (i.e., after each failover) but at least the
>>>>>>>> value of
>>>>>>>>>>>> the
>>>>>>>>>>>>>> aggregate would not depend on the number of failovers.
>>>>>>>> Outputting the
>>>>>>>>>>>>>> same aggregate multiple times would be incorrect under
>>>>>>>> exactly-once
>>>>>>>>>> but
>>>>>>>>>>>>>> it is OK for at-least-once.
>>>>>>>>>>>>>> If it makes sense to add a config to turn on and off
>>>>>>>> transactions on
>>>>>>>>>>>>>> state stores under at-least-once or just use transactions in
>>>>>>>> any case
>>>>>>>>>>>> is
>>>>>>>>>>>>>> a question we should also discuss in this KIP. It depends a
>> bit
>>>>>>>> on
>>>>>>>>>> the
>>>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would add a config.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 4.
>>>>>>>>>>>>>> Your points are all valid. I tend to say to keep the metrics
>>>>>>>> around
>>>>>>>>>>>>>> flush() until we remove flush() completely from the interface.
>>>>>>>> Calls
>>>>>>>>>> to
>>>>>>>>>>>>>> flush() might still exist since existing processors might
>> still
>>>>>>>> call
>>>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For sure, we need
>> to
>>>>>>>>>>>> document
>>>>>>>>>>>>>> how the metrics change due to the transactions in the upgrade
>>>>>>>> notes.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 5.
>>>>>>>>>>>>>> I see. Then you should describe how the .position files are
>>>>>>>> handled
>>>>>>>>>> in
>>>>>>>>>>>>>> a dedicated section of the KIP or incorporate the description
>>>>>>>> in the
>>>>>>>>>>>>>> "Atomic Checkpointing" section instead of only mentioning it
>> in
>>>>>>>> the
>>>>>>>>>>>>>> "Compatibility, 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:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>
>>>>
>> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> It is a bit brittle, but it works.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
>>>>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks for taking the time to review the KIP. I'm back from
>>>>>>>> leave
>>>>>>>>>> now
>>>>>>>>>>>> and
>>>>>>>>>>>>>>> intend to move this forwards as quickly as I can.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Addressing your points:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>> Because flush() is part of the StateStore API, it's exposed
>> to
>>>>>>>>>> custom
>>>>>>>>>>>>>>> Processors, which might be making calls to flush(). This was
>>>>>>>>>> actually
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> case in a few integration tests.
>>>>>>>>>>>>>>> To maintain as much compatibility as possible, I'd prefer not
>>>>>>>> to
>>>>>>>>>> make
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>> an UnsupportedOperationException, as it will cause previously
>>>>>>>>>> working
>>>>>>>>>>>>>>> Processors to start throwing exceptions at runtime.
>>>>>>>>>>>>>>> I agree that it doesn't make sense for it to proxy commit(),
>>>>>>>> though,
>>>>>>>>>>>> as
>>>>>>>>>>>>>>> that would cause it to violate the "StateStores commit only
>>>>>>>> when the
>>>>>>>>>>>> Task
>>>>>>>>>>>>>>> commits" rule.
>>>>>>>>>>>>>>> Instead, I think we should make this a no-op. That way,
>>>>>>>> existing
>>>>>>>>>> user
>>>>>>>>>>>>>>> Processors will continue to work as-before, without violation
>>>>>>>> of
>>>>>>>>>> store
>>>>>>>>>>>>>>> consistency that would be caused by premature flush/commit of
>>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>> data to disk.
>>>>>>>>>>>>>>> What do you think?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore implementation is
>>>>>>>>>>>>>>> transactional, but is unable to estimate the uncommitted
>>>> memory
>>>>>>>>>> usage,
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> method will return -1.
>>>>>>>>>>>>>>> The intention here is to permit third-party implementations
>>>>>>>> that may
>>>>>>>>>>>> not
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>> able to estimate memory usage.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Yes, it will be 0 when nothing has been written to the store
>>>>>>>> yet. I
>>>>>>>>>>>>>> thought
>>>>>>>>>>>>>>> that was implied by "This method will return an approximation
>>>>>>>> of the
>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>> would be freed by the next call to {@link #commit(Map)}" and
>>>>>>>>>> "@return
>>>>>>>>>>>> The
>>>>>>>>>>>>>>> approximate size of all records awaiting {@link
>>>> #commit(Map)}",
>>>>>>>>>>>> however,
>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you think this is
>>>>>>>> unclear?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>> I realise this is probably the most contentious point in my
>>>>>>>> design,
>>>>>>>>>>>> and
>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>> open to changing it if I'm unable to convince you of the
>>>>>>>> benefits.
>>>>>>>>>>>>>>> Nevertheless, here's my argument:
>>>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly provided
>>>>>>>> StateStores
>>>>>>>>>> to
>>>>>>>>>>>>>>> query, and it may be important for users to programmatically
>>>>>>>> know
>>>>>>>>>>>> which
>>>>>>>>>>>>>>> mode the StateStore is operating under. If we simply provide
>>>> an
>>>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the internal streams
>>>>>>>>>>>> engine), or
>>>>>>>>>>>>>>> similar, then users will need to understand the operation and
>>>>>>>>>>>>>> consequences
>>>>>>>>>>>>>>> of each available processing mode and how it pertains to
>> their
>>>>>>>>>>>>>> StateStore.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Interactive Query users aren't the only people that care
>> about
>>>>>>>> the
>>>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore: implementers
>>>> of
>>>>>>>>>> custom
>>>>>>>>>>>>>>> StateStores also need to understand the behaviour expected of
>>>>>>>> their
>>>>>>>>>>>>>>> implementation. KIP-892 introduces some assumptions into the
>>>>>>>> Streams
>>>>>>>>>>>>>> Engine
>>>>>>>>>>>>>>> about how StateStores operate under each processing mode, and
>>>>>>>> it's
>>>>>>>>>>>>>>> important that custom implementations adhere to those
>>>>>>>> assumptions in
>>>>>>>>>>>>>> order
>>>>>>>>>>>>>>> to maintain the consistency guarantees.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> IsolationLevels provide a high-level contract on the
>> behaviour
>>>>>>>> of
>>>>>>>>>> the
>>>>>>>>>>>>>>> StateStore: a user knows that under READ_COMMITTED, they will
>>>>>>>> see
>>>>>>>>>>>> writes
>>>>>>>>>>>>>>> only after the Task has committed, and under READ_UNCOMMITTED
>>>>>>>> they
>>>>>>>>>>>> will
>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>> writes immediately. No understanding of the details of each
>>>>>>>>>>>>>> processing.mode
>>>>>>>>>>>>>>> is required, either for IQ users or StateStore implementers.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> An argument can be made that these contractual guarantees can
>>>>>>>> simply
>>>>>>>>>>>> be
>>>>>>>>>>>>>>> documented for the processing.mode (i.e. that exactly-once
>> and
>>>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and at-least-once
>>>>>>>> behaves
>>>>>>>>>>>> like
>>>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small issues with
>>>>>>>> this I'd
>>>>>>>>>>>>>> prefer
>>>>>>>>>>>>>>> to avoid:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         - Where would we document these contracts, in a way
>> that
>>>>>>>> is
>>>>>>>>>>>> difficult
>>>>>>>>>>>>>>>         for users/implementers to miss/ignore?
>>>>>>>>>>>>>>>         - It's not clear to users that the processing mode is
>>>>>>>>>>>> communicating
>>>>>>>>>>>>>>>         an expectation of read isolation, unless they read the
>>>>>>>>>>>>>> documentation. Users
>>>>>>>>>>>>>>>         rarely consult documentation unless they feel they
>> need
>>>>>>>> to, so
>>>>>>>>>>>> it's
>>>>>>>>>>>>>> likely
>>>>>>>>>>>>>>>         this detail would get missed by many users.
>>>>>>>>>>>>>>>         - It tightly couples processing modes to read
>> isolation.
>>>>>>>> Adding
>>>>>>>>>>>> new
>>>>>>>>>>>>>>>         processing modes, or changing the read isolation of
>>>>>>>> existing
>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>         modes would be difficult/impossible.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ultimately, the cost of introducing IsolationLevels is just a
>>>>>>>> single
>>>>>>>>>>>>>>> method, since we re-use the existing IsolationLevel enum from
>>>>>>>> Kafka.
>>>>>>>>>>>> This
>>>>>>>>>>>>>>> gives us a clear place to document the contractual guarantees
>>>>>>>>>> expected
>>>>>>>>>>>>>>> of/provided by StateStores, that is accessible both by the
>>>>>>>>>> StateStore
>>>>>>>>>>>>>>> itself, and by IQ users.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> (Writing this I've just realised that the StateStore and IQ
>>>>>>>> APIs
>>>>>>>>>>>> actually
>>>>>>>>>>>>>>> don't provide access to StateStoreContext that IQ users would
>>>>>>>> have
>>>>>>>>>>>> direct
>>>>>>>>>>>>>>> access to... Perhaps StateStore should expose
>> isolationLevel()
>>>>>>>>>> itself
>>>>>>>>>>>>>> too?)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 4.
>>>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics in-place
>>>>>>>> either, as
>>>>>>>>>>>> it's a
>>>>>>>>>>>>>>> backwards incompatible change. My concern is that, if we
>> leave
>>>>>>>> the
>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>> "flush" metrics in place, they will be confusing to users.
>>>>>>>> Right
>>>>>>>>>> now,
>>>>>>>>>>>>>>> "flush" metrics record explicit flushes to disk, but under
>>>>>>>> KIP-892,
>>>>>>>>>>>> even
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>> commit() will not explicitly flush data to disk - RocksDB
>> will
>>>>>>>>>> decide
>>>>>>>>>>>> on
>>>>>>>>>>>>>>> when to flush memtables to disk itself.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd have two
>>>> options,
>>>>>>>>>> which
>>>>>>>>>>>>>> both
>>>>>>>>>>>>>>> seem pretty bad to me:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         1. Have them record calls to commit(), which would be
>>>>>>>>>>>> misleading, as
>>>>>>>>>>>>>>>         data is no longer explicitly "flushed" to disk by this
>>>>>>>> call.
>>>>>>>>>>>>>>>         2. Have them record nothing at all, which is
>> equivalent
>>>> to
>>>>>>>>>>>> removing
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>         metrics, except that users will see the metric still
>>>>>>>> exists and
>>>>>>>>>>>> so
>>>>>>>>>>>>>> assume
>>>>>>>>>>>>>>>         that the metric is correct, and that there's a problem
>>>>>>>> with
>>>>>>>>>> their
>>>>>>>>>>>>>> system
>>>>>>>>>>>>>>>         when there isn't.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I agree that removing them is also a bad solution, and I'd
>>>>>>>> like some
>>>>>>>>>>>>>>> guidance on the best path forward here.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 5.
>>>>>>>>>>>>>>> Position files are updated on every write to a StateStore.
>>>>>>>> Since our
>>>>>>>>>>>>>> writes
>>>>>>>>>>>>>>> are now buffered until commit(), we can't update the Position
>>>>>>>> file
>>>>>>>>>>>> until
>>>>>>>>>>>>>>> commit() has been called, otherwise it would be inconsistent
>>>>>>>> with
>>>>>>>>>> the
>>>>>>>>>>>>>> data
>>>>>>>>>>>>>>> in the event of a rollback. Consequently, we need to manage
>>>>>>>> these
>>>>>>>>>>>> offsets
>>>>>>>>>>>>>>> the same way we manage the checkpoint offsets, and ensure
>>>>>>>> they're
>>>>>>>>>> only
>>>>>>>>>>>>>>> written on commit().
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 6.
>>>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what tests to
>> write.
>>>>>>>> How
>>>>>>>>>>>>>> explicit
>>>>>>>>>>>>>>> do we need to be here in the KIP?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be seamless,
>>>>>>>> and we
>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>> definitely add some tests around that. Downgrade, it
>>>>>>>> transpires,
>>>>>>>>>> isn't
>>>>>>>>>>>>>>> currently possible, as the extra column family for offset
>>>>>>>> storage is
>>>>>>>>>>>>>>> incompatible with the pre-KIP-892 implementation: when you
>>>>>>>> open a
>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>> database, you must open all available column families or
>>>>>>>> receive an
>>>>>>>>>>>>>> error.
>>>>>>>>>>>>>>> What currently happens on downgrade is that it attempts to
>>>>>>>> open the
>>>>>>>>>>>>>> store,
>>>>>>>>>>>>>>> throws an error about the offsets column family not being
>>>>>>>> opened,
>>>>>>>>>>>> which
>>>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given that
>> downgrades
>>>>>>>>>> should
>>>>>>>>>>>> be
>>>>>>>>>>>>>>> uncommon, I think this is acceptable behaviour, as the
>>>>>>>> end-state is
>>>>>>>>>>>>>>> consistent, even if it results in an undesirable state
>>>> restore.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Should I document the upgrade/downgrade behaviour explicitly
>>>>>>>> in the
>>>>>>>>>>>> KIP?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> --
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
>>>>>>>> cadonna@apache.org>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi Nick!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for the updates!
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>> Why does StateStore#flush() default to
>>>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
>>>>>>>>>>>>>>>> Since calls to flush() will not exist anymore after this KIP
>>>>>>>> is
>>>>>>>>>>>>>>>> released, I would rather throw an unsupported operation
>>>>>>>> exception
>>>>>>>>>> by
>>>>>>>>>>>>>>>> default.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>> When would a state store return -1 from
>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while being
>>>>>>>>>>>> transactional?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Wouldn't StateStore#approximateNumUncommittedBytes() also
>>>>>>>> return 0
>>>>>>>>>> if
>>>>>>>>>>>>>>>> the state store is transactional but nothing has been
>> written
>>>>>>>> to
>>>>>>>>>> the
>>>>>>>>>>>>>>>> state store yet?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>> Sorry for bringing this up again. Does this KIP really need
>>>> to
>>>>>>>>>>>> introduce
>>>>>>>>>>>>>>>> StateStoreContext#isolationLevel()? StateStoreContext has
>>>>>>>> already
>>>>>>>>>>>>>>>> appConfigs() which basically exposes the same information,
>>>>>>>> i.e., if
>>>>>>>>>>>> EOS
>>>>>>>>>>>>>>>> is enabled or not.
>>>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> "My idea was to try to keep the StateStore interface as
>>>>>>>> loosely
>>>>>>>>>>>> coupled
>>>>>>>>>>>>>>>> from the Streams engine as possible, to give implementers
>>>> more
>>>>>>>>>>>> freedom,
>>>>>>>>>>>>>>>> and reduce the amount of internal knowledge required."
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> While I understand the intent, I doubt that it decreases the
>>>>>>>>>>>> coupling of
>>>>>>>>>>>>>>>> a StateStore interface and the Streams engine.
>> READ_COMMITTED
>>>>>>>> only
>>>>>>>>>>>>>>>> applies to IQ but not to reads by processors. Thus,
>>>>>>>> implementers
>>>>>>>>>>>> need to
>>>>>>>>>>>>>>>> understand how Streams accesses the state stores.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I would like to hear what others think about this.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 4.
>>>>>>>>>>>>>>>> Great exposing new metrics for transactional state stores!
>>>>>>>>>> However, I
>>>>>>>>>>>>>>>> would prefer to add new metrics and deprecate (in the docs)
>>>>>>>> the 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 also need to adapt/add integration tests
>>>>>>>> besides
>>>>>>>>>> unit
>>>>>>>>>>>>>>>> tests. Additionally, we probably need integration or system
>>>>>>>> tests
>>>>>>>>>> to
>>>>>>>>>>>>>>>> verify that upgrades and downgrades between transactional
>> and
>>>>>>>>>>>>>>>> non-transactional state stores work as expected.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
>>>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in the KIP, under
>>>>>>>>>>>> "Rejected
>>>>>>>>>>>>>>>>> Alternatives". I still think it's an idea worth pursuing,
>>>>>>>> but I
>>>>>>>>>>>> believe
>>>>>>>>>>>>>>>>> that it's out of the scope of this KIP, because it solves a
>>>>>>>>>>>> different
>>>>>>>>>>>>>> set
>>>>>>>>>>>>>>>>> of problems to this KIP, and the scope of this one has
>>>>>>>> already
>>>>>>>>>> grown
>>>>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>> large!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I've updated the KIP (
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>> )
>>>>>>>>>>>>>>>>>> with the latest changes; mostly bringing back "Atomic
>>>>>>>>>>>> Checkpointing"
>>>>>>>>>>>>>>>> (for
>>>>>>>>>>>>>>>>>> what feels like the 10th time!). I think the one thing
>>>>>>>> missing is
>>>>>>>>>>>> some
>>>>>>>>>>>>>>>>>> changes to metrics (notably the store "flush" metrics will
>>>>>>>> need
>>>>>>>>>> to
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> renamed to "commit").
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The reason I brought back Atomic Checkpointing was to
>>>>>>>> decouple
>>>>>>>>>>>> store
>>>>>>>>>>>>>>>> flush
>>>>>>>>>>>>>>>>>> from store commit. This is important, because with
>>>>>>>> Transactional
>>>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on *every* Task
>>>>>>>> commit,
>>>>>>>>>>>> and
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> just when the StateStore is closing, otherwise our
>>>>>>>> transaction
>>>>>>>>>>>> buffer
>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>> never be written and persisted, instead growing unbounded!
>>>> I
>>>>>>>>>>>>>>>> experimented
>>>>>>>>>>>>>>>>>> with some simple solutions, like forcing a store flush
>>>>>>>> whenever
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its configured
>>>>>>>> size, but
>>>>>>>>>>>> this
>>>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer from being
>>>>>>>>>> configured
>>>>>>>>>>>> to
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> unbounded, and it still would have required explicit
>>>>>>>> flushes of
>>>>>>>>>>>>>> RocksDB,
>>>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory utilization.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right" way to
>>>>>>>> resolve
>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> problem. By ensuring that the changelog offsets that
>>>>>>>> correspond
>>>>>>>>>> to
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> most
>>>>>>>>>>>>>>>>>> recently written records are always atomically written to
>>>>>>>> the
>>>>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>>>>> (by writing them to the same transaction buffer), we can
>>>>>>>> avoid
>>>>>>>>>>>>>> forcibly
>>>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting RocksDB
>>>>>>>> flush
>>>>>>>>>> them
>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>> when necessary, without losing any of our consistency
>>>>>>>> guarantees.
>>>>>>>>>>>> See
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> updated KIP for more info.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I have fully implemented these changes, although I'm still
>>>>>>>> not
>>>>>>>>>>>>>> entirely
>>>>>>>>>>>>>>>>>> happy with the implementation for segmented StateStores,
>> so
>>>>>>>> I
>>>>>>>>>> plan
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> refactor that. Despite that, all tests pass. If you'd like
>>>>>>>> to try
>>>>>>>>>>>> out
>>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>> review this highly experimental and incomplete branch,
>> it's
>>>>>>>>>>>> available
>>>>>>>>>>>>>>>> here:
>>>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0.
>>>>>>>> Note:
>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>> built
>>>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base to build
>>>>>>>> and test
>>>>>>>>>>>> it
>>>>>>>>>>>>>> on,
>>>>>>>>>>>>>>>>>> and to enable easy apples-to-apples comparisons in a live
>>>>>>>>>>>>>> environment. I
>>>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's nearer
>> completion
>>>>>>>> and
>>>>>>>>>> has
>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>> proven on our main application.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I would really appreciate help in reviewing and testing:
>>>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window) stores
>>>>>>>>>>>>>>>>>> - Global stores
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> As I do not currently use either of these, so my primary
>>>>>>>> test
>>>>>>>>>>>>>>>> environment
>>>>>>>>>>>>>>>>>> doesn't test these areas.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next week for a few
>>>>>>>> weeks,
>>>>>>>>>> so
>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>> not have time to move this forward until late August. That
>>>>>>>> said,
>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just won't be able
>>>> to
>>>>>>>>>>>> respond
>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>> quickly as usual.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi Bruno
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ is not
>>>>>>>> something
>>>>>>>>>> I
>>>>>>>>>>>> had
>>>>>>>>>>>>>>>>>>> considered.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> What about atomically updating the state store from the
>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
>>>>>>>> (thus,
>>>>>>>>>>>>>>>> flushing
>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
>>>>>>>> number of
>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>> intervals?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting that we
>>>>>>>> add an
>>>>>>>>>>>>>>>> additional
>>>>>>>>>>>>>>>>>>> config for the max number of commit intervals between
>>>>>>>>>> checkpoints?
>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the transaction
>>>>>>>> buffers
>>>>>>>>>> are
>>>>>>>>>>>>>>>> nearly
>>>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of commit intervals
>>>>>>>> have
>>>>>>>>>>>>>> elapsed,
>>>>>>>>>>>>>>>>>>> whichever comes first?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> That certainly seems reasonable, although this re-ignites
>>>>>>>> an
>>>>>>>>>>>> earlier
>>>>>>>>>>>>>>>>>>> debate about whether a config should be measured in
>>>>>>>> "number of
>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>> intervals", instead of just an absolute time.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the reason I was
>>>>>>>> pursuing
>>>>>>>>>> the
>>>>>>>>>>>>>>>> Atomic
>>>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush from
>>>>>>>> checkpointing,
>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>> enables us to just checkpoint on every commit without any
>>>>>>>>>>>> performance
>>>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the "best"
>>>>>>>> solution,
>>>>>>>>>>>> but
>>>>>>>>>>>>>>>> I'm not
>>>>>>>>>>>>>>>>>>> sure if this is enough to bring it back into this KIP.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I'm currently working on moving all the transactional
>>>> logic
>>>>>>>>>>>> directly
>>>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with the
>>>>>>>>>>>>>> StateStore#newTransaction
>>>>>>>>>>>>>>>>>>> method, and reduces the number of new classes introduced,
>>>>>>>>>>>>>> significantly
>>>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and the complexity
>>>> is
>>>>>>>>>>>>>> drastically
>>>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic Checkpoints into
>>>>>>>> this
>>>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
>>>>>>>> cadonna@apache.org>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> As far as I understand, you want to atomically update
>> the
>>>>>>>> state
>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the memtable of a
>>>> state
>>>>>>>>>> store
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit time elapsed
>>>> but
>>>>>>>>>> after
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> transaction buffer reached a size that would lead to
>>>>>>>> exceeding
>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes before the next
>>>>>>>> commit
>>>>>>>>>>>>>>>> interval
>>>>>>>>>>>>>>>>>>>> ends.
>>>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would commit every
>>>>>>>> commit
>>>>>>>>>>>> interval
>>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>>>> the state store will only be atomically updated roughly
>>>>>>>> every
>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ
>>>>>>>> would
>>>>>>>>>>>> then
>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>> see new data roughly every
>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
>>>>>>>>>>>>>>>>>>>> After a failure the state store needs to restore up to
>>>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Is this correct?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> What about atomically updating the state store from the
>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
>>>>>>>> (thus,
>>>>>>>>>>>>>>>> flushing
>>>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
>>>>>>>> number of
>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have the same delay
>>>> for
>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because both would
>>>>>>>> appear
>>>>>>>>>> when
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However, after a failure
>>>>>>>> the
>>>>>>>>>>>> state
>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>> still needs to restore up to
>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> it might restore data that is already in the state store
>>>>>>>>>> because
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable offset (i.e. the
>>>>>>>> last
>>>>>>>>>>>>>> committed
>>>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring data that is
>>>>>>>> already
>>>>>>>>>>>> in
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should not violated.
>>>>>>>>>>>>>>>>>>>> This solution needs at least one new config to specify
>>>>>>>> when a
>>>>>>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>>>>>>> should be written.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail that does not
>>>>>>>> change
>>>>>>>>>>>>>>>> anything
>>>>>>>>>>>>>>>>>>>> you said: Under alos the default commit interval is 30
>>>>>>>> seconds,
>>>>>>>>>>>> not
>>>>>>>>>>>>>>>> five
>>>>>>>>>>>>>>>>>>>> seconds.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch on our staging
>>>>>>>>>>>>>> environment,
>>>>>>>>>>>>>>>>>>>>> putting it through its paces in our non-trivial
>>>>>>>> application.
>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>> observing the same increased flush rate that we saw the
>>>>>>>> last
>>>>>>>>>>>> time
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP, but this time,
>> I
>>>>>>>>>> think I
>>>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>>>> why.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at
>>>>>>>> the end
>>>>>>>>>>>> of
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> Task
>>>>>>>>>>>>>>>>>>>>> commit process, has the following behaviour:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>           - Under ALOS: checkpoint the state stores.
>> This
>>>>>>>>>> includes
>>>>>>>>>>>>>>>>>>>>>           flushing memtables in RocksDB. This is
>>>> acceptable
>>>>>>>>>>>> because the
>>>>>>>>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>>>>>           commit.interval.ms is 5 seconds, so forcibly
>>>>>>>> flushing
>>>>>>>>>>>>>> memtables
>>>>>>>>>>>>>>>>>>>> every 5
>>>>>>>>>>>>>>>>>>>>>           seconds is acceptable for most applications.
>>>>>>>>>>>>>>>>>>>>>           - Under EOS: checkpointing is not done,
>> *unless*
>>>>>>>> it's
>>>>>>>>>>>> being
>>>>>>>>>>>>>>>>>>>> forced, due
>>>>>>>>>>>>>>>>>>>>>           to e.g. the Task closing or being revoked.
>> This
>>>>>>>> means
>>>>>>>>>>>> that
>>>>>>>>>>>>>> under
>>>>>>>>>>>>>>>>>>>> normal
>>>>>>>>>>>>>>>>>>>>>           processing conditions, the state stores will
>> not
>>>>>>>> be
>>>>>>>>>>>>>>>> checkpointed,
>>>>>>>>>>>>>>>>>>>> and will
>>>>>>>>>>>>>>>>>>>>>           not have memtables flushed at all , unless
>>>> RocksDB
>>>>>>>>>>>> decides to
>>>>>>>>>>>>>>>>>>>> flush them on
>>>>>>>>>>>>>>>>>>>>>           its own. Checkpointing stores and
>> force-flushing
>>>>>>>> their
>>>>>>>>>>>>>> memtables
>>>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>>>           done when a Task is being closed.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at
>>>> least
>>>>>>>>>> *some*
>>>>>>>>>>>>>>>> normal
>>>>>>>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB transaction
>>>>>>>>>> buffers
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets are synced to
>>>>>>>> disk to
>>>>>>>>>>>>>> prevent
>>>>>>>>>>>>>>>>>>>>> restores from getting out of hand. Consequently, my
>>>>>>>> current
>>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is
>>>>>>>> far too
>>>>>>>>>>>>>>>>>>>> frequent.
>>>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records, which is
>> a
>>>>>>>>>> change
>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> flush
>>>>>>>>>>>>>>>>>>>>> behaviour, potentially causing performance problems for
>>>>>>>> some
>>>>>>>>>>>>>>>>>>>> applications.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm currently
>>>>>>>> leaning
>>>>>>>>>>>>>>>> towards
>>>>>>>>>>>>>>>>>>>>> using the statestore.transaction.buffer.max.bytes
>>>>>>>>>> configuration
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to exceed it. This
>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> complement the
>>>>>>>>>>>>>>>>>>>>> existing "early Task commit" functionality that this
>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>>>> provides, in the following way:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>           - Currently, we use
>>>>>>>>>>>> statestore.transaction.buffer.max.bytes
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> force an
>>>>>>>>>>>>>>>>>>>>>           early Task commit if processing more records
>>>> would
>>>>>>>>>> cause
>>>>>>>>>>>> our
>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>           transactions to exceed the memory assigned to
>>>>>>>> them.
>>>>>>>>>>>>>>>>>>>>>           - New functionality: when a Task *does*
>> commit,
>>>>>>>> we will
>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>>>>>>>>           the stores (and hence flush the transaction
>>>>>>>> buffers)
>>>>>>>>>>>> unless
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> expect to
>>>>>>>>>>>>>>>>>>>>>           cross the
>>>> statestore.transaction.buffer.max.bytes
>>>>>>>>>>>> threshold
>>>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>>>> the next
>>>>>>>>>>>>>>>>>>>>>           commit
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
>>>>>>>>>>>> nick.telford@gmail.com
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi Bruno!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I meant in terms of
>>>>>>>>>>>> understanding
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> performance profile under various circumstances. The
>>>>>>>> more
>>>>>>>>>>>> complex
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would be for users to
>>>>>>>>>>>> understand
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> performance they see. For example, spilling records to
>>>>>>>> disk
>>>>>>>>>>>> when
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold would, I
>> expect,
>>>>>>>>>> reduce
>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write throughput could
>> be
>>>>>>>>>>>>>> unexpected,
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> potentially difficult to diagnose/understand for
>> users.
>>>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early commit" concept is
>>>>>>>>>> relatively
>>>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document, and
>>>> conceptually
>>>>>>>>>> fairly
>>>>>>>>>>>>>>>>>>>> obvious to
>>>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to make it
>> easier
>>>>>>>> to
>>>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>> it happens though.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3. (the second one)
>>>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an indirect way of
>>>>>>>>>> telling
>>>>>>>>>>>>>>>>>>>> StateStores
>>>>>>>>>>>>>>>>>>>>>> whether they should be transactional. READ_COMMITTED
>>>>>>>>>>>> essentially
>>>>>>>>>>>>>>>>>>>> requires
>>>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that two threads
>>>>>>>> calling
>>>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes from the
>> other
>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all
>>>>>>>> bets are
>>>>>>>>>>>> off,
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe written records at
>>>>>>>> any
>>>>>>>>>>>> time,
>>>>>>>>>>>>>>>>>>>> which is
>>>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That said, StateStores
>>>>>>>> are
>>>>>>>>>> free
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> implement
>>>>>>>>>>>>>>>>>>>>>> these guarantees however they can, which is a bit more
>>>>>>>>>> relaxed
>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions". For example,
>>>> with
>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
>>>>>>>>>> "transactions",
>>>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But
>>>>>>>> with
>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>> storage
>>>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to *always* use
>>>>>>>> transactions,
>>>>>>>>>>>> even
>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines that
>> don't
>>>>>>>>>> provide
>>>>>>>>>>>>>>>>>>>>>> transactions, but the isolation guarantees can be met
>>>>>>>> using a
>>>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>>>> technique.
>>>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore interface as
>>>>>>>>>> loosely
>>>>>>>>>>>>>>>> coupled
>>>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
>>>>>>>> implementers
>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>> freedom, and
>>>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge required.
>>>>>>>>>>>>>>>>>>>>>> That said, I understand that "IsolationLevel" might
>> not
>>>>>>>> be
>>>>>>>>>> the
>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it much more
>>>>>>>> explicit if
>>>>>>>>>>>>>>>>>>>> required, e.g.
>>>>>>>>>>>>>>>>>>>>>> boolean transactional()
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 7-8.
>>>>>>>>>>>>>>>>>>>>>> I can make these changes either later today or
>>>> tomorrow.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Small update:
>>>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of
>>>>>>>> issues
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> needed
>>>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass, which is
>>>>>>>>>> promising,
>>>>>>>>>>>> but
>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>> need to undergo some performance testing. I haven't
>>>>>>>> (yet)
>>>>>>>>>>>> worked
>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I would
>>>>>>>> expect
>>>>>>>>>> that,
>>>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no difference. The
>> branch
>>>>>>>> is
>>>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
>> if
>>>>>>>>>> anyone
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
>>>>>>>>>>>> cadonna@apache.org>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually also my
>>>>>>>> point. I
>>>>>>>>>>>>>>>> understood
>>>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion path as a way
>> to
>>>>>>>> avoid
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> early
>>>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually it is public
>>>>>>>> API. My
>>>>>>>>>>>>>> question
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal processing.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the size of the
>>>>>>>> transaction
>>>>>>>>>>>>>> buffer
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit affects the whole
>>>>>>>>>> processing
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a state store is not
>>>>>>>>>>>> confined to
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and changes the
>>>>>>>> behavior
>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you that it is a
>>>>>>>> decent
>>>>>>>>>>>>>>>>>>>> compromise. I
>>>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and list the
>>>>>>>> options to
>>>>>>>>>>>>>>>> overcome
>>>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all options seem
>>>> quite
>>>>>>>>>> heavy
>>>>>>>>>>>>>>>>>>>> compared
>>>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you mean with
>>>>>>>> "less
>>>>>>>>>>>>>>>>>>>> predictable
>>>>>>>>>>>>>>>>>>>>>>> for users", though.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I found the discussions about the alternatives really
>>>>>>>>>>>>>> interesting.
>>>>>>>>>>>>>>>>>>>> But I
>>>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good and we should
>>>>>>>> continue
>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> it!
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on June
>> 20th:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind putting
>>>>>>>> isolation
>>>>>>>>>>>>>> level
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should that also be
>> a
>>>>>>>> way
>>>>>>>>>> to
>>>>>>>>>>>>>> give
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to decide whether to
>>>>>>>> turn on
>>>>>>>>>>>>>>>>>>>>>>> transactions or not?
>>>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned about how do
>> you
>>>>>>>> know
>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written under EOS, if you
>>>>>>>> do not
>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>>>> to know if the state store is transactional or not.
>> If
>>>>>>>> a
>>>>>>>>>> state
>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be written
>>>>>>>> during
>>>>>>>>>>>> normal
>>>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state store is not
>>>>>>>>>> transactional,
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written under EOS.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 7.
>>>>>>>>>>>>>>>>>>>>>>> My point was about not only considering the bytes in
>>>>>>>> memory
>>>>>>>>>> in
>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that
>>>>>>>> might
>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> spilled
>>>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering whether you
>> should
>>>>>>>>>> remove
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory bytes to be
>> used
>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My thinking
>>>>>>>> was
>>>>>>>>>> that
>>>>>>>>>>>>>> even
>>>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to disk,
>> limiting
>>>>>>>> the
>>>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>>>> bytes
>>>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it again and
>>>>>>>> considering
>>>>>>>>>> the
>>>>>>>>>>>>>>>> recent
>>>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much sense anymore.
>>>>>>>>>>>>>>>>>>>>>>> I like the name
>>>>>>>> statestore.transaction.buffer.max.bytes that
>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>> proposed.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 8.
>>>>>>>>>>>>>>>>>>>>>>> A high-level description (without implementation
>>>>>>>> details) of
>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
>>>>>>>> transactions,
>>>>>>>>>>>> state
>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be great. Would
>>>> be
>>>>>>>>>> great
>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the behavior in
>>>>>>>> case of
>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional state store
>>>>>>>> recover
>>>>>>>>>>>> after a
>>>>>>>>>>>>>>>>>>>>>>> failure or what happens with the transaction buffer,
>>>>>>>> etc.
>>>>>>>>>>>> (that
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
>>>>>>>> WriteBatchWithIndex
>>>>>>>>>>>>>>>>>>>> transactions
>>>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
>>>>>>>> configured
>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>>>> needs to
>>>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns. Ultimately, any
>>>>>>>> approach
>>>>>>>>>>>> based
>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>> cracking
>>>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in ways it's not
>>>>>>>> really
>>>>>>>>>>>>>>>> designed
>>>>>>>>>>>>>>>>>>>>>>> for is
>>>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen performance or
>>>>>>>> consistency
>>>>>>>>>>>> issues.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing these early
>>>>>>>> commits?
>>>>>>>>>>>> While
>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>> ideal, I
>>>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to ensure
>>>>>>>> consistency
>>>>>>>>>>>> whilst
>>>>>>>>>>>>>>>>>>>>>>> maintaining
>>>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
>>>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
>>>>>>>> complicated, and
>>>>>>>>>>>> might
>>>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for users as a
>>>>>>>> consequence.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is
>>>>>>>> growing a
>>>>>>>>>>>> bit
>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas for future
>>>>>>>>>>>>>>>> improvements, I
>>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope down to a design
>>>>>>>> that
>>>>>>>>>>>>>> achieves
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> most
>>>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized restorations
>>>>>>>> during
>>>>>>>>>> dirty
>>>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that this KIP
>>>>>>>> produces
>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>> ultimately
>>>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially if the bulk of
>>>>>>>> it is
>>>>>>>>>>>>>> internal
>>>>>>>>>>>>>>>>>>>>>>>> behaviour.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week trying to
>>>>>>>> re-work
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
>>>>>>>> newTransaction()
>>>>>>>>>>>>>> method,
>>>>>>>>>>>>>>>>>>>> such
>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of RocksDBStore.
>>>>>>>> That
>>>>>>>>>>>> way, if
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>>> want to
>>>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the future, like the
>>>>>>>> SST
>>>>>>>>>> file
>>>>>>>>>>>>>>>>>>>> management
>>>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with little/no
>>>> API
>>>>>>>>>>>> changes.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Bruno,

Thanks for getting back to me.

1.
I think this should be possible. Are you thinking of the situation where a
user may downgrade to a previous version of Kafka Streams? In that case,
sadly, the RocksDBStore would get wiped by the older version of Kafka
Streams anyway, because that version wouldn't understand the extra column
family (that holds offsets), so the missing Position file would
automatically get rebuilt when the store is rebuilt from the changelog.
Are there other situations than downgrade where a transactional store could
be replaced by a non-transactional one? I can't think of any.

2.
Ahh yes, the Test Plan - my Kryptonite! This section definitely needs to be
fleshed out. I'll work on that. How much detail do you need?

3.
See my previous email discussing this.

4.
Hmm, this is an interesting point. Are you suggesting that under ALOS
READ_COMMITTED should not be supported?

Regards,
Nick

On Fri, 13 Oct 2023 at 13:52, Bruno Cadonna <ca...@apache.org> wrote:

> Hi Nick,
>
> I think the KIP is converging!
>
>
> 1.
> I am wondering whether it makes sense to write the position file during
> close as we do for the checkpoint file, so that in case the state store
> is replaced with a non-transactional state store the non-transactional
> state store finds the position file. I think, this is not strictly
> needed, but would be a nice behavior instead of just deleting the
> position file.
>
>
> 2.
> The test plan does not mention integration tests. Do you not need to
> extend existing ones and add new ones. Also for upgrading and
> downgrading you might need integration and/or system tests.
>
>
> 3.
> I think Sophie made a point. Although, IQ reading from uncommitted data
> under EOS might be considered a bug by some people. Thus, your KIP would
> fix a bug rather than changing the intended behavior. However, I also
> see that a feature flag would help users that rely on this buggy
> behavior (at least until AK 4.0).
>
>
> 4.
> This is related to the previous point. I assume that the difference
> between READ_COMMITTED and READ_UNCOMMITTED for ALOS is that in the
> former you enable transactions on the state store and in the latter you
> disable them. If my assumption is correct, I think that is an issue.
> Let's assume under ALOS Streams fails over a couple of times more or
> less at the same step in processing after value 3 is added to an
> aggregation but the offset of the corresponding input record was not
> committed. Without transactions disabled, the aggregation value would
> increase by 3 for each failover. With transactions enabled, value 3
> would only be added to the aggregation once when the offset of the input
> record is committed and the transaction finally completes. So the
> content of the state store would change depending on the configuration
> for IQ. IMO, the content of the state store should be independent from
> IQ. Given this issue, I propose to not use transactions with ALOS at
> all. I was a big proponent of using transactions with ALOS, but I
> realized that transactions with ALOS is not as easy as enabling
> transactions on state stores. Another aspect that is problematic is that
> the changelog topic which actually replicates the state store is not
> transactional under ALOS. Thus, it might happen that the state store and
> the changelog differ in their content. All of this is maybe solvable
> somehow, but for the sake of this KIP, I would leave it for the future.
>
>
> Best,
> Bruno
>
>
>
> On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> > Hey Nick! First of all thanks for taking up this awesome feature, I'm
> sure
> > every single
> > Kafka Streams user and dev would agree that it is sorely needed.
> >
> > I've just been catching up on the KIP and surrounding discussion, so
> please
> > forgive me
> > for any misunderstandings or misinterpretations of the current plan and
> > don't hesitate to
> > correct me.
> >
> > Before I jump in, I just want to say that having seen this drag on for so
> > long, my singular
> > goal in responding is to help this KIP past a perceived impasse so we can
> > finally move on
> > to voting and implementing it. Long discussions are to be expected for
> > major features like
> > this but it's completely on us as the Streams devs to make sure there is
> an
> > end in sight
> > for any ongoing discussion.
> >
> > With that said, it's my understanding that the KIP as currently proposed
> is
> > just not tenable
> > for Kafka Streams, and would prevent some EOS users from upgrading to the
> > version it
> > first appears in. Given that we can't predict or guarantee whether any of
> > the followup KIPs
> > would be completed in the same release cycle as this one, we need to make
> > sure that the
> > feature is either compatible with all current users or else
> feature-flagged
> > so that they may
> > opt in/out.
> >
> > Therefore, IIUC we need to have either (or both) of these as
> > fully-implemented config options:
> > 1. default.state.isolation.level
> > 2. enable.transactional.state.stores
> >
> > This way EOS users for whom read_committed semantics are not viable can
> > still upgrade,
> > and either use the isolation.level config to leverage the new txn state
> > stores without sacrificing
> > their application semantics, or else simply keep the transactional state
> > stores disabled until we
> > are able to fully implement the isolation level configuration at either
> an
> > application or query level.
> >
> > Frankly you are the expert here and know much more about the tradeoffs in
> > both semantics and
> > effort level of implementing one of these configs vs the other. In my
> > opinion, either option would
> > be fine and I would leave the decision of which one to include in this
> KIP
> > completely up to you.
> > I just don't see a way for the KIP to proceed without some variation of
> the
> > above that would allow
> > EOS users to opt-out of read_committed.
> >
> > (If it's all the same to you, I would recommend 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 that gets released.
> > So personally, my choice would be to add the feature flag and leave it
> off
> > by default. If all 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
> >
> > On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <ni...@gmail.com>
> wrote:
> >
> >> Hi Bruno,
> >>
> >> Agreed, I can live with that for now.
> >>
> >> In an effort to keep the scope of this KIP from expanding, I'm leaning
> >> towards just providing a configurable default.state.isolation.level and
> >> removing IsolationLevel from the StateStoreContext. This would be
> >> compatible with adding support for query-time IsolationLevels in the
> >> future, whilst providing a way for users to select an isolation level
> now.
> >>
> >> The big problem with this, however, is that if a user selects
> >> processing.mode
> >> = "exactly-once(-v2|-beta)", and default.state.isolation.level =
> >> "READ_UNCOMMITTED", we need to guarantee that the data isn't written to
> >> disk until commit() is called, but we also need to permit IQ threads to
> >> read from the ongoing transaction.
> >>
> >> A simple solution would be to (temporarily) forbid this combination of
> >> configuration, and have default.state.isolation.level automatically
> switch
> >> to READ_COMMITTED when processing.mode is anything other than
> >> at-least-once. Do you think this would be acceptable?
> >>
> >> In a later KIP, we can add support for query-time isolation levels and
> >> solve this particular problem there, which would relax this restriction.
> >>
> >> Regards,
> >> Nick
> >>
> >> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <ca...@apache.org>
> 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,
> >>> Bruno
> >>>
> >>> On 9/18/23 7:12 PM, Nick Telford wrote:
> >>>> 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 <ni...@gmail.com>
> >>> wrote:
> >>>>
> >>>>> Hi everyone,
> >>>>>
> >>>>> I agree that having IsolationLevel be determined at query-time is the
> >>>>> ideal design, but there are a few sticking points:
> >>>>>
> >>>>> 1.
> >>>>> There needs to be some way to communicate the IsolationLevel down to
> >> the
> >>>>> RocksDBStore itself, so that the query can respect it. Since stores
> >> are
> >>>>> "layered" in functionality (i.e. ChangeLoggingStore, MeteredStore,
> >>> etc.),
> >>>>> we need some way to deliver that information to the bottom layer. For
> >>> IQv2,
> >>>>> we can use the existing State#query() method, but IQv1 has no way to
> >> do
> >>>>> this.
> >>>>>
> >>>>> A simple approach, which would potentially open up other options,
> >> would
> >>> be
> >>>>> to add something like: ReadOnlyKeyValueStore<K, V>
> >>>>> readOnlyView(IsolationLevel isolationLevel) to ReadOnlyKeyValueStore
> >>> (and
> >>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
> >>>>>
> >>>>> 2.
> >>>>> As mentioned above, RocksDB WriteBatches are not thread-safe, which
> >>> causes
> >>>>> a problem if we want to provide READ_UNCOMMITTED Iterators. I also
> >> had a
> >>>>> look at RocksDB Transactions[1], but they solve a very different
> >>> problem,
> >>>>> and have the same thread-safety issue.
> >>>>>
> >>>>> One possible approach that I mentioned is chaining WriteBatches:
> every
> >>>>> time a new Interactive Query is received (i.e. readOnlyView, see
> >> above,
> >>>>> is called) we "freeze" the existing WriteBatch, and start a new one
> >> for
> >>> new
> >>>>> writes. The Interactive Query queries the "chain" of previous
> >>> WriteBatches
> >>>>> + the underlying database; while the StreamThread starts writing to
> >> the
> >>>>> *new* WriteBatch. On-commit, the StreamThread would write *all*
> >>>>> WriteBatches in the chain to the database (that have not yet been
> >>> written).
> >>>>>
> >>>>> WriteBatches would be closed/freed only when they have been both
> >>>>> committed, and all open Interactive Queries on them have been closed.
> >>> This
> >>>>> would require some reference counting.
> >>>>>
> >>>>> Obviously a drawback of this approach is the potential for increased
> >>>>> memory usage: if an Interactive Query is long-lived, for example by
> >>> doing a
> >>>>> full scan over a large database, or even just pausing in the middle
> of
> >>> an
> >>>>> iteration, then the existing chain of WriteBatches could be kept
> >> around
> >>> for
> >>>>> a long time, potentially forever.
> >>>>>
> >>>>> --
> >>>>>
> >>>>> A.
> >>>>> Going off on a tangent, it looks like in addition to supporting
> >>>>> READ_COMMITTED queries, we could go further and support
> >> REPEATABLE_READ
> >>>>> queries (i.e. where subsequent reads to the same key in the same
> >>>>> Interactive Query are guaranteed to yield the same value) by making
> >> use
> >>> of
> >>>>> RocksDB Snapshots[2]. These are fairly lightweight, so the
> performance
> >>>>> impact is likely to be negligible, but they do require that the
> >>> Interactive
> >>>>> Query session can be explicitly closed.
> >>>>>
> >>>>> This could be achieved if we made the above readOnlyView interface
> >> look
> >>>>> more like:
> >>>>>
> >>>>> interface ReadOnlyKeyValueView<K, V> implements
> >> ReadOnlyKeyValueStore<K,
> >>>>> V>, AutoCloseable {}
> >>>>>
> >>>>> interface ReadOnlyKeyValueStore<K, V> {
> >>>>>       ...
> >>>>>       ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel
> >>> isolationLevel);
> >>>>> }
> >>>>>
> >>>>> But this would be a breaking change, as existing IQv1 queries are
> >>>>> guaranteed to never call store.close(), and therefore these would
> leak
> >>>>> memory under REPEATABLE_READ.
> >>>>>
> >>>>> B.
> >>>>> One thing that's notable: MyRocks states that they support
> >>> READ_COMMITTED
> >>>>> and REPEATABLE_READ, but they make no mention of
> >> READ_UNCOMMITTED[3][4].
> >>>>> This could be because doing so is technically difficult/impossible
> >> using
> >>>>> the primitives available in RocksDB.
> >>>>>
> >>>>> --
> >>>>>
> >>>>> Lucas, to address your points:
> >>>>>
> >>>>> U1.
> >>>>> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
> >>>>> implementations of StateStore that do not support atomic writes.
> >>> Obviously
> >>>>> in those cases, the guarantees Kafka Streams provides/expects would
> be
> >>>>> relaxed. Do you think we should require all implementations to
> support
> >>>>> atomic writes?
> >>>>>
> >>>>> U2.
> >>>>> Stores can support multiple IsolationLevels. As we've discussed
> above,
> >>> the
> >>>>> ideal scenario would be to specify the IsolationLevel at query-time.
> >>>>> Failing that, I think the second-best approach is to define the
> >>>>> IsolationLevel for *all* queries based on the processing.mode, which
> >> is
> >>>>> what the default StateStoreContext#isolationLevel() achieves. Would
> >> you
> >>>>> prefer an alternative?
> >>>>>
> >>>>> While the existing implementation is equivalent to READ_UNCOMMITTED,
> >>> this
> >>>>> can yield unexpected results/errors under EOS, if a transaction is
> >>> rolled
> >>>>> back. While this would be a change in behaviour for users, it would
> >> look
> >>>>> more like a bug fix than a breaking change. That said, we *could*
> make
> >>> it
> >>>>> configurable, and default to the existing behaviour
> (READ_UNCOMMITTED)
> >>>>> instead of inferring it from the processing.mode?
> >>>>>
> >>>>> N1, N2.
> >>>>> These were only primitives to avoid boxing costs, but since this is
> >> not
> >>> a
> >>>>> performance sensitive area, it should be fine to change if that's
> >>> desirable.
> >>>>>
> >>>>> N3.
> >>>>> It's because the store "manages its own offsets", which includes both
> >>>>> committing the offset, *and providing it* via getCommittedOffset().
> >>>>> Personally, I think "managesOffsets" conveys this best, but I don't
> >> mind
> >>>>> changing it if the nomenclature is unclear.
> >>>>>
> >>>>> Sorry for the massive emails/essays!
> >>>>> --
> >>>>> Nick
> >>>>>
> >>>>> 1: https://github.com/facebook/rocksdb/wiki/Transactions
> >>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> >>>>> 3: https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> >>>>> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
> >>>>>
> >>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> >>>>> <lb...@confluent.io.invalid> wrote:
> >>>>>
> >>>>>> Hi Nick,
> >>>>>>
> >>>>>> since I last read it in April, the KIP has become much cleaner and
> >>>>>> easier to read. Great work!
> >>>>>>
> >>>>>> It feels to me the last big open point is whether we can implement
> >>>>>> isolation level as a query parameter. I understand that there are
> >>>>>> implementation concerns, but as Colt says, it would be a great
> >>>>>> addition, and would also simplify the migration path for this
> change.
> >>>>>> Is the implementation problem you mentioned caused by the WriteBatch
> >>>>>> not having a notion of a snapshot, as the underlying DB iterator
> >> does?
> >>>>>> In that case, I am not sure a chain of WriteBatches as you propose
> >>>>>> would fully solve the problem, but maybe I didn't dig enough into
> the
> >>>>>> details to fully understand it.
> >>>>>>
> >>>>>> If it's not possible to implement it now, would it be an option to
> >>>>>> make sure in this KIP that we do not fully close the door on
> >> per-query
> >>>>>> isolation levels in the interface, as it may be possible to
> implement
> >>>>>> the missing primitives in RocksDB or Speedb in the future.
> >>>>>>
> >>>>>> Understanding:
> >>>>>>
> >>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to be persisted
> >>>>>> atomically with the records?
> >>>>>> * U2) Don't understand the default implementation of
> >> `isolationLevel`.
> >>>>>> The isolation level should be a property of the underlying store,
> and
> >>>>>> not be defined by the default config? Existing stores probably don't
> >>>>>> guarantee READ_COMMITTED, so the default should be to return
> >>>>>> READ_UNCOMMITTED.
> >>>>>>
> >>>>>> Nits:
> >>>>>> * N1) Could `getComittedOffset` use an `OptionalLong` return type,
> to
> >>>>>> avoid the `null`?
> >>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
> >>>>>> return type, to avoid the `-1`?
> >>>>>> * N3) I don't understand why `managesOffsets` uses the 'manage'
> verb,
> >>>>>> whereas all other methods use the "commits" verb. I'd suggest
> >>>>>> `commitsOffsets`.
> >>>>>>
> >>>>>> Either way, it feels this KIP is very close to the finish line, I'm
> >>>>>> looking forward to seeing this in production!
> >>>>>>
> >>>>>> Cheers,
> >>>>>> Lucas
> >>>>>>
> >>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <co...@littlehorse.io>
> >>> wrote:
> >>>>>>>
> >>>>>>>> Making IsolationLevel a query-time constraint, rather than linking
> >> it
> >>>>>> to
> >>>>>>> the processing.guarantee.
> >>>>>>>
> >>>>>>> As I understand it, would this allow even a user of EOS to control
> >>>>>> whether
> >>>>>>> reading committed or uncommitted records? If so, I am highly in
> >> favor
> >>> of
> >>>>>>> this.
> >>>>>>>
> >>>>>>> I know that I was one of the early people to point out the current
> >>>>>>> shortcoming that IQ reads uncommitted records, but just this
> >> morning I
> >>>>>>> realized a pattern we use which means that (for certain queries)
> our
> >>>>>> system
> >>>>>>> needs to be able to read uncommitted records, which is the current
> >>>>>> behavior
> >>>>>>> of Kafka Streams in EOS.***
> >>>>>>>
> >>>>>>> If IsolationLevel being a query-time decision allows for this, then
> >>> that
> >>>>>>> would be amazing. I would also vote that the default behavior
> should
> >>> be
> >>>>>> for
> >>>>>>> reading uncommitted records, because it is totally possible for a
> >>> valid
> >>>>>>> application to depend on that behavior, and breaking it in a minor
> >>>>>> release
> >>>>>>> might be a bit strong.
> >>>>>>>
> >>>>>>> *** (Note, for the curious reader....) Our use-case/query pattern
> >> is a
> >>>>>> bit
> >>>>>>> complex, but reading "uncommitted" records is actually safe in our
> >>> case
> >>>>>>> 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 soon as the processor processes the result. If
> >>> they
> >>>>>>> can't read uncommitted records, they may receive a "201 - Created"
> >>>>>>> response, immediately followed by a "404 - 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 <
> >> nick.telford@gmail.com>
> >>>>>> wrote:
> >>>>>>>
> >>>>>>>> Addendum:
> >>>>>>>>
> >>>>>>>> I think we would also face the same problem with the approach John
> >>>>>> outlined
> >>>>>>>> earlier (using the record cache as a transaction buffer and
> >> flushing
> >>>>>> it
> >>>>>>>> straight to SST files). This is because the record cache (the
> >>>>>> ThreadCache
> >>>>>>>> class) is not thread-safe, so every commit would invalidate open
> IQ
> >>>>>>>> Iterators in the same way that RocksDB WriteBatches do.
> >>>>>>>> --
> >>>>>>>> Nick
> >>>>>>>>
> >>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> >> nick.telford@gmail.com>
> >>>>>>>> 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.
> >>>>>>>>> 2. Making IsolationLevel a query-time constraint, rather than
> >>>>>> linking it
> >>>>>>>>> to the processing.guarantee.
> >>>>>>>>>
> >>>>>>>>> There's a wrinkle that makes this a challenge: Interactive
> Queries
> >>>>>> that
> >>>>>>>>> open an Iterator, when using transactions and READ_UNCOMMITTED.
> >>>>>>>>> The problem is that under READ_UNCOMMITTED, queries need to be
> >> able
> >>>>>> to
> >>>>>>>>> read records from the currently uncommitted transaction buffer
> >>>>>>>>> (WriteBatch). This includes for Iterators, which should iterate
> >>>>>> both the
> >>>>>>>>> transaction buffer and underlying database (using
> >>>>>>>>> WriteBatch#iteratorWithBase()).
> >>>>>>>>>
> >>>>>>>>> The issue is that when the StreamThread commits, it writes the
> >>>>>> current
> >>>>>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing
> >> the
> >>>>>>>>> WriteBatch while an Interactive Query holds an open Iterator on
> it
> >>>>>> will
> >>>>>>>>> invalidate the Iterator. Worse, it turns out that Iterators over
> a
> >>>>>>>>> WriteBatch become invalidated not just when the WriteBatch is
> >>>>>> cleared,
> >>>>>>>> but
> >>>>>>>>> also when the Iterators' current key receives a new write.
> >>>>>>>>>
> >>>>>>>>> Now that I'm writing this, I remember that this is the major
> >> reason
> >>>>>> that
> >>>>>>>> I
> >>>>>>>>> switched the original design from having a query-time
> >>>>>> IsolationLevel to
> >>>>>>>>> having the IsolationLevel linked to the transactionality of the
> >>>>>> stores
> >>>>>>>>> themselves.
> >>>>>>>>>
> >>>>>>>>> It *might* be possible to resolve this, by having a "chain" of
> >>>>>>>>> WriteBatches, with the StreamThread switching to a new WriteBatch
> >>>>>>>> whenever
> >>>>>>>>> a new Interactive Query attempts to read from the database, but
> >> that
> >>>>>>>> could
> >>>>>>>>> cause some performance problems/memory pressure when subjected to
> >> a
> >>>>>> high
> >>>>>>>>> Interactive Query load. It would also reduce the efficiency of
> >>>>>>>> WriteBatches
> >>>>>>>>> on-commit, as we'd have to write N WriteBatches, where N is the
> >>>>>> number of
> >>>>>>>>> Interactive Queries since the last commit.
> >>>>>>>>>
> >>>>>>>>> I realise this is getting into the weeds of the implementation,
> >> and
> >>>>>> you'd
> >>>>>>>>> rather we focus on the API for now, but I think it's important to
> >>>>>>>> consider
> >>>>>>>>> how to implement the desired API, in case we come up with an API
> >>>>>> that
> >>>>>>>>> cannot be implemented efficiently, or even at all!
> >>>>>>>>>
> >>>>>>>>> Thoughts?
> >>>>>>>>> --
> >>>>>>>>> Nick
> >>>>>>>>>
> >>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <cadonna@apache.org
> >
> >>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi Nick,
> >>>>>>>>>>
> >>>>>>>>>> 6.
> >>>>>>>>>> Of course, you are right! My bad!
> >>>>>>>>>> Wiping out the state in the downgrading case is fine.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> 3a.
> >>>>>>>>>> Focus on the public facing changes for the KIP. We will manage
> 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.
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> Best,
> >>>>>>>>>> Bruno
> >>>>>>>>>>
> >>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> >>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks for getting back to me!
> >>>>>>>>>>>
> >>>>>>>>>>> 2.
> >>>>>>>>>>> The fact that implementations can always track estimated memory
> >>>>>> usage
> >>>>>>>> in
> >>>>>>>>>>> the wrapper is a good point. I can remove -1 as an option, and
> >>>>>> I'll
> >>>>>>>>>> clarify
> >>>>>>>>>>> the JavaDoc that 0 is not just for non-transactional stores,
> >>>>>> which is
> >>>>>>>>>>> currently misleading.
> >>>>>>>>>>>
> >>>>>>>>>>> 6.
> >>>>>>>>>>> The problem with catching the exception in the downgrade
> process
> >>>>>> is
> >>>>>>>> that
> >>>>>>>>>>> would require new code in the Kafka version being downgraded
> to.
> >>>>>> Since
> >>>>>>>>>>> users could conceivably downgrade to almost *any* older version
> >>>>>> of
> >>>>>>>> Kafka
> >>>>>>>>>>> Streams, I'm not sure how we could add that code?
> >>>>>>>>>>> The only way I can think of doing it would be to provide a
> >>>>>> dedicated
> >>>>>>>>>>> downgrade tool, that goes through every local store and removes
> >>>>>> the
> >>>>>>>>>>> offsets column families. But that seems like an unnecessary
> >>>>>> amount of
> >>>>>>>>>> extra
> >>>>>>>>>>> code to maintain just to handle a somewhat niche situation,
> when
> >>>>>> the
> >>>>>>>>>>> alternative (automatically wipe and restore stores) should be
> >>>>>>>>>> acceptable.
> >>>>>>>>>>>
> >>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've requested.
> >>>>>>>>>>>
> >>>>>>>>>>> 3a.
> >>>>>>>>>>> I agree that IsolationLevel makes more sense at query-time, and
> >> I
> >>>>>>>>>> actually
> >>>>>>>>>>> initially attempted to place the IsolationLevel at query-time,
> >>>>>> but I
> >>>>>>>> ran
> >>>>>>>>>>> into some problems:
> >>>>>>>>>>> - The key issue is that, under ALOS we're not staging writes in
> >>>>>>>>>>> transactions, so can't perform writes at the READ_COMMITTED
> >>>>>> isolation
> >>>>>>>>>>> level. However, this may be addressed if we decide to *always*
> >>>>>> use
> >>>>>>>>>>> transactions as discussed under 3b.
> >>>>>>>>>>> - IQv1 and IQv2 have quite different implementations. I
> remember
> >>>>>>>> having
> >>>>>>>>>>> some difficulty understanding the IQv1 internals, which made it
> >>>>>>>>>> difficult
> >>>>>>>>>>> to determine what needed to be changed. However, I *think* this
> >>>>>> can be
> >>>>>>>>>>> addressed for both implementations by wrapping the RocksDBStore
> >>>>>> in an
> >>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides read methods
> >>>>>> (get,
> >>>>>>>>>> etc.)
> >>>>>>>>>>> to either read directly from the database or from the ongoing
> >>>>>>>>>> transaction.
> >>>>>>>>>>> But IQv1 might still be difficult.
> >>>>>>>>>>> - If IsolationLevel becomes a query constraint, then all other
> >>>>>>>>>> StateStores
> >>>>>>>>>>> will need to respect it, including the in-memory stores. This
> >>>>>> would
> >>>>>>>>>> require
> >>>>>>>>>>> us to adapt in-memory stores to stage their writes so they can
> >> be
> >>>>>>>>>> isolated
> >>>>>>>>>>> from READ_COMMITTTED queries. It would also become an important
> >>>>>>>>>>> consideration for third-party stores on upgrade, as without
> >>>>>> changes,
> >>>>>>>>>> they
> >>>>>>>>>>> would not support READ_COMMITTED queries correctly.
> >>>>>>>>>>>
> >>>>>>>>>>> Ultimately, I may need some help making the necessary change to
> >>>>>> IQv1
> >>>>>>>> to
> >>>>>>>>>>> support this, but I don't think it's fundamentally impossible,
> >>>>>> if we
> >>>>>>>>>> want
> >>>>>>>>>>> to pursue this route.
> >>>>>>>>>>>
> >>>>>>>>>>> 3b.
> >>>>>>>>>>> The main reason I chose to keep ALOS un-transactional was to
> >>>>>> minimize
> >>>>>>>>>>> behavioural change for most users (I believe most Streams users
> >>>>>> use
> >>>>>>>> the
> >>>>>>>>>>> default configuration, which is ALOS). That said, it's clear
> >>>>>> that if
> >>>>>>>>>> ALOS
> >>>>>>>>>>> also used transactional stores, the only change in behaviour
> >>>>>> would be
> >>>>>>>>>> that
> >>>>>>>>>>> it would become *more correct*, which could be considered a
> "bug
> >>>>>> fix"
> >>>>>>>> by
> >>>>>>>>>>> users, rather than a change they need to handle.
> >>>>>>>>>>>
> >>>>>>>>>>> I believe that performance using transactions (aka. RocksDB
> >>>>>>>>>> WriteBatches)
> >>>>>>>>>>> should actually be *better* than the un-batched write-path that
> >>>>>> is
> >>>>>>>>>>> currently used[1]. The only "performance" consideration will be
> >>>>>> the
> >>>>>>>>>>> increased memory usage that transactions require. Given the
> >>>>>>>> mitigations
> >>>>>>>>>> for
> >>>>>>>>>>> this memory that we have in place, I would expect that this is
> >>>>>> not a
> >>>>>>>>>>> problem for most users.
> >>>>>>>>>>>
> >>>>>>>>>>> If we're happy to do so, we can make ALOS also use
> transactions.
> >>>>>>>>>>>
> >>>>>>>>>>> Regards,
> >>>>>>>>>>> Nick
> >>>>>>>>>>>
> >>>>>>>>>>> Link 1:
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>>>>
> >>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> >> cadonna@apache.org
> >>>>>>>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks for the updates and sorry for the delay on my side!
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> 1.
> >>>>>>>>>>>> Making the default implementation for flush() a no-op sounds
> >>>>>> good to
> >>>>>>>>>> me.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> 2.
> >>>>>>>>>>>> I think what was bugging me here is that a third-party state
> >>>>>> store
> >>>>>>>>>> needs
> >>>>>>>>>>>> to implement the state store interface. That means they need
> to
> >>>>>>>>>>>> implement a wrapper around the actual state store as we do for
> >>>>>>>> RocksDB
> >>>>>>>>>>>> with RocksDBStore. So, a third-party state store can always
> >>>>>> estimate
> >>>>>>>>>> the
> >>>>>>>>>>>> uncommitted bytes, if it wants, because the wrapper can record
> >>>>>> the
> >>>>>>>>>> added
> >>>>>>>>>>>> bytes.
> >>>>>>>>>>>> One case I can think of where returning -1 makes sense is when
> >>>>>>>> Streams
> >>>>>>>>>>>> does not need to estimate the size of the write batch and
> >>>>>> trigger
> >>>>>>>>>>>> extraordinary commits, because the third-party state store
> >>>>>> takes care
> >>>>>>>>>> of
> >>>>>>>>>>>> memory. But in that case the method could also just return 0.
> >>>>>> Even
> >>>>>>>> that
> >>>>>>>>>>>> case would be better solved with a method that returns whether
> >>>>>> the
> >>>>>>>>>> state
> >>>>>>>>>>>> store manages itself the memory used for uncommitted bytes or
> >>>>>> not.
> >>>>>>>>>>>> Said that, I am fine with keeping the -1 return value, I was
> >>>>>> just
> >>>>>>>>>>>> wondering when and if it will be used.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Regarding returning 0 for transactional state stores when the
> >>>>>> batch
> >>>>>>>> is
> >>>>>>>>>>>> empty, I was just wondering because you explicitly stated
> >>>>>>>>>>>>
> >>>>>>>>>>>> "or {@code 0} if this StateStore does not support
> >> transactions."
> >>>>>>>>>>>>
> >>>>>>>>>>>> So it seemed to me returning 0 could only happen for
> >>>>>>>> non-transactional
> >>>>>>>>>>>> state stores.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> 3.
> >>>>>>>>>>>>
> >>>>>>>>>>>> a) What do you think if we move the isolation level to IQ (v1
> >>>>>> and
> >>>>>>>> v2)?
> >>>>>>>>>>>> In the end this is the only component that really needs to
> >>>>>> specify
> >>>>>>>> the
> >>>>>>>>>>>> isolation level. It is similar to the Kafka consumer that can
> >>>>>> choose
> >>>>>>>>>>>> with what isolation level to read the input topic.
> >>>>>>>>>>>> For IQv1 the isolation level should go into
> >>>>>> StoreQueryParameters. For
> >>>>>>>>>>>> IQv2, I would add it to the Query interface.
> >>>>>>>>>>>>
> >>>>>>>>>>>> b) Point a) raises the question what should happen during
> >>>>>>>> at-least-once
> >>>>>>>>>>>> processing when the state store does not use transactions?
> John
> >>>>>> in
> >>>>>>>> the
> >>>>>>>>>>>> past proposed to also use transactions on state stores for
> >>>>>>>>>>>> at-least-once. I like that idea, because it avoids aggregating
> >>>>>> the
> >>>>>>>> same
> >>>>>>>>>>>> records over and over again in the case of a failure. We had a
> >>>>>> case
> >>>>>>>> in
> >>>>>>>>>>>> the past where a Streams applications in at-least-once mode
> was
> >>>>>>>> failing
> >>>>>>>>>>>> continuously for some reasons I do not remember before
> >>>>>> committing the
> >>>>>>>>>>>> offsets. After each failover, the app aggregated again and
> >>>>>> again the
> >>>>>>>>>>>> same records. Of course the aggregate increased to very wrong
> >>>>>> values
> >>>>>>>>>>>> just because of the failover. With transactions on the state
> >>>>>> stores
> >>>>>>>> we
> >>>>>>>>>>>> could have avoided this. The app would have output the same
> >>>>>> aggregate
> >>>>>>>>>>>> multiple times (i.e., after each failover) but at least the
> >>>>>> value of
> >>>>>>>>>> the
> >>>>>>>>>>>> aggregate would not depend on the number of failovers.
> >>>>>> Outputting the
> >>>>>>>>>>>> same aggregate multiple times would be incorrect under
> >>>>>> exactly-once
> >>>>>>>> but
> >>>>>>>>>>>> it is OK for at-least-once.
> >>>>>>>>>>>> If it makes sense to add a config to turn on and off
> >>>>>> transactions on
> >>>>>>>>>>>> state stores under at-least-once or just use transactions in
> >>>>>> any case
> >>>>>>>>>> is
> >>>>>>>>>>>> a question we should also discuss in this KIP. It depends a
> bit
> >>>>>> on
> >>>>>>>> the
> >>>>>>>>>>>> performance trade-off. Maybe to be safe, I would add a config.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> 4.
> >>>>>>>>>>>> Your points are all valid. I tend to say to keep the metrics
> >>>>>> around
> >>>>>>>>>>>> flush() until we remove flush() completely from the interface.
> >>>>>> Calls
> >>>>>>>> to
> >>>>>>>>>>>> flush() might still exist since existing processors might
> still
> >>>>>> call
> >>>>>>>>>>>> flush() explicitly as you mentioned in 1). For sure, we need
> to
> >>>>>>>>>> document
> >>>>>>>>>>>> how the metrics change due to the transactions in the upgrade
> >>>>>> notes.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> 5.
> >>>>>>>>>>>> I see. Then you should describe how the .position files are
> >>>>>> handled
> >>>>>>>> in
> >>>>>>>>>>>> a dedicated section of the KIP or incorporate the description
> >>>>>> in the
> >>>>>>>>>>>> "Atomic Checkpointing" section instead of only mentioning it
> in
> >>>>>> the
> >>>>>>>>>>>> "Compatibility, 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:
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>
> >>
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> >>>>>>>>>>>>
> >>>>>>>>>>>> It is a bit brittle, but it works.
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> Best,
> >>>>>>>>>>>> Bruno
> >>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> >>>>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks for taking the time to review the KIP. I'm back from
> >>>>>> leave
> >>>>>>>> now
> >>>>>>>>>> and
> >>>>>>>>>>>>> intend to move this forwards as quickly as I can.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Addressing your points:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 1.
> >>>>>>>>>>>>> Because flush() is part of the StateStore API, it's exposed
> to
> >>>>>>>> custom
> >>>>>>>>>>>>> Processors, which might be making calls to flush(). This was
> >>>>>>>> actually
> >>>>>>>>>> the
> >>>>>>>>>>>>> case in a few integration tests.
> >>>>>>>>>>>>> To maintain as much compatibility as possible, I'd prefer not
> >>>>>> to
> >>>>>>>> make
> >>>>>>>>>>>> this
> >>>>>>>>>>>>> an UnsupportedOperationException, as it will cause previously
> >>>>>>>> working
> >>>>>>>>>>>>> Processors to start throwing exceptions at runtime.
> >>>>>>>>>>>>> I agree that it doesn't make sense for it to proxy commit(),
> >>>>>> though,
> >>>>>>>>>> as
> >>>>>>>>>>>>> that would cause it to violate the "StateStores commit only
> >>>>>> when the
> >>>>>>>>>> Task
> >>>>>>>>>>>>> commits" rule.
> >>>>>>>>>>>>> Instead, I think we should make this a no-op. That way,
> >>>>>> existing
> >>>>>>>> user
> >>>>>>>>>>>>> Processors will continue to work as-before, without violation
> >>>>>> of
> >>>>>>>> store
> >>>>>>>>>>>>> consistency that would be caused by premature flush/commit of
> >>>>>>>>>> StateStore
> >>>>>>>>>>>>> data to disk.
> >>>>>>>>>>>>> What do you think?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 2.
> >>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore implementation is
> >>>>>>>>>>>>> transactional, but is unable to estimate the uncommitted
> >> memory
> >>>>>>>> usage,
> >>>>>>>>>>>> the
> >>>>>>>>>>>>> method will return -1.
> >>>>>>>>>>>>> The intention here is to permit third-party implementations
> >>>>>> that may
> >>>>>>>>>> not
> >>>>>>>>>>>> be
> >>>>>>>>>>>>> able to estimate memory usage.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Yes, it will be 0 when nothing has been written to the store
> >>>>>> yet. I
> >>>>>>>>>>>> thought
> >>>>>>>>>>>>> that was implied by "This method will return an approximation
> >>>>>> of the
> >>>>>>>>>>>> memory
> >>>>>>>>>>>>> would be freed by the next call to {@link #commit(Map)}" and
> >>>>>>>> "@return
> >>>>>>>>>> The
> >>>>>>>>>>>>> approximate size of all records awaiting {@link
> >> #commit(Map)}",
> >>>>>>>>>> however,
> >>>>>>>>>>>> I
> >>>>>>>>>>>>> can add it explicitly to the JavaDoc if you think this is
> >>>>>> unclear?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 3.
> >>>>>>>>>>>>> I realise this is probably the most contentious point in my
> >>>>>> design,
> >>>>>>>>>> and
> >>>>>>>>>>>> I'm
> >>>>>>>>>>>>> open to changing it if I'm unable to convince you of the
> >>>>>> benefits.
> >>>>>>>>>>>>> Nevertheless, here's my argument:
> >>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly provided
> >>>>>> StateStores
> >>>>>>>> to
> >>>>>>>>>>>>> query, and it may be important for users to programmatically
> >>>>>> know
> >>>>>>>>>> which
> >>>>>>>>>>>>> mode the StateStore is operating under. If we simply provide
> >> an
> >>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the internal streams
> >>>>>>>>>> engine), or
> >>>>>>>>>>>>> similar, then users will need to understand the operation and
> >>>>>>>>>>>> consequences
> >>>>>>>>>>>>> of each available processing mode and how it pertains to
> their
> >>>>>>>>>>>> StateStore.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Interactive Query users aren't the only people that care
> about
> >>>>>> the
> >>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore: implementers
> >> of
> >>>>>>>> custom
> >>>>>>>>>>>>> StateStores also need to understand the behaviour expected of
> >>>>>> their
> >>>>>>>>>>>>> implementation. KIP-892 introduces some assumptions into the
> >>>>>> Streams
> >>>>>>>>>>>> Engine
> >>>>>>>>>>>>> about how StateStores operate under each processing mode, and
> >>>>>> it's
> >>>>>>>>>>>>> important that custom implementations adhere to those
> >>>>>> assumptions in
> >>>>>>>>>>>> order
> >>>>>>>>>>>>> to maintain the consistency guarantees.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> IsolationLevels provide a high-level contract on the
> behaviour
> >>>>>> of
> >>>>>>>> the
> >>>>>>>>>>>>> StateStore: a user knows that under READ_COMMITTED, they will
> >>>>>> see
> >>>>>>>>>> writes
> >>>>>>>>>>>>> only after the Task has committed, and under READ_UNCOMMITTED
> >>>>>> they
> >>>>>>>>>> will
> >>>>>>>>>>>> see
> >>>>>>>>>>>>> writes immediately. No understanding of the details of each
> >>>>>>>>>>>> processing.mode
> >>>>>>>>>>>>> is required, either for IQ users or StateStore implementers.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> An argument can be made that these contractual guarantees can
> >>>>>> simply
> >>>>>>>>>> be
> >>>>>>>>>>>>> documented for the processing.mode (i.e. that exactly-once
> and
> >>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and at-least-once
> >>>>>> behaves
> >>>>>>>>>> like
> >>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small issues with
> >>>>>> this I'd
> >>>>>>>>>>>> prefer
> >>>>>>>>>>>>> to avoid:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>        - Where would we document these contracts, in a way
> that
> >>>>>> is
> >>>>>>>>>> difficult
> >>>>>>>>>>>>>        for users/implementers to miss/ignore?
> >>>>>>>>>>>>>        - It's not clear to users that the processing mode is
> >>>>>>>>>> communicating
> >>>>>>>>>>>>>        an expectation of read isolation, unless they read the
> >>>>>>>>>>>> documentation. Users
> >>>>>>>>>>>>>        rarely consult documentation unless they feel they
> need
> >>>>>> to, so
> >>>>>>>>>> it's
> >>>>>>>>>>>> likely
> >>>>>>>>>>>>>        this detail would get missed by many users.
> >>>>>>>>>>>>>        - It tightly couples processing modes to read
> isolation.
> >>>>>> Adding
> >>>>>>>>>> new
> >>>>>>>>>>>>>        processing modes, or changing the read isolation of
> >>>>>> existing
> >>>>>>>>>>>> processing
> >>>>>>>>>>>>>        modes would be difficult/impossible.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Ultimately, the cost of introducing IsolationLevels is just a
> >>>>>> single
> >>>>>>>>>>>>> method, since we re-use the existing IsolationLevel enum from
> >>>>>> Kafka.
> >>>>>>>>>> This
> >>>>>>>>>>>>> gives us a clear place to document the contractual guarantees
> >>>>>>>> expected
> >>>>>>>>>>>>> of/provided by StateStores, that is accessible both by the
> >>>>>>>> StateStore
> >>>>>>>>>>>>> itself, and by IQ users.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> (Writing this I've just realised that the StateStore and IQ
> >>>>>> APIs
> >>>>>>>>>> actually
> >>>>>>>>>>>>> don't provide access to StateStoreContext that IQ users would
> >>>>>> have
> >>>>>>>>>> direct
> >>>>>>>>>>>>> access to... Perhaps StateStore should expose
> isolationLevel()
> >>>>>>>> itself
> >>>>>>>>>>>> too?)
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 4.
> >>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics in-place
> >>>>>> either, as
> >>>>>>>>>> it's a
> >>>>>>>>>>>>> backwards incompatible change. My concern is that, if we
> leave
> >>>>>> the
> >>>>>>>>>>>> existing
> >>>>>>>>>>>>> "flush" metrics in place, they will be confusing to users.
> >>>>>> Right
> >>>>>>>> now,
> >>>>>>>>>>>>> "flush" metrics record explicit flushes to disk, but under
> >>>>>> KIP-892,
> >>>>>>>>>> even
> >>>>>>>>>>>> a
> >>>>>>>>>>>>> commit() will not explicitly flush data to disk - RocksDB
> will
> >>>>>>>> decide
> >>>>>>>>>> on
> >>>>>>>>>>>>> when to flush memtables to disk itself.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd have two
> >> options,
> >>>>>>>> which
> >>>>>>>>>>>> both
> >>>>>>>>>>>>> seem pretty bad to me:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>        1. Have them record calls to commit(), which would be
> >>>>>>>>>> misleading, as
> >>>>>>>>>>>>>        data is no longer explicitly "flushed" to disk by this
> >>>>>> call.
> >>>>>>>>>>>>>        2. Have them record nothing at all, which is
> equivalent
> >> to
> >>>>>>>>>> removing
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>        metrics, except that users will see the metric still
> >>>>>> exists and
> >>>>>>>>>> so
> >>>>>>>>>>>> assume
> >>>>>>>>>>>>>        that the metric is correct, and that there's a problem
> >>>>>> with
> >>>>>>>> their
> >>>>>>>>>>>> system
> >>>>>>>>>>>>>        when there isn't.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I agree that removing them is also a bad solution, and I'd
> >>>>>> like some
> >>>>>>>>>>>>> guidance on the best path forward here.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 5.
> >>>>>>>>>>>>> Position files are updated on every write to a StateStore.
> >>>>>> Since our
> >>>>>>>>>>>> writes
> >>>>>>>>>>>>> are now buffered until commit(), we can't update the Position
> >>>>>> file
> >>>>>>>>>> until
> >>>>>>>>>>>>> commit() has been called, otherwise it would be inconsistent
> >>>>>> with
> >>>>>>>> the
> >>>>>>>>>>>> data
> >>>>>>>>>>>>> in the event of a rollback. Consequently, we need to manage
> >>>>>> these
> >>>>>>>>>> offsets
> >>>>>>>>>>>>> the same way we manage the checkpoint offsets, and ensure
> >>>>>> they're
> >>>>>>>> only
> >>>>>>>>>>>>> written on commit().
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> 6.
> >>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what tests to
> write.
> >>>>>> How
> >>>>>>>>>>>> explicit
> >>>>>>>>>>>>> do we need to be here in the KIP?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be seamless,
> >>>>>> and we
> >>>>>>>>>>>> should
> >>>>>>>>>>>>> definitely add some tests around that. Downgrade, it
> >>>>>> transpires,
> >>>>>>>> isn't
> >>>>>>>>>>>>> currently possible, as the extra column family for offset
> >>>>>> storage is
> >>>>>>>>>>>>> incompatible with the pre-KIP-892 implementation: when you
> >>>>>> open a
> >>>>>>>>>> RocksDB
> >>>>>>>>>>>>> database, you must open all available column families or
> >>>>>> receive an
> >>>>>>>>>>>> error.
> >>>>>>>>>>>>> What currently happens on downgrade is that it attempts to
> >>>>>> open the
> >>>>>>>>>>>> store,
> >>>>>>>>>>>>> throws an error about the offsets column family not being
> >>>>>> opened,
> >>>>>>>>>> which
> >>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given that
> downgrades
> >>>>>>>> should
> >>>>>>>>>> be
> >>>>>>>>>>>>> uncommon, I think this is acceptable behaviour, as the
> >>>>>> end-state is
> >>>>>>>>>>>>> consistent, even if it results in an undesirable state
> >> restore.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Should I document the upgrade/downgrade behaviour explicitly
> >>>>>> in the
> >>>>>>>>>> KIP?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> --
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> >>>>>> cadonna@apache.org>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi Nick!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks for the updates!
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>> Why does StateStore#flush() default to
> >>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> >>>>>>>>>>>>>> Since calls to flush() will not exist anymore after this KIP
> >>>>>> is
> >>>>>>>>>>>>>> released, I would rather throw an unsupported operation
> >>>>>> exception
> >>>>>>>> by
> >>>>>>>>>>>>>> default.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>> When would a state store return -1 from
> >>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while being
> >>>>>>>>>> transactional?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Wouldn't StateStore#approximateNumUncommittedBytes() also
> >>>>>> return 0
> >>>>>>>> if
> >>>>>>>>>>>>>> the state store is transactional but nothing has been
> written
> >>>>>> to
> >>>>>>>> the
> >>>>>>>>>>>>>> state store yet?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>> Sorry for bringing this up again. Does this KIP really need
> >> to
> >>>>>>>>>> introduce
> >>>>>>>>>>>>>> StateStoreContext#isolationLevel()? StateStoreContext has
> >>>>>> already
> >>>>>>>>>>>>>> appConfigs() which basically exposes the same information,
> >>>>>> i.e., if
> >>>>>>>>>> EOS
> >>>>>>>>>>>>>> is enabled or not.
> >>>>>>>>>>>>>> In one of your previous e-mails you wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> "My idea was to try to keep the StateStore interface as
> >>>>>> loosely
> >>>>>>>>>> coupled
> >>>>>>>>>>>>>> from the Streams engine as possible, to give implementers
> >> more
> >>>>>>>>>> freedom,
> >>>>>>>>>>>>>> and reduce the amount of internal knowledge required."
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> While I understand the intent, I doubt that it decreases the
> >>>>>>>>>> coupling of
> >>>>>>>>>>>>>> a StateStore interface and the Streams engine.
> READ_COMMITTED
> >>>>>> only
> >>>>>>>>>>>>>> applies to IQ but not to reads by processors. Thus,
> >>>>>> implementers
> >>>>>>>>>> need to
> >>>>>>>>>>>>>> understand how Streams accesses the state stores.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I would like to hear what others think about this.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 4.
> >>>>>>>>>>>>>> Great exposing new metrics for transactional state stores!
> >>>>>>>> However, I
> >>>>>>>>>>>>>> would prefer to add new metrics and deprecate (in the docs)
> >>>>>> the 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 also need to adapt/add integration tests
> >>>>>> besides
> >>>>>>>> unit
> >>>>>>>>>>>>>> tests. Additionally, we probably need integration or system
> >>>>>> tests
> >>>>>>>> to
> >>>>>>>>>>>>>> verify that upgrades and downgrades between transactional
> and
> >>>>>>>>>>>>>> non-transactional state stores work as expected.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> >>>>>>>>>>>>>>> One more thing: I noted John's suggestion in the KIP, under
> >>>>>>>>>> "Rejected
> >>>>>>>>>>>>>>> Alternatives". I still think it's an idea worth pursuing,
> >>>>>> but I
> >>>>>>>>>> believe
> >>>>>>>>>>>>>>> that it's out of the scope of this KIP, because it solves a
> >>>>>>>>>> different
> >>>>>>>>>>>> set
> >>>>>>>>>>>>>>> of problems to this KIP, and the scope of this one has
> >>>>>> already
> >>>>>>>> grown
> >>>>>>>>>>>>>> quite
> >>>>>>>>>>>>>>> large!
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> >>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I've updated the KIP (
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>> )
> >>>>>>>>>>>>>>>> with the latest changes; mostly bringing back "Atomic
> >>>>>>>>>> Checkpointing"
> >>>>>>>>>>>>>> (for
> >>>>>>>>>>>>>>>> what feels like the 10th time!). I think the one thing
> >>>>>> missing is
> >>>>>>>>>> some
> >>>>>>>>>>>>>>>> changes to metrics (notably the store "flush" metrics will
> >>>>>> need
> >>>>>>>> to
> >>>>>>>>>> be
> >>>>>>>>>>>>>>>> renamed to "commit").
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> The reason I brought back Atomic Checkpointing was to
> >>>>>> decouple
> >>>>>>>>>> store
> >>>>>>>>>>>>>> flush
> >>>>>>>>>>>>>>>> from store commit. This is important, because with
> >>>>>> Transactional
> >>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on *every* Task
> >>>>>> commit,
> >>>>>>>>>> and
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>>>> just when the StateStore is closing, otherwise our
> >>>>>> transaction
> >>>>>>>>>> buffer
> >>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>> never be written and persisted, instead growing unbounded!
> >> I
> >>>>>>>>>>>>>> experimented
> >>>>>>>>>>>>>>>> with some simple solutions, like forcing a store flush
> >>>>>> whenever
> >>>>>>>> the
> >>>>>>>>>>>>>>>> transaction buffer was likely to exceed its configured
> >>>>>> size, but
> >>>>>>>>>> this
> >>>>>>>>>>>>>> was
> >>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer from being
> >>>>>>>> configured
> >>>>>>>>>> to
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>>> unbounded, and it still would have required explicit
> >>>>>> flushes of
> >>>>>>>>>>>> RocksDB,
> >>>>>>>>>>>>>>>> yielding sub-optimal performance and memory utilization.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right" way to
> >>>>>> resolve
> >>>>>>>> this
> >>>>>>>>>>>>>>>> problem. By ensuring that the changelog offsets that
> >>>>>> correspond
> >>>>>>>> to
> >>>>>>>>>> the
> >>>>>>>>>>>>>> most
> >>>>>>>>>>>>>>>> recently written records are always atomically written to
> >>>>>> the
> >>>>>>>>>>>> StateStore
> >>>>>>>>>>>>>>>> (by writing them to the same transaction buffer), we can
> >>>>>> avoid
> >>>>>>>>>>>> forcibly
> >>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting RocksDB
> >>>>>> flush
> >>>>>>>> them
> >>>>>>>>>>>> only
> >>>>>>>>>>>>>>>> when necessary, without losing any of our consistency
> >>>>>> guarantees.
> >>>>>>>>>> See
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> updated KIP for more info.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I have fully implemented these changes, although I'm still
> >>>>>> not
> >>>>>>>>>>>> entirely
> >>>>>>>>>>>>>>>> happy with the implementation for segmented StateStores,
> so
> >>>>>> I
> >>>>>>>> plan
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>> refactor that. Despite that, all tests pass. If you'd like
> >>>>>> to try
> >>>>>>>>>> out
> >>>>>>>>>>>> or
> >>>>>>>>>>>>>>>> review this highly experimental and incomplete branch,
> it's
> >>>>>>>>>> available
> >>>>>>>>>>>>>> here:
> >>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0.
> >>>>>> Note:
> >>>>>>>>>> it's
> >>>>>>>>>>>>>> built
> >>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base to build
> >>>>>> and test
> >>>>>>>>>> it
> >>>>>>>>>>>> on,
> >>>>>>>>>>>>>>>> and to enable easy apples-to-apples comparisons in a live
> >>>>>>>>>>>> environment. I
> >>>>>>>>>>>>>>>> plan to rebase it against trunk once it's nearer
> completion
> >>>>>> and
> >>>>>>>> has
> >>>>>>>>>>>> been
> >>>>>>>>>>>>>>>> proven on our main application.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I would really appreciate help in reviewing and testing:
> >>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window) stores
> >>>>>>>>>>>>>>>> - Global stores
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> As I do not currently use either of these, so my primary
> >>>>>> test
> >>>>>>>>>>>>>> environment
> >>>>>>>>>>>>>>>> doesn't test these areas.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I'm going on Parental Leave starting next week for a few
> >>>>>> weeks,
> >>>>>>>> so
> >>>>>>>>>>>> will
> >>>>>>>>>>>>>>>> not have time to move this forward until late August. That
> >>>>>> said,
> >>>>>>>>>> your
> >>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just won't be able
> >> to
> >>>>>>>>>> respond
> >>>>>>>>>>>> as
> >>>>>>>>>>>>>>>> quickly as usual.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> >>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi Bruno
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ is not
> >>>>>> something
> >>>>>>>> I
> >>>>>>>>>> had
> >>>>>>>>>>>>>>>>> considered.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> What about atomically updating the state store from the
> >>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
> >>>>>> (thus,
> >>>>>>>>>>>>>> flushing
> >>>>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> >>>>>> number of
> >>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>> intervals?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting that we
> >>>>>> add an
> >>>>>>>>>>>>>> additional
> >>>>>>>>>>>>>>>>> config for the max number of commit intervals between
> >>>>>>>> checkpoints?
> >>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the transaction
> >>>>>> buffers
> >>>>>>>> are
> >>>>>>>>>>>>>> nearly
> >>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of commit intervals
> >>>>>> have
> >>>>>>>>>>>> elapsed,
> >>>>>>>>>>>>>>>>> whichever comes first?
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> That certainly seems reasonable, although this re-ignites
> >>>>>> an
> >>>>>>>>>> earlier
> >>>>>>>>>>>>>>>>> debate about whether a config should be measured in
> >>>>>> "number of
> >>>>>>>>>> commit
> >>>>>>>>>>>>>>>>> intervals", instead of just an absolute time.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the reason I was
> >>>>>> pursuing
> >>>>>>>> the
> >>>>>>>>>>>>>> Atomic
> >>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush from
> >>>>>> checkpointing,
> >>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>> enables us to just checkpoint on every commit without any
> >>>>>>>>>> performance
> >>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the "best"
> >>>>>> solution,
> >>>>>>>>>> but
> >>>>>>>>>>>>>> I'm not
> >>>>>>>>>>>>>>>>> sure if this is enough to bring it back into this KIP.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I'm currently working on moving all the transactional
> >> logic
> >>>>>>>>>> directly
> >>>>>>>>>>>>>> into
> >>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with the
> >>>>>>>>>>>> StateStore#newTransaction
> >>>>>>>>>>>>>>>>> method, and reduces the number of new classes introduced,
> >>>>>>>>>>>> significantly
> >>>>>>>>>>>>>>>>> reducing the complexity. If it works, and the complexity
> >> is
> >>>>>>>>>>>> drastically
> >>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic Checkpoints into
> >>>>>> this
> >>>>>>>> KIP.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
> >>>>>> cadonna@apache.org>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> As far as I understand, you want to atomically update
> the
> >>>>>> state
> >>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>> from the transaction buffer, flush the memtable of a
> >> state
> >>>>>>>> store
> >>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> write the checkpoint not after the commit time elapsed
> >> but
> >>>>>>>> after
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> transaction buffer reached a size that would lead to
> >>>>>> exceeding
> >>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes before the next
> >>>>>> commit
> >>>>>>>>>>>>>> interval
> >>>>>>>>>>>>>>>>>> ends.
> >>>>>>>>>>>>>>>>>> That means, the Kafka transaction would commit every
> >>>>>> commit
> >>>>>>>>>> interval
> >>>>>>>>>>>>>> but
> >>>>>>>>>>>>>>>>>> the state store will only be atomically updated roughly
> >>>>>> every
> >>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ
> >>>>>> would
> >>>>>>>>>> then
> >>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>> see new data roughly every
> >>>>>>>>>> statestore.transaction.buffer.max.bytes.
> >>>>>>>>>>>>>>>>>> After a failure the state store needs to restore up to
> >>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Is this correct?
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> What about atomically updating the state store from the
> >>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
> >>>>>> (thus,
> >>>>>>>>>>>>>> flushing
> >>>>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> >>>>>> number of
> >>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>> intervals? In such a way, we would have the same delay
> >> for
> >>>>>>>>>> records
> >>>>>>>>>>>>>>>>>> appearing in output topics and IQ because both would
> >>>>>> appear
> >>>>>>>> when
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> Kafka transaction is committed. However, after a failure
> >>>>>> the
> >>>>>>>>>> state
> >>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>> still needs to restore up to
> >>>>>>>>>> statestore.transaction.buffer.max.bytes
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> it might restore data that is already in the state store
> >>>>>>>> because
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable offset (i.e. the
> >>>>>> last
> >>>>>>>>>>>> committed
> >>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring data that is
> >>>>>> already
> >>>>>>>>>> in
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> state store is idempotent, so eos should not violated.
> >>>>>>>>>>>>>>>>>> This solution needs at least one new config to specify
> >>>>>> when a
> >>>>>>>>>>>>>> checkpoint
> >>>>>>>>>>>>>>>>>> should be written.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> A small correction to your previous e-mail that does not
> >>>>>> change
> >>>>>>>>>>>>>> anything
> >>>>>>>>>>>>>>>>>> you said: Under alos the default commit interval is 30
> >>>>>> seconds,
> >>>>>>>>>> not
> >>>>>>>>>>>>>> five
> >>>>>>>>>>>>>>>>>> seconds.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I've begun performance testing my branch on our staging
> >>>>>>>>>>>> environment,
> >>>>>>>>>>>>>>>>>>> putting it through its paces in our non-trivial
> >>>>>> application.
> >>>>>>>> I'm
> >>>>>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>>>>> observing the same increased flush rate that we saw the
> >>>>>> last
> >>>>>>>>>> time
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP, but this time,
> I
> >>>>>>>> think I
> >>>>>>>>>>>> know
> >>>>>>>>>>>>>>>>>> why.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at
> >>>>>> the end
> >>>>>>>>>> of
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> Task
> >>>>>>>>>>>>>>>>>>> commit process, has the following behaviour:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>          - Under ALOS: checkpoint the state stores.
> This
> >>>>>>>> includes
> >>>>>>>>>>>>>>>>>>>          flushing memtables in RocksDB. This is
> >> acceptable
> >>>>>>>>>> because the
> >>>>>>>>>>>>>>>>>> default
> >>>>>>>>>>>>>>>>>>>          commit.interval.ms is 5 seconds, so forcibly
> >>>>>> flushing
> >>>>>>>>>>>> memtables
> >>>>>>>>>>>>>>>>>> every 5
> >>>>>>>>>>>>>>>>>>>          seconds is acceptable for most applications.
> >>>>>>>>>>>>>>>>>>>          - Under EOS: checkpointing is not done,
> *unless*
> >>>>>> it's
> >>>>>>>>>> being
> >>>>>>>>>>>>>>>>>> forced, due
> >>>>>>>>>>>>>>>>>>>          to e.g. the Task closing or being revoked.
> This
> >>>>>> means
> >>>>>>>>>> that
> >>>>>>>>>>>> under
> >>>>>>>>>>>>>>>>>> normal
> >>>>>>>>>>>>>>>>>>>          processing conditions, the state stores will
> not
> >>>>>> be
> >>>>>>>>>>>>>> checkpointed,
> >>>>>>>>>>>>>>>>>> and will
> >>>>>>>>>>>>>>>>>>>          not have memtables flushed at all , unless
> >> RocksDB
> >>>>>>>>>> decides to
> >>>>>>>>>>>>>>>>>> flush them on
> >>>>>>>>>>>>>>>>>>>          its own. Checkpointing stores and
> force-flushing
> >>>>>> their
> >>>>>>>>>>>> memtables
> >>>>>>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>>>>>>>          done when a Task is being closed.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at
> >> least
> >>>>>>>> *some*
> >>>>>>>>>>>>>> normal
> >>>>>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB transaction
> >>>>>>>> buffers
> >>>>>>>>>> to
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets are synced to
> >>>>>> disk to
> >>>>>>>>>>>> prevent
> >>>>>>>>>>>>>>>>>>> restores from getting out of hand. Consequently, my
> >>>>>> current
> >>>>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is
> >>>>>> far too
> >>>>>>>>>>>>>>>>>> frequent.
> >>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records, which is
> a
> >>>>>>>> change
> >>>>>>>>>> in
> >>>>>>>>>>>>>>>>>> flush
> >>>>>>>>>>>>>>>>>>> behaviour, potentially causing performance problems for
> >>>>>> some
> >>>>>>>>>>>>>>>>>> applications.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm currently
> >>>>>> leaning
> >>>>>>>>>>>>>> towards
> >>>>>>>>>>>>>>>>>>> using the statestore.transaction.buffer.max.bytes
> >>>>>>>> configuration
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to exceed it. This
> >>>>>> would
> >>>>>>>>>>>>>>>>>> complement the
> >>>>>>>>>>>>>>>>>>> existing "early Task commit" functionality that this
> >>>>>>>>>> configuration
> >>>>>>>>>>>>>>>>>>> provides, in the following way:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>          - Currently, we use
> >>>>>>>>>> statestore.transaction.buffer.max.bytes
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> force an
> >>>>>>>>>>>>>>>>>>>          early Task commit if processing more records
> >> would
> >>>>>>>> cause
> >>>>>>>>>> our
> >>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>          transactions to exceed the memory assigned to
> >>>>>> them.
> >>>>>>>>>>>>>>>>>>>          - New functionality: when a Task *does*
> commit,
> >>>>>> we will
> >>>>>>>>>> not
> >>>>>>>>>>>>>>>>>> checkpoint
> >>>>>>>>>>>>>>>>>>>          the stores (and hence flush the transaction
> >>>>>> buffers)
> >>>>>>>>>> unless
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> expect to
> >>>>>>>>>>>>>>>>>>>          cross the
> >> statestore.transaction.buffer.max.bytes
> >>>>>>>>>> threshold
> >>>>>>>>>>>>>> before
> >>>>>>>>>>>>>>>>>> the next
> >>>>>>>>>>>>>>>>>>>          commit
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> >>>>>>>>>> nick.telford@gmail.com
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi Bruno!
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>> By "less predictable for users", I meant in terms of
> >>>>>>>>>> understanding
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> performance profile under various circumstances. The
> >>>>>> more
> >>>>>>>>>> complex
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> solution, the more difficult it would be for users to
> >>>>>>>>>> understand
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> performance they see. For example, spilling records to
> >>>>>> disk
> >>>>>>>>>> when
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold would, I
> expect,
> >>>>>>>> reduce
> >>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>> throughput. This reduction in write throughput could
> be
> >>>>>>>>>>>> unexpected,
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> potentially difficult to diagnose/understand for
> users.
> >>>>>>>>>>>>>>>>>>>> At the moment, I think the "early commit" concept is
> >>>>>>>> relatively
> >>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document, and
> >> conceptually
> >>>>>>>> fairly
> >>>>>>>>>>>>>>>>>> obvious to
> >>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to make it
> easier
> >>>>>> to
> >>>>>>>>>>>>>> understand
> >>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>> it happens though.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 3. (the second one)
> >>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an indirect way of
> >>>>>>>> telling
> >>>>>>>>>>>>>>>>>> StateStores
> >>>>>>>>>>>>>>>>>>>> whether they should be transactional. READ_COMMITTED
> >>>>>>>>>> essentially
> >>>>>>>>>>>>>>>>>> requires
> >>>>>>>>>>>>>>>>>>>> transactions, because it dictates that two threads
> >>>>>> calling
> >>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes from the
> other
> >>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all
> >>>>>> bets are
> >>>>>>>>>> off,
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> stores can allow threads to observe written records at
> >>>>>> any
> >>>>>>>>>> time,
> >>>>>>>>>>>>>>>>>> which is
> >>>>>>>>>>>>>>>>>>>> essentially "no transactions". That said, StateStores
> >>>>>> are
> >>>>>>>> free
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> implement
> >>>>>>>>>>>>>>>>>>>> these guarantees however they can, which is a bit more
> >>>>>>>> relaxed
> >>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>> dictating "you must use transactions". For example,
> >> with
> >>>>>>>>>> RocksDB
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
> >>>>>>>> "transactions",
> >>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But
> >>>>>> with
> >>>>>>>>>> other
> >>>>>>>>>>>>>>>>>> storage
> >>>>>>>>>>>>>>>>>>>> engines, it might be preferable to *always* use
> >>>>>> transactions,
> >>>>>>>>>> even
> >>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines that
> don't
> >>>>>>>> provide
> >>>>>>>>>>>>>>>>>>>> transactions, but the isolation guarantees can be met
> >>>>>> using a
> >>>>>>>>>>>>>>>>>> different
> >>>>>>>>>>>>>>>>>>>> technique.
> >>>>>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore interface as
> >>>>>>>> loosely
> >>>>>>>>>>>>>> coupled
> >>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> >>>>>> implementers
> >>>>>>>> more
> >>>>>>>>>>>>>>>>>> freedom, and
> >>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge required.
> >>>>>>>>>>>>>>>>>>>> That said, I understand that "IsolationLevel" might
> not
> >>>>>> be
> >>>>>>>> the
> >>>>>>>>>>>> right
> >>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it much more
> >>>>>> explicit if
> >>>>>>>>>>>>>>>>>> required, e.g.
> >>>>>>>>>>>>>>>>>>>> boolean transactional()
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 7-8.
> >>>>>>>>>>>>>>>>>>>> I can make these changes either later today or
> >> tomorrow.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Small update:
> >>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of
> >>>>>> issues
> >>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> needed
> >>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass, which is
> >>>>>>>> promising,
> >>>>>>>>>> but
> >>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>> need to undergo some performance testing. I haven't
> >>>>>> (yet)
> >>>>>>>>>> worked
> >>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I would
> >>>>>> expect
> >>>>>>>> that,
> >>>>>>>>>>>>>>>>>>>> behaviourally, it should make no difference. The
> branch
> >>>>>> is
> >>>>>>>>>>>> available
> >>>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> if
> >>>>>>>> anyone
> >>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>> interested in taking an early look.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> >>>>>>>>>> cadonna@apache.org>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually also my
> >>>>>> point. I
> >>>>>>>>>>>>>> understood
> >>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion path as a way
> to
> >>>>>> avoid
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> early
> >>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually it is public
> >>>>>> API. My
> >>>>>>>>>>>> question
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal processing.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>> My concern is that checking for the size of the
> >>>>>> transaction
> >>>>>>>>>>>> buffer
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit affects the whole
> >>>>>>>> processing
> >>>>>>>>>> of
> >>>>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a state store is not
> >>>>>>>>>> confined to
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and changes the
> >>>>>> behavior
> >>>>>>>>>> of
> >>>>>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you that it is a
> >>>>>> decent
> >>>>>>>>>>>>>>>>>> compromise. I
> >>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and list the
> >>>>>> options to
> >>>>>>>>>>>>>> overcome
> >>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all options seem
> >> quite
> >>>>>>>> heavy
> >>>>>>>>>>>>>>>>>> compared
> >>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you mean with
> >>>>>> "less
> >>>>>>>>>>>>>>>>>> predictable
> >>>>>>>>>>>>>>>>>>>>> for users", though.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I found the discussions about the alternatives really
> >>>>>>>>>>>> interesting.
> >>>>>>>>>>>>>>>>>> But I
> >>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good and we should
> >>>>>> continue
> >>>>>>>>>> with
> >>>>>>>>>>>>>> it!
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on June
> 20th:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind putting
> >>>>>> isolation
> >>>>>>>>>>>> level
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should that also be
> a
> >>>>>> way
> >>>>>>>> to
> >>>>>>>>>>>> give
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> the state store the opportunity to decide whether to
> >>>>>> turn on
> >>>>>>>>>>>>>>>>>>>>> transactions or not?
> >>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned about how do
> you
> >>>>>> know
> >>>>>>>>>> if a
> >>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written under EOS, if you
> >>>>>> do not
> >>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>>>>> to know if the state store is transactional or not.
> If
> >>>>>> a
> >>>>>>>> state
> >>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be written
> >>>>>> during
> >>>>>>>>>> normal
> >>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state store is not
> >>>>>>>> transactional,
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written under EOS.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 7.
> >>>>>>>>>>>>>>>>>>>>> My point was about not only considering the bytes in
> >>>>>> memory
> >>>>>>>> in
> >>>>>>>>>>>>>> config
> >>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that
> >>>>>> might
> >>>>>>>> be
> >>>>>>>>>>>>>>>>>> spilled
> >>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering whether you
> should
> >>>>>>>> remove
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory bytes to be
> used
> >>>>>> to
> >>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My thinking
> >>>>>> was
> >>>>>>>> that
> >>>>>>>>>>>> even
> >>>>>>>>>>>>>>>>>> if a
> >>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to disk,
> limiting
> >>>>>> the
> >>>>>>>>>>>> overall
> >>>>>>>>>>>>>>>>>> bytes
> >>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it again and
> >>>>>> considering
> >>>>>>>> the
> >>>>>>>>>>>>>> recent
> >>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much sense anymore.
> >>>>>>>>>>>>>>>>>>>>> I like the name
> >>>>>> statestore.transaction.buffer.max.bytes that
> >>>>>>>>>> you
> >>>>>>>>>>>>>>>>>> proposed.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 8.
> >>>>>>>>>>>>>>>>>>>>> A high-level description (without implementation
> >>>>>> details) of
> >>>>>>>>>> how
> >>>>>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
> >>>>>> transactions,
> >>>>>>>>>> state
> >>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be great. Would
> >> be
> >>>>>>>> great
> >>>>>>>>>> if
> >>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>> could also add some sentences about the behavior in
> >>>>>> case of
> >>>>>>>> a
> >>>>>>>>>>>>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>>>> For instance how does a transactional state store
> >>>>>> recover
> >>>>>>>>>> after a
> >>>>>>>>>>>>>>>>>>>>> failure or what happens with the transaction buffer,
> >>>>>> etc.
> >>>>>>>>>> (that
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> >>>>>> WriteBatchWithIndex
> >>>>>>>>>>>>>>>>>> transactions
> >>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
> >>>>>> configured
> >>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>> needs to
> >>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns. Ultimately, any
> >>>>>> approach
> >>>>>>>>>> based
> >>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>> cracking
> >>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in ways it's not
> >>>>>> really
> >>>>>>>>>>>>>> designed
> >>>>>>>>>>>>>>>>>>>>> for is
> >>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen performance or
> >>>>>> consistency
> >>>>>>>>>> issues.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing these early
> >>>>>> commits?
> >>>>>>>>>> While
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>> ideal, I
> >>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to ensure
> >>>>>> consistency
> >>>>>>>>>> whilst
> >>>>>>>>>>>>>>>>>>>>> maintaining
> >>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
> >>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
> >>>>>> complicated, and
> >>>>>>>>>> might
> >>>>>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for users as a
> >>>>>> consequence.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is
> >>>>>> growing a
> >>>>>>>>>> bit
> >>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas for future
> >>>>>>>>>>>>>> improvements, I
> >>>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope down to a design
> >>>>>> that
> >>>>>>>>>>>> achieves
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> most
> >>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized restorations
> >>>>>> during
> >>>>>>>> dirty
> >>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that this KIP
> >>>>>> produces
> >>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>> ultimately
> >>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially if the bulk of
> >>>>>> it is
> >>>>>>>>>>>> internal
> >>>>>>>>>>>>>>>>>>>>>> behaviour.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week trying to
> >>>>>> re-work
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
> >>>>>> newTransaction()
> >>>>>>>>>>>> method,
> >>>>>>>>>>>>>>>>>> such
> >>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of RocksDBStore.
> >>>>>> That
> >>>>>>>>>> way, if
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>>> want to
> >>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the future, like the
> >>>>>> SST
> >>>>>>>> file
> >>>>>>>>>>>>>>>>>> management
> >>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with little/no
> >> API
> >>>>>>>>>> changes.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>

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

Posted by Bruno Cadonna <ca...@apache.org>.
Hi Nick,

I think the KIP is converging!


1.
I am wondering whether it makes sense to write the position file during 
close as we do for the checkpoint file, so that in case the state store 
is replaced with a non-transactional state store the non-transactional 
state store finds the position file. I think, this is not strictly 
needed, but would be a nice behavior instead of just deleting the 
position file.


2.
The test plan does not mention integration tests. Do you not need to 
extend existing ones and add new ones. Also for upgrading and 
downgrading you might need integration and/or system tests.


3.
I think Sophie made a point. Although, IQ reading from uncommitted data 
under EOS might be considered a bug by some people. Thus, your KIP would 
fix a bug rather than changing the intended behavior. However, I also 
see that a feature flag would help users that rely on this buggy 
behavior (at least until AK 4.0).


4.
This is related to the previous point. I assume that the difference 
between READ_COMMITTED and READ_UNCOMMITTED for ALOS is that in the 
former you enable transactions on the state store and in the latter you 
disable them. If my assumption is correct, I think that is an issue. 
Let's assume under ALOS Streams fails over a couple of times more or 
less at the same step in processing after value 3 is added to an 
aggregation but the offset of the corresponding input record was not 
committed. Without transactions disabled, the aggregation value would 
increase by 3 for each failover. With transactions enabled, value 3 
would only be added to the aggregation once when the offset of the input 
record is committed and the transaction finally completes. So the 
content of the state store would change depending on the configuration 
for IQ. IMO, the content of the state store should be independent from 
IQ. Given this issue, I propose to not use transactions with ALOS at 
all. I was a big proponent of using transactions with ALOS, but I 
realized that transactions with ALOS is not as easy as enabling 
transactions on state stores. Another aspect that is problematic is that 
the changelog topic which actually replicates the state store is not 
transactional under ALOS. Thus, it might happen that the state store and 
the changelog differ in their content. All of this is maybe solvable 
somehow, but for the sake of this KIP, I would leave it for the future.


Best,
Bruno



On 10/12/23 10:32 PM, Sophie Blee-Goldman wrote:
> Hey Nick! First of all thanks for taking up this awesome feature, I'm sure
> every single
> Kafka Streams user and dev would agree that it is sorely needed.
> 
> I've just been catching up on the KIP and surrounding discussion, so please
> forgive me
> for any misunderstandings or misinterpretations of the current plan and
> don't hesitate to
> correct me.
> 
> Before I jump in, I just want to say that having seen this drag on for so
> long, my singular
> goal in responding is to help this KIP past a perceived impasse so we can
> finally move on
> to voting and implementing it. Long discussions are to be expected for
> major features like
> this but it's completely on us as the Streams devs to make sure there is an
> end in sight
> for any ongoing discussion.
> 
> With that said, it's my understanding that the KIP as currently proposed is
> just not tenable
> for Kafka Streams, and would prevent some EOS users from upgrading to the
> version it
> first appears in. Given that we can't predict or guarantee whether any of
> the followup KIPs
> would be completed in the same release cycle as this one, we need to make
> sure that the
> feature is either compatible with all current users or else feature-flagged
> so that they may
> opt in/out.
> 
> Therefore, IIUC we need to have either (or both) of these as
> fully-implemented config options:
> 1. default.state.isolation.level
> 2. enable.transactional.state.stores
> 
> This way EOS users for whom read_committed semantics are not viable can
> still upgrade,
> and either use the isolation.level config to leverage the new txn state
> stores without sacrificing
> their application semantics, or else simply keep the transactional state
> stores disabled until we
> are able to fully implement the isolation level configuration at either an
> application or query level.
> 
> Frankly you are the expert here and know much more about the tradeoffs in
> both semantics and
> effort level of implementing one of these configs vs the other. In my
> opinion, either option would
> be fine and I would leave the decision of which one to include in this KIP
> completely up to you.
> I just don't see a way for the KIP to proceed without some variation of the
> above that would allow
> EOS users to opt-out of read_committed.
> 
> (If it's all the same to you, I would recommend 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 that gets released.
> So personally, my choice would be to add the feature flag and leave it off
> by default. If all 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
> 
> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <ni...@gmail.com> wrote:
> 
>> Hi Bruno,
>>
>> Agreed, I can live with that for now.
>>
>> In an effort to keep the scope of this KIP from expanding, I'm leaning
>> towards just providing a configurable default.state.isolation.level and
>> removing IsolationLevel from the StateStoreContext. This would be
>> compatible with adding support for query-time IsolationLevels in the
>> future, whilst providing a way for users to select an isolation level now.
>>
>> The big problem with this, however, is that if a user selects
>> processing.mode
>> = "exactly-once(-v2|-beta)", and default.state.isolation.level =
>> "READ_UNCOMMITTED", we need to guarantee that the data isn't written to
>> disk until commit() is called, but we also need to permit IQ threads to
>> read from the ongoing transaction.
>>
>> A simple solution would be to (temporarily) forbid this combination of
>> configuration, and have default.state.isolation.level automatically switch
>> to READ_COMMITTED when processing.mode is anything other than
>> at-least-once. Do you think this would be acceptable?
>>
>> In a later KIP, we can add support for query-time isolation levels and
>> solve this particular problem there, which would relax this restriction.
>>
>> Regards,
>> Nick
>>
>> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <ca...@apache.org> 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,
>>> Bruno
>>>
>>> On 9/18/23 7:12 PM, Nick Telford wrote:
>>>> 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 <ni...@gmail.com>
>>> wrote:
>>>>
>>>>> Hi everyone,
>>>>>
>>>>> I agree that having IsolationLevel be determined at query-time is the
>>>>> ideal design, but there are a few sticking points:
>>>>>
>>>>> 1.
>>>>> There needs to be some way to communicate the IsolationLevel down to
>> the
>>>>> RocksDBStore itself, so that the query can respect it. Since stores
>> are
>>>>> "layered" in functionality (i.e. ChangeLoggingStore, MeteredStore,
>>> etc.),
>>>>> we need some way to deliver that information to the bottom layer. For
>>> IQv2,
>>>>> we can use the existing State#query() method, but IQv1 has no way to
>> do
>>>>> this.
>>>>>
>>>>> A simple approach, which would potentially open up other options,
>> would
>>> be
>>>>> to add something like: ReadOnlyKeyValueStore<K, V>
>>>>> readOnlyView(IsolationLevel isolationLevel) to ReadOnlyKeyValueStore
>>> (and
>>>>> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
>>>>>
>>>>> 2.
>>>>> As mentioned above, RocksDB WriteBatches are not thread-safe, which
>>> causes
>>>>> a problem if we want to provide READ_UNCOMMITTED Iterators. I also
>> had a
>>>>> look at RocksDB Transactions[1], but they solve a very different
>>> problem,
>>>>> and have the same thread-safety issue.
>>>>>
>>>>> One possible approach that I mentioned is chaining WriteBatches: every
>>>>> time a new Interactive Query is received (i.e. readOnlyView, see
>> above,
>>>>> is called) we "freeze" the existing WriteBatch, and start a new one
>> for
>>> new
>>>>> writes. The Interactive Query queries the "chain" of previous
>>> WriteBatches
>>>>> + the underlying database; while the StreamThread starts writing to
>> the
>>>>> *new* WriteBatch. On-commit, the StreamThread would write *all*
>>>>> WriteBatches in the chain to the database (that have not yet been
>>> written).
>>>>>
>>>>> WriteBatches would be closed/freed only when they have been both
>>>>> committed, and all open Interactive Queries on them have been closed.
>>> This
>>>>> would require some reference counting.
>>>>>
>>>>> Obviously a drawback of this approach is the potential for increased
>>>>> memory usage: if an Interactive Query is long-lived, for example by
>>> doing a
>>>>> full scan over a large database, or even just pausing in the middle of
>>> an
>>>>> iteration, then the existing chain of WriteBatches could be kept
>> around
>>> for
>>>>> a long time, potentially forever.
>>>>>
>>>>> --
>>>>>
>>>>> A.
>>>>> Going off on a tangent, it looks like in addition to supporting
>>>>> READ_COMMITTED queries, we could go further and support
>> REPEATABLE_READ
>>>>> queries (i.e. where subsequent reads to the same key in the same
>>>>> Interactive Query are guaranteed to yield the same value) by making
>> use
>>> of
>>>>> RocksDB Snapshots[2]. These are fairly lightweight, so the performance
>>>>> impact is likely to be negligible, but they do require that the
>>> Interactive
>>>>> Query session can be explicitly closed.
>>>>>
>>>>> This could be achieved if we made the above readOnlyView interface
>> look
>>>>> more like:
>>>>>
>>>>> interface ReadOnlyKeyValueView<K, V> implements
>> ReadOnlyKeyValueStore<K,
>>>>> V>, AutoCloseable {}
>>>>>
>>>>> interface ReadOnlyKeyValueStore<K, V> {
>>>>>       ...
>>>>>       ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel
>>> isolationLevel);
>>>>> }
>>>>>
>>>>> But this would be a breaking change, as existing IQv1 queries are
>>>>> guaranteed to never call store.close(), and therefore these would leak
>>>>> memory under REPEATABLE_READ.
>>>>>
>>>>> B.
>>>>> One thing that's notable: MyRocks states that they support
>>> READ_COMMITTED
>>>>> and REPEATABLE_READ, but they make no mention of
>> READ_UNCOMMITTED[3][4].
>>>>> This could be because doing so is technically difficult/impossible
>> using
>>>>> the primitives available in RocksDB.
>>>>>
>>>>> --
>>>>>
>>>>> Lucas, to address your points:
>>>>>
>>>>> U1.
>>>>> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
>>>>> implementations of StateStore that do not support atomic writes.
>>> Obviously
>>>>> in those cases, the guarantees Kafka Streams provides/expects would be
>>>>> relaxed. Do you think we should require all implementations to support
>>>>> atomic writes?
>>>>>
>>>>> U2.
>>>>> Stores can support multiple IsolationLevels. As we've discussed above,
>>> the
>>>>> ideal scenario would be to specify the IsolationLevel at query-time.
>>>>> Failing that, I think the second-best approach is to define the
>>>>> IsolationLevel for *all* queries based on the processing.mode, which
>> is
>>>>> what the default StateStoreContext#isolationLevel() achieves. Would
>> you
>>>>> prefer an alternative?
>>>>>
>>>>> While the existing implementation is equivalent to READ_UNCOMMITTED,
>>> this
>>>>> can yield unexpected results/errors under EOS, if a transaction is
>>> rolled
>>>>> back. While this would be a change in behaviour for users, it would
>> look
>>>>> more like a bug fix than a breaking change. That said, we *could* make
>>> it
>>>>> configurable, and default to the existing behaviour (READ_UNCOMMITTED)
>>>>> instead of inferring it from the processing.mode?
>>>>>
>>>>> N1, N2.
>>>>> These were only primitives to avoid boxing costs, but since this is
>> not
>>> a
>>>>> performance sensitive area, it should be fine to change if that's
>>> desirable.
>>>>>
>>>>> N3.
>>>>> It's because the store "manages its own offsets", which includes both
>>>>> committing the offset, *and providing it* via getCommittedOffset().
>>>>> Personally, I think "managesOffsets" conveys this best, but I don't
>> mind
>>>>> changing it if the nomenclature is unclear.
>>>>>
>>>>> Sorry for the massive emails/essays!
>>>>> --
>>>>> Nick
>>>>>
>>>>> 1: https://github.com/facebook/rocksdb/wiki/Transactions
>>>>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
>>>>> 3: https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
>>>>> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
>>>>>
>>>>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
>>>>> <lb...@confluent.io.invalid> wrote:
>>>>>
>>>>>> Hi Nick,
>>>>>>
>>>>>> since I last read it in April, the KIP has become much cleaner and
>>>>>> easier to read. Great work!
>>>>>>
>>>>>> It feels to me the last big open point is whether we can implement
>>>>>> isolation level as a query parameter. I understand that there are
>>>>>> implementation concerns, but as Colt says, it would be a great
>>>>>> addition, and would also simplify the migration path for this change.
>>>>>> Is the implementation problem you mentioned caused by the WriteBatch
>>>>>> not having a notion of a snapshot, as the underlying DB iterator
>> does?
>>>>>> In that case, I am not sure a chain of WriteBatches as you propose
>>>>>> would fully solve the problem, but maybe I didn't dig enough into the
>>>>>> details to fully understand it.
>>>>>>
>>>>>> If it's not possible to implement it now, would it be an option to
>>>>>> make sure in this KIP that we do not fully close the door on
>> per-query
>>>>>> isolation levels in the interface, as it may be possible to implement
>>>>>> the missing primitives in RocksDB or Speedb in the future.
>>>>>>
>>>>>> Understanding:
>>>>>>
>>>>>> * U1) Why is it only "SHOULD" for changelogOffsets to be persisted
>>>>>> atomically with the records?
>>>>>> * U2) Don't understand the default implementation of
>> `isolationLevel`.
>>>>>> The isolation level should be a property of the underlying store, and
>>>>>> not be defined by the default config? Existing stores probably don't
>>>>>> guarantee READ_COMMITTED, so the default should be to return
>>>>>> READ_UNCOMMITTED.
>>>>>>
>>>>>> Nits:
>>>>>> * N1) Could `getComittedOffset` use an `OptionalLong` return type, to
>>>>>> avoid the `null`?
>>>>>> * N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
>>>>>> return type, to avoid the `-1`?
>>>>>> * N3) I don't understand why `managesOffsets` uses the 'manage' verb,
>>>>>> whereas all other methods use the "commits" verb. I'd suggest
>>>>>> `commitsOffsets`.
>>>>>>
>>>>>> Either way, it feels this KIP is very close to the finish line, I'm
>>>>>> looking forward to seeing this in production!
>>>>>>
>>>>>> Cheers,
>>>>>> Lucas
>>>>>>
>>>>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <co...@littlehorse.io>
>>> wrote:
>>>>>>>
>>>>>>>> Making IsolationLevel a query-time constraint, rather than linking
>> it
>>>>>> to
>>>>>>> the processing.guarantee.
>>>>>>>
>>>>>>> As I understand it, would this allow even a user of EOS to control
>>>>>> whether
>>>>>>> reading committed or uncommitted records? If so, I am highly in
>> favor
>>> of
>>>>>>> this.
>>>>>>>
>>>>>>> I know that I was one of the early people to point out the current
>>>>>>> shortcoming that IQ reads uncommitted records, but just this
>> morning I
>>>>>>> realized a pattern we use which means that (for certain queries) our
>>>>>> system
>>>>>>> needs to be able to read uncommitted records, which is the current
>>>>>> behavior
>>>>>>> of Kafka Streams in EOS.***
>>>>>>>
>>>>>>> If IsolationLevel being a query-time decision allows for this, then
>>> that
>>>>>>> would be amazing. I would also vote that the default behavior should
>>> be
>>>>>> for
>>>>>>> reading uncommitted records, because it is totally possible for a
>>> valid
>>>>>>> application to depend on that behavior, and breaking it in a minor
>>>>>> release
>>>>>>> might be a bit strong.
>>>>>>>
>>>>>>> *** (Note, for the curious reader....) Our use-case/query pattern
>> is a
>>>>>> bit
>>>>>>> complex, but reading "uncommitted" records is actually safe in our
>>> case
>>>>>>> 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 soon as the processor processes the result. If
>>> they
>>>>>>> can't read uncommitted records, they may receive a "201 - Created"
>>>>>>> response, immediately followed by a "404 - 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 <
>> nick.telford@gmail.com>
>>>>>> wrote:
>>>>>>>
>>>>>>>> Addendum:
>>>>>>>>
>>>>>>>> I think we would also face the same problem with the approach John
>>>>>> outlined
>>>>>>>> earlier (using the record cache as a transaction buffer and
>> flushing
>>>>>> it
>>>>>>>> straight to SST files). This is because the record cache (the
>>>>>> ThreadCache
>>>>>>>> class) is not thread-safe, so every commit would invalidate open IQ
>>>>>>>> Iterators in the same way that RocksDB WriteBatches do.
>>>>>>>> --
>>>>>>>> Nick
>>>>>>>>
>>>>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
>> nick.telford@gmail.com>
>>>>>>>> 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.
>>>>>>>>> 2. Making IsolationLevel a query-time constraint, rather than
>>>>>> linking it
>>>>>>>>> to the processing.guarantee.
>>>>>>>>>
>>>>>>>>> There's a wrinkle that makes this a challenge: Interactive Queries
>>>>>> that
>>>>>>>>> open an Iterator, when using transactions and READ_UNCOMMITTED.
>>>>>>>>> The problem is that under READ_UNCOMMITTED, queries need to be
>> able
>>>>>> to
>>>>>>>>> read records from the currently uncommitted transaction buffer
>>>>>>>>> (WriteBatch). This includes for Iterators, which should iterate
>>>>>> both the
>>>>>>>>> transaction buffer and underlying database (using
>>>>>>>>> WriteBatch#iteratorWithBase()).
>>>>>>>>>
>>>>>>>>> The issue is that when the StreamThread commits, it writes the
>>>>>> current
>>>>>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing
>> the
>>>>>>>>> WriteBatch while an Interactive Query holds an open Iterator on it
>>>>>> will
>>>>>>>>> invalidate the Iterator. Worse, it turns out that Iterators over a
>>>>>>>>> WriteBatch become invalidated not just when the WriteBatch is
>>>>>> cleared,
>>>>>>>> but
>>>>>>>>> also when the Iterators' current key receives a new write.
>>>>>>>>>
>>>>>>>>> Now that I'm writing this, I remember that this is the major
>> reason
>>>>>> that
>>>>>>>> I
>>>>>>>>> switched the original design from having a query-time
>>>>>> IsolationLevel to
>>>>>>>>> having the IsolationLevel linked to the transactionality of the
>>>>>> stores
>>>>>>>>> themselves.
>>>>>>>>>
>>>>>>>>> It *might* be possible to resolve this, by having a "chain" of
>>>>>>>>> WriteBatches, with the StreamThread switching to a new WriteBatch
>>>>>>>> whenever
>>>>>>>>> a new Interactive Query attempts to read from the database, but
>> that
>>>>>>>> could
>>>>>>>>> cause some performance problems/memory pressure when subjected to
>> a
>>>>>> high
>>>>>>>>> Interactive Query load. It would also reduce the efficiency of
>>>>>>>> WriteBatches
>>>>>>>>> on-commit, as we'd have to write N WriteBatches, where N is the
>>>>>> number of
>>>>>>>>> Interactive Queries since the last commit.
>>>>>>>>>
>>>>>>>>> I realise this is getting into the weeds of the implementation,
>> and
>>>>>> you'd
>>>>>>>>> rather we focus on the API for now, but I think it's important to
>>>>>>>> consider
>>>>>>>>> how to implement the desired API, in case we come up with an API
>>>>>> that
>>>>>>>>> cannot be implemented efficiently, or even at all!
>>>>>>>>>
>>>>>>>>> Thoughts?
>>>>>>>>> --
>>>>>>>>> Nick
>>>>>>>>>
>>>>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <ca...@apache.org>
>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Nick,
>>>>>>>>>>
>>>>>>>>>> 6.
>>>>>>>>>> Of course, you are right! My bad!
>>>>>>>>>> Wiping out the state in the downgrading case is fine.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 3a.
>>>>>>>>>> Focus on the public facing changes for the KIP. We will manage 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.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> Bruno
>>>>>>>>>>
>>>>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>
>>>>>>>>>>> Thanks for getting back to me!
>>>>>>>>>>>
>>>>>>>>>>> 2.
>>>>>>>>>>> The fact that implementations can always track estimated memory
>>>>>> usage
>>>>>>>> in
>>>>>>>>>>> the wrapper is a good point. I can remove -1 as an option, and
>>>>>> I'll
>>>>>>>>>> clarify
>>>>>>>>>>> the JavaDoc that 0 is not just for non-transactional stores,
>>>>>> which is
>>>>>>>>>>> currently misleading.
>>>>>>>>>>>
>>>>>>>>>>> 6.
>>>>>>>>>>> The problem with catching the exception in the downgrade process
>>>>>> is
>>>>>>>> that
>>>>>>>>>>> would require new code in the Kafka version being downgraded to.
>>>>>> Since
>>>>>>>>>>> users could conceivably downgrade to almost *any* older version
>>>>>> of
>>>>>>>> Kafka
>>>>>>>>>>> Streams, I'm not sure how we could add that code?
>>>>>>>>>>> The only way I can think of doing it would be to provide a
>>>>>> dedicated
>>>>>>>>>>> downgrade tool, that goes through every local store and removes
>>>>>> the
>>>>>>>>>>> offsets column families. But that seems like an unnecessary
>>>>>> amount of
>>>>>>>>>> extra
>>>>>>>>>>> code to maintain just to handle a somewhat niche situation, when
>>>>>> the
>>>>>>>>>>> alternative (automatically wipe and restore stores) should be
>>>>>>>>>> acceptable.
>>>>>>>>>>>
>>>>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've requested.
>>>>>>>>>>>
>>>>>>>>>>> 3a.
>>>>>>>>>>> I agree that IsolationLevel makes more sense at query-time, and
>> I
>>>>>>>>>> actually
>>>>>>>>>>> initially attempted to place the IsolationLevel at query-time,
>>>>>> but I
>>>>>>>> ran
>>>>>>>>>>> into some problems:
>>>>>>>>>>> - The key issue is that, under ALOS we're not staging writes in
>>>>>>>>>>> transactions, so can't perform writes at the READ_COMMITTED
>>>>>> isolation
>>>>>>>>>>> level. However, this may be addressed if we decide to *always*
>>>>>> use
>>>>>>>>>>> transactions as discussed under 3b.
>>>>>>>>>>> - IQv1 and IQv2 have quite different implementations. I remember
>>>>>>>> having
>>>>>>>>>>> some difficulty understanding the IQv1 internals, which made it
>>>>>>>>>> difficult
>>>>>>>>>>> to determine what needed to be changed. However, I *think* this
>>>>>> can be
>>>>>>>>>>> addressed for both implementations by wrapping the RocksDBStore
>>>>>> in an
>>>>>>>>>>> IsolationLevel-dependent wrapper, that overrides read methods
>>>>>> (get,
>>>>>>>>>> etc.)
>>>>>>>>>>> to either read directly from the database or from the ongoing
>>>>>>>>>> transaction.
>>>>>>>>>>> But IQv1 might still be difficult.
>>>>>>>>>>> - If IsolationLevel becomes a query constraint, then all other
>>>>>>>>>> StateStores
>>>>>>>>>>> will need to respect it, including the in-memory stores. This
>>>>>> would
>>>>>>>>>> require
>>>>>>>>>>> us to adapt in-memory stores to stage their writes so they can
>> be
>>>>>>>>>> isolated
>>>>>>>>>>> from READ_COMMITTTED queries. It would also become an important
>>>>>>>>>>> consideration for third-party stores on upgrade, as without
>>>>>> changes,
>>>>>>>>>> they
>>>>>>>>>>> would not support READ_COMMITTED queries correctly.
>>>>>>>>>>>
>>>>>>>>>>> Ultimately, I may need some help making the necessary change to
>>>>>> IQv1
>>>>>>>> to
>>>>>>>>>>> support this, but I don't think it's fundamentally impossible,
>>>>>> if we
>>>>>>>>>> want
>>>>>>>>>>> to pursue this route.
>>>>>>>>>>>
>>>>>>>>>>> 3b.
>>>>>>>>>>> The main reason I chose to keep ALOS un-transactional was to
>>>>>> minimize
>>>>>>>>>>> behavioural change for most users (I believe most Streams users
>>>>>> use
>>>>>>>> the
>>>>>>>>>>> default configuration, which is ALOS). That said, it's clear
>>>>>> that if
>>>>>>>>>> ALOS
>>>>>>>>>>> also used transactional stores, the only change in behaviour
>>>>>> would be
>>>>>>>>>> that
>>>>>>>>>>> it would become *more correct*, which could be considered a "bug
>>>>>> fix"
>>>>>>>> by
>>>>>>>>>>> users, rather than a change they need to handle.
>>>>>>>>>>>
>>>>>>>>>>> I believe that performance using transactions (aka. RocksDB
>>>>>>>>>> WriteBatches)
>>>>>>>>>>> should actually be *better* than the un-batched write-path that
>>>>>> is
>>>>>>>>>>> currently used[1]. The only "performance" consideration will be
>>>>>> the
>>>>>>>>>>> increased memory usage that transactions require. Given the
>>>>>>>> mitigations
>>>>>>>>>> for
>>>>>>>>>>> this memory that we have in place, I would expect that this is
>>>>>> not a
>>>>>>>>>>> problem for most users.
>>>>>>>>>>>
>>>>>>>>>>> If we're happy to do so, we can make ALOS also use transactions.
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>> Nick
>>>>>>>>>>>
>>>>>>>>>>> Link 1:
>>>>>>>>>>>
>>>>>>>>
>>>>>>
>>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>
>>>>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
>> cadonna@apache.org
>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for the updates and sorry for the delay on my side!
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 1.
>>>>>>>>>>>> Making the default implementation for flush() a no-op sounds
>>>>>> good to
>>>>>>>>>> me.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 2.
>>>>>>>>>>>> I think what was bugging me here is that a third-party state
>>>>>> store
>>>>>>>>>> needs
>>>>>>>>>>>> to implement the state store interface. That means they need to
>>>>>>>>>>>> implement a wrapper around the actual state store as we do for
>>>>>>>> RocksDB
>>>>>>>>>>>> with RocksDBStore. So, a third-party state store can always
>>>>>> estimate
>>>>>>>>>> the
>>>>>>>>>>>> uncommitted bytes, if it wants, because the wrapper can record
>>>>>> the
>>>>>>>>>> added
>>>>>>>>>>>> bytes.
>>>>>>>>>>>> One case I can think of where returning -1 makes sense is when
>>>>>>>> Streams
>>>>>>>>>>>> does not need to estimate the size of the write batch and
>>>>>> trigger
>>>>>>>>>>>> extraordinary commits, because the third-party state store
>>>>>> takes care
>>>>>>>>>> of
>>>>>>>>>>>> memory. But in that case the method could also just return 0.
>>>>>> Even
>>>>>>>> that
>>>>>>>>>>>> case would be better solved with a method that returns whether
>>>>>> the
>>>>>>>>>> state
>>>>>>>>>>>> store manages itself the memory used for uncommitted bytes or
>>>>>> not.
>>>>>>>>>>>> Said that, I am fine with keeping the -1 return value, I was
>>>>>> just
>>>>>>>>>>>> wondering when and if it will be used.
>>>>>>>>>>>>
>>>>>>>>>>>> Regarding returning 0 for transactional state stores when the
>>>>>> batch
>>>>>>>> is
>>>>>>>>>>>> empty, I was just wondering because you explicitly stated
>>>>>>>>>>>>
>>>>>>>>>>>> "or {@code 0} if this StateStore does not support
>> transactions."
>>>>>>>>>>>>
>>>>>>>>>>>> So it seemed to me returning 0 could only happen for
>>>>>>>> non-transactional
>>>>>>>>>>>> state stores.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 3.
>>>>>>>>>>>>
>>>>>>>>>>>> a) What do you think if we move the isolation level to IQ (v1
>>>>>> and
>>>>>>>> v2)?
>>>>>>>>>>>> In the end this is the only component that really needs to
>>>>>> specify
>>>>>>>> the
>>>>>>>>>>>> isolation level. It is similar to the Kafka consumer that can
>>>>>> choose
>>>>>>>>>>>> with what isolation level to read the input topic.
>>>>>>>>>>>> For IQv1 the isolation level should go into
>>>>>> StoreQueryParameters. For
>>>>>>>>>>>> IQv2, I would add it to the Query interface.
>>>>>>>>>>>>
>>>>>>>>>>>> b) Point a) raises the question what should happen during
>>>>>>>> at-least-once
>>>>>>>>>>>> processing when the state store does not use transactions? John
>>>>>> in
>>>>>>>> the
>>>>>>>>>>>> past proposed to also use transactions on state stores for
>>>>>>>>>>>> at-least-once. I like that idea, because it avoids aggregating
>>>>>> the
>>>>>>>> same
>>>>>>>>>>>> records over and over again in the case of a failure. We had a
>>>>>> case
>>>>>>>> in
>>>>>>>>>>>> the past where a Streams applications in at-least-once mode was
>>>>>>>> failing
>>>>>>>>>>>> continuously for some reasons I do not remember before
>>>>>> committing the
>>>>>>>>>>>> offsets. After each failover, the app aggregated again and
>>>>>> again the
>>>>>>>>>>>> same records. Of course the aggregate increased to very wrong
>>>>>> values
>>>>>>>>>>>> just because of the failover. With transactions on the state
>>>>>> stores
>>>>>>>> we
>>>>>>>>>>>> could have avoided this. The app would have output the same
>>>>>> aggregate
>>>>>>>>>>>> multiple times (i.e., after each failover) but at least the
>>>>>> value of
>>>>>>>>>> the
>>>>>>>>>>>> aggregate would not depend on the number of failovers.
>>>>>> Outputting the
>>>>>>>>>>>> same aggregate multiple times would be incorrect under
>>>>>> exactly-once
>>>>>>>> but
>>>>>>>>>>>> it is OK for at-least-once.
>>>>>>>>>>>> If it makes sense to add a config to turn on and off
>>>>>> transactions on
>>>>>>>>>>>> state stores under at-least-once or just use transactions in
>>>>>> any case
>>>>>>>>>> is
>>>>>>>>>>>> a question we should also discuss in this KIP. It depends a bit
>>>>>> on
>>>>>>>> the
>>>>>>>>>>>> performance trade-off. Maybe to be safe, I would add a config.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 4.
>>>>>>>>>>>> Your points are all valid. I tend to say to keep the metrics
>>>>>> around
>>>>>>>>>>>> flush() until we remove flush() completely from the interface.
>>>>>> Calls
>>>>>>>> to
>>>>>>>>>>>> flush() might still exist since existing processors might still
>>>>>> call
>>>>>>>>>>>> flush() explicitly as you mentioned in 1). For sure, we need to
>>>>>>>>>> document
>>>>>>>>>>>> how the metrics change due to the transactions in the upgrade
>>>>>> notes.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 5.
>>>>>>>>>>>> I see. Then you should describe how the .position files are
>>>>>> handled
>>>>>>>> in
>>>>>>>>>>>> a dedicated section of the KIP or incorporate the description
>>>>>> in the
>>>>>>>>>>>> "Atomic Checkpointing" section instead of only mentioning it in
>>>>>> the
>>>>>>>>>>>> "Compatibility, 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:
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>
>> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
>>>>>>>>>>>>
>>>>>>>>>>>> It is a bit brittle, but it works.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Best,
>>>>>>>>>>>> Bruno
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
>>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for taking the time to review the KIP. I'm back from
>>>>>> leave
>>>>>>>> now
>>>>>>>>>> and
>>>>>>>>>>>>> intend to move this forwards as quickly as I can.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Addressing your points:
>>>>>>>>>>>>>
>>>>>>>>>>>>> 1.
>>>>>>>>>>>>> Because flush() is part of the StateStore API, it's exposed to
>>>>>>>> custom
>>>>>>>>>>>>> Processors, which might be making calls to flush(). This was
>>>>>>>> actually
>>>>>>>>>> the
>>>>>>>>>>>>> case in a few integration tests.
>>>>>>>>>>>>> To maintain as much compatibility as possible, I'd prefer not
>>>>>> to
>>>>>>>> make
>>>>>>>>>>>> this
>>>>>>>>>>>>> an UnsupportedOperationException, as it will cause previously
>>>>>>>> working
>>>>>>>>>>>>> Processors to start throwing exceptions at runtime.
>>>>>>>>>>>>> I agree that it doesn't make sense for it to proxy commit(),
>>>>>> though,
>>>>>>>>>> as
>>>>>>>>>>>>> that would cause it to violate the "StateStores commit only
>>>>>> when the
>>>>>>>>>> Task
>>>>>>>>>>>>> commits" rule.
>>>>>>>>>>>>> Instead, I think we should make this a no-op. That way,
>>>>>> existing
>>>>>>>> user
>>>>>>>>>>>>> Processors will continue to work as-before, without violation
>>>>>> of
>>>>>>>> store
>>>>>>>>>>>>> consistency that would be caused by premature flush/commit of
>>>>>>>>>> StateStore
>>>>>>>>>>>>> data to disk.
>>>>>>>>>>>>> What do you think?
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2.
>>>>>>>>>>>>> As stated in the JavaDoc, when a StateStore implementation is
>>>>>>>>>>>>> transactional, but is unable to estimate the uncommitted
>> memory
>>>>>>>> usage,
>>>>>>>>>>>> the
>>>>>>>>>>>>> method will return -1.
>>>>>>>>>>>>> The intention here is to permit third-party implementations
>>>>>> that may
>>>>>>>>>> not
>>>>>>>>>>>> be
>>>>>>>>>>>>> able to estimate memory usage.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Yes, it will be 0 when nothing has been written to the store
>>>>>> yet. I
>>>>>>>>>>>> thought
>>>>>>>>>>>>> that was implied by "This method will return an approximation
>>>>>> of the
>>>>>>>>>>>> memory
>>>>>>>>>>>>> would be freed by the next call to {@link #commit(Map)}" and
>>>>>>>> "@return
>>>>>>>>>> The
>>>>>>>>>>>>> approximate size of all records awaiting {@link
>> #commit(Map)}",
>>>>>>>>>> however,
>>>>>>>>>>>> I
>>>>>>>>>>>>> can add it explicitly to the JavaDoc if you think this is
>>>>>> unclear?
>>>>>>>>>>>>>
>>>>>>>>>>>>> 3.
>>>>>>>>>>>>> I realise this is probably the most contentious point in my
>>>>>> design,
>>>>>>>>>> and
>>>>>>>>>>>> I'm
>>>>>>>>>>>>> open to changing it if I'm unable to convince you of the
>>>>>> benefits.
>>>>>>>>>>>>> Nevertheless, here's my argument:
>>>>>>>>>>>>> The Interactive Query (IQ) API(s) are directly provided
>>>>>> StateStores
>>>>>>>> to
>>>>>>>>>>>>> query, and it may be important for users to programmatically
>>>>>> know
>>>>>>>>>> which
>>>>>>>>>>>>> mode the StateStore is operating under. If we simply provide
>> an
>>>>>>>>>>>>> "eosEnabled" boolean (as used throughout the internal streams
>>>>>>>>>> engine), or
>>>>>>>>>>>>> similar, then users will need to understand the operation and
>>>>>>>>>>>> consequences
>>>>>>>>>>>>> of each available processing mode and how it pertains to their
>>>>>>>>>>>> StateStore.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Interactive Query users aren't the only people that care about
>>>>>> the
>>>>>>>>>>>>> processing.mode/IsolationLevel of a StateStore: implementers
>> of
>>>>>>>> custom
>>>>>>>>>>>>> StateStores also need to understand the behaviour expected of
>>>>>> their
>>>>>>>>>>>>> implementation. KIP-892 introduces some assumptions into the
>>>>>> Streams
>>>>>>>>>>>> Engine
>>>>>>>>>>>>> about how StateStores operate under each processing mode, and
>>>>>> it's
>>>>>>>>>>>>> important that custom implementations adhere to those
>>>>>> assumptions in
>>>>>>>>>>>> order
>>>>>>>>>>>>> to maintain the consistency guarantees.
>>>>>>>>>>>>>
>>>>>>>>>>>>> IsolationLevels provide a high-level contract on the behaviour
>>>>>> of
>>>>>>>> the
>>>>>>>>>>>>> StateStore: a user knows that under READ_COMMITTED, they will
>>>>>> see
>>>>>>>>>> writes
>>>>>>>>>>>>> only after the Task has committed, and under READ_UNCOMMITTED
>>>>>> they
>>>>>>>>>> will
>>>>>>>>>>>> see
>>>>>>>>>>>>> writes immediately. No understanding of the details of each
>>>>>>>>>>>> processing.mode
>>>>>>>>>>>>> is required, either for IQ users or StateStore implementers.
>>>>>>>>>>>>>
>>>>>>>>>>>>> An argument can be made that these contractual guarantees can
>>>>>> simply
>>>>>>>>>> be
>>>>>>>>>>>>> documented for the processing.mode (i.e. that exactly-once and
>>>>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and at-least-once
>>>>>> behaves
>>>>>>>>>> like
>>>>>>>>>>>>> READ_UNCOMMITTED), but there are several small issues with
>>>>>> this I'd
>>>>>>>>>>>> prefer
>>>>>>>>>>>>> to avoid:
>>>>>>>>>>>>>
>>>>>>>>>>>>>        - Where would we document these contracts, in a way that
>>>>>> is
>>>>>>>>>> difficult
>>>>>>>>>>>>>        for users/implementers to miss/ignore?
>>>>>>>>>>>>>        - It's not clear to users that the processing mode is
>>>>>>>>>> communicating
>>>>>>>>>>>>>        an expectation of read isolation, unless they read the
>>>>>>>>>>>> documentation. Users
>>>>>>>>>>>>>        rarely consult documentation unless they feel they need
>>>>>> to, so
>>>>>>>>>> it's
>>>>>>>>>>>> likely
>>>>>>>>>>>>>        this detail would get missed by many users.
>>>>>>>>>>>>>        - It tightly couples processing modes to read isolation.
>>>>>> Adding
>>>>>>>>>> new
>>>>>>>>>>>>>        processing modes, or changing the read isolation of
>>>>>> existing
>>>>>>>>>>>> processing
>>>>>>>>>>>>>        modes would be difficult/impossible.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Ultimately, the cost of introducing IsolationLevels is just a
>>>>>> single
>>>>>>>>>>>>> method, since we re-use the existing IsolationLevel enum from
>>>>>> Kafka.
>>>>>>>>>> This
>>>>>>>>>>>>> gives us a clear place to document the contractual guarantees
>>>>>>>> expected
>>>>>>>>>>>>> of/provided by StateStores, that is accessible both by the
>>>>>>>> StateStore
>>>>>>>>>>>>> itself, and by IQ users.
>>>>>>>>>>>>>
>>>>>>>>>>>>> (Writing this I've just realised that the StateStore and IQ
>>>>>> APIs
>>>>>>>>>> actually
>>>>>>>>>>>>> don't provide access to StateStoreContext that IQ users would
>>>>>> have
>>>>>>>>>> direct
>>>>>>>>>>>>> access to... Perhaps StateStore should expose isolationLevel()
>>>>>>>> itself
>>>>>>>>>>>> too?)
>>>>>>>>>>>>>
>>>>>>>>>>>>> 4.
>>>>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics in-place
>>>>>> either, as
>>>>>>>>>> it's a
>>>>>>>>>>>>> backwards incompatible change. My concern is that, if we leave
>>>>>> the
>>>>>>>>>>>> existing
>>>>>>>>>>>>> "flush" metrics in place, they will be confusing to users.
>>>>>> Right
>>>>>>>> now,
>>>>>>>>>>>>> "flush" metrics record explicit flushes to disk, but under
>>>>>> KIP-892,
>>>>>>>>>> even
>>>>>>>>>>>> a
>>>>>>>>>>>>> commit() will not explicitly flush data to disk - RocksDB will
>>>>>>>> decide
>>>>>>>>>> on
>>>>>>>>>>>>> when to flush memtables to disk itself.
>>>>>>>>>>>>>
>>>>>>>>>>>>> If we keep the existing "flush" metrics, we'd have two
>> options,
>>>>>>>> which
>>>>>>>>>>>> both
>>>>>>>>>>>>> seem pretty bad to me:
>>>>>>>>>>>>>
>>>>>>>>>>>>>        1. Have them record calls to commit(), which would be
>>>>>>>>>> misleading, as
>>>>>>>>>>>>>        data is no longer explicitly "flushed" to disk by this
>>>>>> call.
>>>>>>>>>>>>>        2. Have them record nothing at all, which is equivalent
>> to
>>>>>>>>>> removing
>>>>>>>>>>>> the
>>>>>>>>>>>>>        metrics, except that users will see the metric still
>>>>>> exists and
>>>>>>>>>> so
>>>>>>>>>>>> assume
>>>>>>>>>>>>>        that the metric is correct, and that there's a problem
>>>>>> with
>>>>>>>> their
>>>>>>>>>>>> system
>>>>>>>>>>>>>        when there isn't.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I agree that removing them is also a bad solution, and I'd
>>>>>> like some
>>>>>>>>>>>>> guidance on the best path forward here.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 5.
>>>>>>>>>>>>> Position files are updated on every write to a StateStore.
>>>>>> Since our
>>>>>>>>>>>> writes
>>>>>>>>>>>>> are now buffered until commit(), we can't update the Position
>>>>>> file
>>>>>>>>>> until
>>>>>>>>>>>>> commit() has been called, otherwise it would be inconsistent
>>>>>> with
>>>>>>>> the
>>>>>>>>>>>> data
>>>>>>>>>>>>> in the event of a rollback. Consequently, we need to manage
>>>>>> these
>>>>>>>>>> offsets
>>>>>>>>>>>>> the same way we manage the checkpoint offsets, and ensure
>>>>>> they're
>>>>>>>> only
>>>>>>>>>>>>> written on commit().
>>>>>>>>>>>>>
>>>>>>>>>>>>> 6.
>>>>>>>>>>>>> Agreed, although I'm not exactly sure yet what tests to write.
>>>>>> How
>>>>>>>>>>>> explicit
>>>>>>>>>>>>> do we need to be here in the KIP?
>>>>>>>>>>>>>
>>>>>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be seamless,
>>>>>> and we
>>>>>>>>>>>> should
>>>>>>>>>>>>> definitely add some tests around that. Downgrade, it
>>>>>> transpires,
>>>>>>>> isn't
>>>>>>>>>>>>> currently possible, as the extra column family for offset
>>>>>> storage is
>>>>>>>>>>>>> incompatible with the pre-KIP-892 implementation: when you
>>>>>> open a
>>>>>>>>>> RocksDB
>>>>>>>>>>>>> database, you must open all available column families or
>>>>>> receive an
>>>>>>>>>>>> error.
>>>>>>>>>>>>> What currently happens on downgrade is that it attempts to
>>>>>> open the
>>>>>>>>>>>> store,
>>>>>>>>>>>>> throws an error about the offsets column family not being
>>>>>> opened,
>>>>>>>>>> which
>>>>>>>>>>>>> triggers a wipe and rebuild of the Task. Given that downgrades
>>>>>>>> should
>>>>>>>>>> be
>>>>>>>>>>>>> uncommon, I think this is acceptable behaviour, as the
>>>>>> end-state is
>>>>>>>>>>>>> consistent, even if it results in an undesirable state
>> restore.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Should I document the upgrade/downgrade behaviour explicitly
>>>>>> in the
>>>>>>>>>> KIP?
>>>>>>>>>>>>>
>>>>>>>>>>>>> --
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
>>>>>> cadonna@apache.org>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Nick!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for the updates!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>> Why does StateStore#flush() default to
>>>>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
>>>>>>>>>>>>>> Since calls to flush() will not exist anymore after this KIP
>>>>>> is
>>>>>>>>>>>>>> released, I would rather throw an unsupported operation
>>>>>> exception
>>>>>>>> by
>>>>>>>>>>>>>> default.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>> When would a state store return -1 from
>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while being
>>>>>>>>>> transactional?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Wouldn't StateStore#approximateNumUncommittedBytes() also
>>>>>> return 0
>>>>>>>> if
>>>>>>>>>>>>>> the state store is transactional but nothing has been written
>>>>>> to
>>>>>>>> the
>>>>>>>>>>>>>> state store yet?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>> Sorry for bringing this up again. Does this KIP really need
>> to
>>>>>>>>>> introduce
>>>>>>>>>>>>>> StateStoreContext#isolationLevel()? StateStoreContext has
>>>>>> already
>>>>>>>>>>>>>> appConfigs() which basically exposes the same information,
>>>>>> i.e., if
>>>>>>>>>> EOS
>>>>>>>>>>>>>> is enabled or not.
>>>>>>>>>>>>>> In one of your previous e-mails you wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> "My idea was to try to keep the StateStore interface as
>>>>>> loosely
>>>>>>>>>> coupled
>>>>>>>>>>>>>> from the Streams engine as possible, to give implementers
>> more
>>>>>>>>>> freedom,
>>>>>>>>>>>>>> and reduce the amount of internal knowledge required."
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> While I understand the intent, I doubt that it decreases the
>>>>>>>>>> coupling of
>>>>>>>>>>>>>> a StateStore interface and the Streams engine. READ_COMMITTED
>>>>>> only
>>>>>>>>>>>>>> applies to IQ but not to reads by processors. Thus,
>>>>>> implementers
>>>>>>>>>> need to
>>>>>>>>>>>>>> understand how Streams accesses the state stores.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I would like to hear what others think about this.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 4.
>>>>>>>>>>>>>> Great exposing new metrics for transactional state stores!
>>>>>>>> However, I
>>>>>>>>>>>>>> would prefer to add new metrics and deprecate (in the docs)
>>>>>> the 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 also need to adapt/add integration tests
>>>>>> besides
>>>>>>>> unit
>>>>>>>>>>>>>> tests. Additionally, we probably need integration or system
>>>>>> tests
>>>>>>>> to
>>>>>>>>>>>>>> verify that upgrades and downgrades between transactional and
>>>>>>>>>>>>>> non-transactional state stores work as expected.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
>>>>>>>>>>>>>>> One more thing: I noted John's suggestion in the KIP, under
>>>>>>>>>> "Rejected
>>>>>>>>>>>>>>> Alternatives". I still think it's an idea worth pursuing,
>>>>>> but I
>>>>>>>>>> believe
>>>>>>>>>>>>>>> that it's out of the scope of this KIP, because it solves a
>>>>>>>>>> different
>>>>>>>>>>>> set
>>>>>>>>>>>>>>> of problems to this KIP, and the scope of this one has
>>>>>> already
>>>>>>>> grown
>>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>> large!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I've updated the KIP (
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>> )
>>>>>>>>>>>>>>>> with the latest changes; mostly bringing back "Atomic
>>>>>>>>>> Checkpointing"
>>>>>>>>>>>>>> (for
>>>>>>>>>>>>>>>> what feels like the 10th time!). I think the one thing
>>>>>> missing is
>>>>>>>>>> some
>>>>>>>>>>>>>>>> changes to metrics (notably the store "flush" metrics will
>>>>>> need
>>>>>>>> to
>>>>>>>>>> be
>>>>>>>>>>>>>>>> renamed to "commit").
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The reason I brought back Atomic Checkpointing was to
>>>>>> decouple
>>>>>>>>>> store
>>>>>>>>>>>>>> flush
>>>>>>>>>>>>>>>> from store commit. This is important, because with
>>>>>> Transactional
>>>>>>>>>>>>>>>> StateStores, we now need to call "flush" on *every* Task
>>>>>> commit,
>>>>>>>>>> and
>>>>>>>>>>>> not
>>>>>>>>>>>>>>>> just when the StateStore is closing, otherwise our
>>>>>> transaction
>>>>>>>>>> buffer
>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>> never be written and persisted, instead growing unbounded!
>> I
>>>>>>>>>>>>>> experimented
>>>>>>>>>>>>>>>> with some simple solutions, like forcing a store flush
>>>>>> whenever
>>>>>>>> the
>>>>>>>>>>>>>>>> transaction buffer was likely to exceed its configured
>>>>>> size, but
>>>>>>>>>> this
>>>>>>>>>>>>>> was
>>>>>>>>>>>>>>>> brittle: it prevented the transaction buffer from being
>>>>>>>> configured
>>>>>>>>>> to
>>>>>>>>>>>> be
>>>>>>>>>>>>>>>> unbounded, and it still would have required explicit
>>>>>> flushes of
>>>>>>>>>>>> RocksDB,
>>>>>>>>>>>>>>>> yielding sub-optimal performance and memory utilization.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right" way to
>>>>>> resolve
>>>>>>>> this
>>>>>>>>>>>>>>>> problem. By ensuring that the changelog offsets that
>>>>>> correspond
>>>>>>>> to
>>>>>>>>>> the
>>>>>>>>>>>>>> most
>>>>>>>>>>>>>>>> recently written records are always atomically written to
>>>>>> the
>>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>>> (by writing them to the same transaction buffer), we can
>>>>>> avoid
>>>>>>>>>>>> forcibly
>>>>>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting RocksDB
>>>>>> flush
>>>>>>>> them
>>>>>>>>>>>> only
>>>>>>>>>>>>>>>> when necessary, without losing any of our consistency
>>>>>> guarantees.
>>>>>>>>>> See
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> updated KIP for more info.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I have fully implemented these changes, although I'm still
>>>>>> not
>>>>>>>>>>>> entirely
>>>>>>>>>>>>>>>> happy with the implementation for segmented StateStores, so
>>>>>> I
>>>>>>>> plan
>>>>>>>>>> to
>>>>>>>>>>>>>>>> refactor that. Despite that, all tests pass. If you'd like
>>>>>> to try
>>>>>>>>>> out
>>>>>>>>>>>> or
>>>>>>>>>>>>>>>> review this highly experimental and incomplete branch, it's
>>>>>>>>>> available
>>>>>>>>>>>>>> here:
>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0.
>>>>>> Note:
>>>>>>>>>> it's
>>>>>>>>>>>>>> built
>>>>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base to build
>>>>>> and test
>>>>>>>>>> it
>>>>>>>>>>>> on,
>>>>>>>>>>>>>>>> and to enable easy apples-to-apples comparisons in a live
>>>>>>>>>>>> environment. I
>>>>>>>>>>>>>>>> plan to rebase it against trunk once it's nearer completion
>>>>>> and
>>>>>>>> has
>>>>>>>>>>>> been
>>>>>>>>>>>>>>>> proven on our main application.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I would really appreciate help in reviewing and testing:
>>>>>>>>>>>>>>>> - Segmented (Versioned, Session and Window) stores
>>>>>>>>>>>>>>>> - Global stores
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> As I do not currently use either of these, so my primary
>>>>>> test
>>>>>>>>>>>>>> environment
>>>>>>>>>>>>>>>> doesn't test these areas.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I'm going on Parental Leave starting next week for a few
>>>>>> weeks,
>>>>>>>> so
>>>>>>>>>>>> will
>>>>>>>>>>>>>>>> not have time to move this forward until late August. That
>>>>>> said,
>>>>>>>>>> your
>>>>>>>>>>>>>>>> feedback is welcome and appreciated, I just won't be able
>> to
>>>>>>>>>> respond
>>>>>>>>>>>> as
>>>>>>>>>>>>>>>> quickly as usual.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Bruno
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ is not
>>>>>> something
>>>>>>>> I
>>>>>>>>>> had
>>>>>>>>>>>>>>>>> considered.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> What about atomically updating the state store from the
>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
>>>>>> (thus,
>>>>>>>>>>>>>> flushing
>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
>>>>>> number of
>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>> intervals?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting that we
>>>>>> add an
>>>>>>>>>>>>>> additional
>>>>>>>>>>>>>>>>> config for the max number of commit intervals between
>>>>>>>> checkpoints?
>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>> way, we would checkpoint *either* when the transaction
>>>>>> buffers
>>>>>>>> are
>>>>>>>>>>>>>> nearly
>>>>>>>>>>>>>>>>> full, *OR* whenever a certain number of commit intervals
>>>>>> have
>>>>>>>>>>>> elapsed,
>>>>>>>>>>>>>>>>> whichever comes first?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> That certainly seems reasonable, although this re-ignites
>>>>>> an
>>>>>>>>>> earlier
>>>>>>>>>>>>>>>>> debate about whether a config should be measured in
>>>>>> "number of
>>>>>>>>>> commit
>>>>>>>>>>>>>>>>> intervals", instead of just an absolute time.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> FWIW, I realised that this issue is the reason I was
>>>>>> pursuing
>>>>>>>> the
>>>>>>>>>>>>>> Atomic
>>>>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush from
>>>>>> checkpointing,
>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>> enables us to just checkpoint on every commit without any
>>>>>>>>>> performance
>>>>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the "best"
>>>>>> solution,
>>>>>>>>>> but
>>>>>>>>>>>>>> I'm not
>>>>>>>>>>>>>>>>> sure if this is enough to bring it back into this KIP.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I'm currently working on moving all the transactional
>> logic
>>>>>>>>>> directly
>>>>>>>>>>>>>> into
>>>>>>>>>>>>>>>>> RocksDBStore itself, which does away with the
>>>>>>>>>>>> StateStore#newTransaction
>>>>>>>>>>>>>>>>> method, and reduces the number of new classes introduced,
>>>>>>>>>>>> significantly
>>>>>>>>>>>>>>>>> reducing the complexity. If it works, and the complexity
>> is
>>>>>>>>>>>> drastically
>>>>>>>>>>>>>>>>> reduced, I may try bringing back Atomic Checkpoints into
>>>>>> this
>>>>>>>> KIP.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
>>>>>> cadonna@apache.org>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> As far as I understand, you want to atomically update the
>>>>>> state
>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>> from the transaction buffer, flush the memtable of a
>> state
>>>>>>>> store
>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> write the checkpoint not after the commit time elapsed
>> but
>>>>>>>> after
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> transaction buffer reached a size that would lead to
>>>>>> exceeding
>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes before the next
>>>>>> commit
>>>>>>>>>>>>>> interval
>>>>>>>>>>>>>>>>>> ends.
>>>>>>>>>>>>>>>>>> That means, the Kafka transaction would commit every
>>>>>> commit
>>>>>>>>>> interval
>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>>> the state store will only be atomically updated roughly
>>>>>> every
>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ
>>>>>> would
>>>>>>>>>> then
>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>> see new data roughly every
>>>>>>>>>> statestore.transaction.buffer.max.bytes.
>>>>>>>>>>>>>>>>>> After a failure the state store needs to restore up to
>>>>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Is this correct?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> What about atomically updating the state store from the
>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
>>>>>> (thus,
>>>>>>>>>>>>>> flushing
>>>>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
>>>>>> number of
>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>> intervals? In such a way, we would have the same delay
>> for
>>>>>>>>>> records
>>>>>>>>>>>>>>>>>> appearing in output topics and IQ because both would
>>>>>> appear
>>>>>>>> when
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> Kafka transaction is committed. However, after a failure
>>>>>> the
>>>>>>>>>> state
>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>> still needs to restore up to
>>>>>>>>>> statestore.transaction.buffer.max.bytes
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> it might restore data that is already in the state store
>>>>>>>> because
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> checkpoint lags behind the last stable offset (i.e. the
>>>>>> last
>>>>>>>>>>>> committed
>>>>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring data that is
>>>>>> already
>>>>>>>>>> in
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> state store is idempotent, so eos should not violated.
>>>>>>>>>>>>>>>>>> This solution needs at least one new config to specify
>>>>>> when a
>>>>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>>>>> should be written.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> A small correction to your previous e-mail that does not
>>>>>> change
>>>>>>>>>>>>>> anything
>>>>>>>>>>>>>>>>>> you said: Under alos the default commit interval is 30
>>>>>> seconds,
>>>>>>>>>> not
>>>>>>>>>>>>>> five
>>>>>>>>>>>>>>>>>> seconds.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I've begun performance testing my branch on our staging
>>>>>>>>>>>> environment,
>>>>>>>>>>>>>>>>>>> putting it through its paces in our non-trivial
>>>>>> application.
>>>>>>>> I'm
>>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>> observing the same increased flush rate that we saw the
>>>>>> last
>>>>>>>>>> time
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> attempted to use a version of this KIP, but this time, I
>>>>>>>> think I
>>>>>>>>>>>> know
>>>>>>>>>>>>>>>>>> why.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at
>>>>>> the end
>>>>>>>>>> of
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> Task
>>>>>>>>>>>>>>>>>>> commit process, has the following behaviour:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>          - Under ALOS: checkpoint the state stores. This
>>>>>>>> includes
>>>>>>>>>>>>>>>>>>>          flushing memtables in RocksDB. This is
>> acceptable
>>>>>>>>>> because the
>>>>>>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>>>          commit.interval.ms is 5 seconds, so forcibly
>>>>>> flushing
>>>>>>>>>>>> memtables
>>>>>>>>>>>>>>>>>> every 5
>>>>>>>>>>>>>>>>>>>          seconds is acceptable for most applications.
>>>>>>>>>>>>>>>>>>>          - Under EOS: checkpointing is not done, *unless*
>>>>>> it's
>>>>>>>>>> being
>>>>>>>>>>>>>>>>>> forced, due
>>>>>>>>>>>>>>>>>>>          to e.g. the Task closing or being revoked. This
>>>>>> means
>>>>>>>>>> that
>>>>>>>>>>>> under
>>>>>>>>>>>>>>>>>> normal
>>>>>>>>>>>>>>>>>>>          processing conditions, the state stores will not
>>>>>> be
>>>>>>>>>>>>>> checkpointed,
>>>>>>>>>>>>>>>>>> and will
>>>>>>>>>>>>>>>>>>>          not have memtables flushed at all , unless
>> RocksDB
>>>>>>>>>> decides to
>>>>>>>>>>>>>>>>>> flush them on
>>>>>>>>>>>>>>>>>>>          its own. Checkpointing stores and force-flushing
>>>>>> their
>>>>>>>>>>>> memtables
>>>>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>>>>          done when a Task is being closed.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at
>> least
>>>>>>>> *some*
>>>>>>>>>>>>>> normal
>>>>>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB transaction
>>>>>>>> buffers
>>>>>>>>>> to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> state stores, and to ensure the offsets are synced to
>>>>>> disk to
>>>>>>>>>>>> prevent
>>>>>>>>>>>>>>>>>>> restores from getting out of hand. Consequently, my
>>>>>> current
>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is
>>>>>> far too
>>>>>>>>>>>>>>>>>> frequent.
>>>>>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records, which is a
>>>>>>>> change
>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> flush
>>>>>>>>>>>>>>>>>>> behaviour, potentially causing performance problems for
>>>>>> some
>>>>>>>>>>>>>>>>>> applications.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm currently
>>>>>> leaning
>>>>>>>>>>>>>> towards
>>>>>>>>>>>>>>>>>>> using the statestore.transaction.buffer.max.bytes
>>>>>>>> configuration
>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to exceed it. This
>>>>>> would
>>>>>>>>>>>>>>>>>> complement the
>>>>>>>>>>>>>>>>>>> existing "early Task commit" functionality that this
>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>> provides, in the following way:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>          - Currently, we use
>>>>>>>>>> statestore.transaction.buffer.max.bytes
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> force an
>>>>>>>>>>>>>>>>>>>          early Task commit if processing more records
>> would
>>>>>>>> cause
>>>>>>>>>> our
>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>          transactions to exceed the memory assigned to
>>>>>> them.
>>>>>>>>>>>>>>>>>>>          - New functionality: when a Task *does* commit,
>>>>>> we will
>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>>>>>>          the stores (and hence flush the transaction
>>>>>> buffers)
>>>>>>>>>> unless
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> expect to
>>>>>>>>>>>>>>>>>>>          cross the
>> statestore.transaction.buffer.max.bytes
>>>>>>>>>> threshold
>>>>>>>>>>>>>> before
>>>>>>>>>>>>>>>>>> the next
>>>>>>>>>>>>>>>>>>>          commit
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I'm also open to suggestions.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
>>>>>>>>>> nick.telford@gmail.com
>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi Bruno!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>> By "less predictable for users", I meant in terms of
>>>>>>>>>> understanding
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> performance profile under various circumstances. The
>>>>>> more
>>>>>>>>>> complex
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> solution, the more difficult it would be for users to
>>>>>>>>>> understand
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> performance they see. For example, spilling records to
>>>>>> disk
>>>>>>>>>> when
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold would, I expect,
>>>>>>>> reduce
>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>> throughput. This reduction in write throughput could be
>>>>>>>>>>>> unexpected,
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> potentially difficult to diagnose/understand for users.
>>>>>>>>>>>>>>>>>>>> At the moment, I think the "early commit" concept is
>>>>>>>> relatively
>>>>>>>>>>>>>>>>>>>> straightforward; it's easy to document, and
>> conceptually
>>>>>>>> fairly
>>>>>>>>>>>>>>>>>> obvious to
>>>>>>>>>>>>>>>>>>>> users. We could probably add a metric to make it easier
>>>>>> to
>>>>>>>>>>>>>> understand
>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>> it happens though.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 3. (the second one)
>>>>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an indirect way of
>>>>>>>> telling
>>>>>>>>>>>>>>>>>> StateStores
>>>>>>>>>>>>>>>>>>>> whether they should be transactional. READ_COMMITTED
>>>>>>>>>> essentially
>>>>>>>>>>>>>>>>>> requires
>>>>>>>>>>>>>>>>>>>> transactions, because it dictates that two threads
>>>>>> calling
>>>>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes from the other
>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all
>>>>>> bets are
>>>>>>>>>> off,
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> stores can allow threads to observe written records at
>>>>>> any
>>>>>>>>>> time,
>>>>>>>>>>>>>>>>>> which is
>>>>>>>>>>>>>>>>>>>> essentially "no transactions". That said, StateStores
>>>>>> are
>>>>>>>> free
>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> implement
>>>>>>>>>>>>>>>>>>>> these guarantees however they can, which is a bit more
>>>>>>>> relaxed
>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>> dictating "you must use transactions". For example,
>> with
>>>>>>>>>> RocksDB
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
>>>>>>>> "transactions",
>>>>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But
>>>>>> with
>>>>>>>>>> other
>>>>>>>>>>>>>>>>>> storage
>>>>>>>>>>>>>>>>>>>> engines, it might be preferable to *always* use
>>>>>> transactions,
>>>>>>>>>> even
>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines that don't
>>>>>>>> provide
>>>>>>>>>>>>>>>>>>>> transactions, but the isolation guarantees can be met
>>>>>> using a
>>>>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>>>>> technique.
>>>>>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore interface as
>>>>>>>> loosely
>>>>>>>>>>>>>> coupled
>>>>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
>>>>>> implementers
>>>>>>>> more
>>>>>>>>>>>>>>>>>> freedom, and
>>>>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge required.
>>>>>>>>>>>>>>>>>>>> That said, I understand that "IsolationLevel" might not
>>>>>> be
>>>>>>>> the
>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>> abstraction, and we can always make it much more
>>>>>> explicit if
>>>>>>>>>>>>>>>>>> required, e.g.
>>>>>>>>>>>>>>>>>>>> boolean transactional()
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 7-8.
>>>>>>>>>>>>>>>>>>>> I can make these changes either later today or
>> tomorrow.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Small update:
>>>>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of
>>>>>> issues
>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> needed
>>>>>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass, which is
>>>>>>>> promising,
>>>>>>>>>> but
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>> need to undergo some performance testing. I haven't
>>>>>> (yet)
>>>>>>>>>> worked
>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I would
>>>>>> expect
>>>>>>>> that,
>>>>>>>>>>>>>>>>>>>> behaviourally, it should make no difference. The branch
>>>>>> is
>>>>>>>>>>>> available
>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if
>>>>>>>> anyone
>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> interested in taking an early look.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
>>>>>>>>>> cadonna@apache.org>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually also my
>>>>>> point. I
>>>>>>>>>>>>>> understood
>>>>>>>>>>>>>>>>>>>>> that John was proposing the ingestion path as a way to
>>>>>> avoid
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> early
>>>>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>>>> I agree with John here, that actually it is public
>>>>>> API. My
>>>>>>>>>>>> question
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> how this usage pattern affects normal processing.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>> My concern is that checking for the size of the
>>>>>> transaction
>>>>>>>>>>>> buffer
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> maybe triggering an early commit affects the whole
>>>>>>>> processing
>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>> Streams. The transactionality of a state store is not
>>>>>>>>>> confined to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> state store itself, but spills over and changes the
>>>>>> behavior
>>>>>>>>>> of
>>>>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>>>>> parts of the system. I agree with you that it is a
>>>>>> decent
>>>>>>>>>>>>>>>>>> compromise. I
>>>>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and list the
>>>>>> options to
>>>>>>>>>>>>>> overcome
>>>>>>>>>>>>>>>>>>>>> them. I also agree with you that all options seem
>> quite
>>>>>>>> heavy
>>>>>>>>>>>>>>>>>> compared
>>>>>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you mean with
>>>>>> "less
>>>>>>>>>>>>>>>>>> predictable
>>>>>>>>>>>>>>>>>>>>> for users", though.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I found the discussions about the alternatives really
>>>>>>>>>>>> interesting.
>>>>>>>>>>>>>>>>>> But I
>>>>>>>>>>>>>>>>>>>>> also think that your plan sounds good and we should
>>>>>> continue
>>>>>>>>>> with
>>>>>>>>>>>>>> it!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind putting
>>>>>> isolation
>>>>>>>>>>>> level
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should that also be a
>>>>>> way
>>>>>>>> to
>>>>>>>>>>>> give
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> the state store the opportunity to decide whether to
>>>>>> turn on
>>>>>>>>>>>>>>>>>>>>> transactions or not?
>>>>>>>>>>>>>>>>>>>>> With my comment, I was more concerned about how do you
>>>>>> know
>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>> checkpoint file needs to be written under EOS, if you
>>>>>> do not
>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>>>>> to know if the state store is transactional or not. If
>>>>>> a
>>>>>>>> state
>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be written
>>>>>> during
>>>>>>>>>> normal
>>>>>>>>>>>>>>>>>>>>> processing under EOS. If the state store is not
>>>>>>>> transactional,
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> checkpoint file must not be written under EOS.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 7.
>>>>>>>>>>>>>>>>>>>>> My point was about not only considering the bytes in
>>>>>> memory
>>>>>>>> in
>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that
>>>>>> might
>>>>>>>> be
>>>>>>>>>>>>>>>>>> spilled
>>>>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering whether you should
>>>>>>>> remove
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory bytes to be used
>>>>>> to
>>>>>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My thinking
>>>>>> was
>>>>>>>> that
>>>>>>>>>>>> even
>>>>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to disk, limiting
>>>>>> the
>>>>>>>>>>>> overall
>>>>>>>>>>>>>>>>>> bytes
>>>>>>>>>>>>>>>>>>>>> might make sense. Thinking about it again and
>>>>>> considering
>>>>>>>> the
>>>>>>>>>>>>>> recent
>>>>>>>>>>>>>>>>>>>>> discussions, it does not make too much sense anymore.
>>>>>>>>>>>>>>>>>>>>> I like the name
>>>>>> statestore.transaction.buffer.max.bytes that
>>>>>>>>>> you
>>>>>>>>>>>>>>>>>> proposed.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 8.
>>>>>>>>>>>>>>>>>>>>> A high-level description (without implementation
>>>>>> details) of
>>>>>>>>>> how
>>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
>>>>>> transactions,
>>>>>>>>>> state
>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>> transactions and checkpointing would be great. Would
>> be
>>>>>>>> great
>>>>>>>>>> if
>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>> could also add some sentences about the behavior in
>>>>>> case of
>>>>>>>> a
>>>>>>>>>>>>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>> For instance how does a transactional state store
>>>>>> recover
>>>>>>>>>> after a
>>>>>>>>>>>>>>>>>>>>> failure or what happens with the transaction buffer,
>>>>>> etc.
>>>>>>>>>> (that
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
>>>>>> WriteBatchWithIndex
>>>>>>>>>>>>>>>>>> transactions
>>>>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
>>>>>> configured
>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>> needs to
>>>>>>>>>>>>>>>>>>>>>> trigger an early commit?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>>>>> This is one of my big concerns. Ultimately, any
>>>>>> approach
>>>>>>>>>> based
>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>> cracking
>>>>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in ways it's not
>>>>>> really
>>>>>>>>>>>>>> designed
>>>>>>>>>>>>>>>>>>>>> for is
>>>>>>>>>>>>>>>>>>>>>> likely to have some unforseen performance or
>>>>>> consistency
>>>>>>>>>> issues.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>>>>> What's your motivation for removing these early
>>>>>> commits?
>>>>>>>>>> While
>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> ideal, I
>>>>>>>>>>>>>>>>>>>>>> think they're a decent compromise to ensure
>>>>>> consistency
>>>>>>>>>> whilst
>>>>>>>>>>>>>>>>>>>>> maintaining
>>>>>>>>>>>>>>>>>>>>>> good and predictable performance.
>>>>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
>>>>>> complicated, and
>>>>>>>>>> might
>>>>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>>>>> make behaviour less predictable for users as a
>>>>>> consequence.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is
>>>>>> growing a
>>>>>>>>>> bit
>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas for future
>>>>>>>>>>>>>> improvements, I
>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>> it's important to narrow the scope down to a design
>>>>>> that
>>>>>>>>>>>> achieves
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> most
>>>>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized restorations
>>>>>> during
>>>>>>>> dirty
>>>>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that this KIP
>>>>>> produces
>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>> ultimately
>>>>>>>>>>>>>>>>>>>>>> be changed in the future, especially if the bulk of
>>>>>> it is
>>>>>>>>>>>> internal
>>>>>>>>>>>>>>>>>>>>>> behaviour.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week trying to
>>>>>> re-work
>>>>>>>> the
>>>>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
>>>>>> newTransaction()
>>>>>>>>>>>> method,
>>>>>>>>>>>>>>>>>> such
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> it's just an implementation detail of RocksDBStore.
>>>>>> That
>>>>>>>>>> way, if
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> want to
>>>>>>>>>>>>>>>>>>>>>> replace WBWI with something in the future, like the
>>>>>> SST
>>>>>>>> file
>>>>>>>>>>>>>>>>>> management
>>>>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with little/no
>> API
>>>>>>>>>> changes.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Sophie,

Thanks for taking the time to review the KIP and catch up.

> my singular goal in responding is to help this KIP past a perceived
impasse so we can finally move on to voting and implementing it
Just so we're clear, is the impasse you're referring to this limitation in
the current version of the KIP?

> The READ_UNCOMMITTED isolation level will only be available under the
at-least-once processing.mode. If READ_UNCOMMITTED is selected with an EOS
processing.mode, it will be automatically upgraded to READ_COMMITTED and a
warning will be produced.

Firstly, I'd like to make an argument for this being reasonable for all EOS
users, and then I'm going to discuss the technical reasons why I didn't
include a feature flag, and why including one might be problematic.

Note: I'm using emphasis to highlight important points, not to convey
emotion.

*READ_COMMITTED by default for EOS:*
The main thing to bear in mind is that the isolation level only affects
Interactive Queries. Stream processing (processors that read from stores,
like join) will *always* read from the ongoing transaction, irrespective of
isolation level.
This means that to be affected by this change, you must:

   1. have processing.mode = exactly-once(-v2|-beta)
   2. be conducting Interactive Queries against stores

At READ_COMMITTED, the maximum latency (the time between a record being
processed and a record being visible to Interactive Queries) is dictated by
the commit.interval.ms. Under EOS this defaults to 100ms, which should be
sufficiently low that a difference will be (nearly) undetectable to users.

Regardless, there is a change being introduced here, so I want to think
about what that means, semantically.

Presently under EOS, (essentially READ_UNCOMMITTED) records are visible to
Interactive Queries as soon as they are written to the local StateStore,
but *before* the records are made available to the changelog topic by a
Task commit. This creates the curious situation where an Interactive Query
could see a write rolled back. The order of events would be this:

   1. Record A is processed and written to a StateStore
   2. An Interactive Query observes Record A
   3. The application crashes *before* the Task commits.
   4. On restart, the application rebuilds its state from the changelog.
   5. The Task transitions to RUNNING, making its StateStore queryable by
   Interactive Queries.
   6. *An Interactive Query attempts to observe Record A, but finds it
   missing.*
   7. Record A is processed and written to the StateStore.

For this reason, I would classify an isolation level of READ_UNCOMMITTED
under EOS as a bug, rather than a feature. The changelogs are intended to
be the gold-store for StateStores, so *Interactive Queries should only ever
be able to observe state that has been persisted to the changelog*.

Colt did raise a use-case above that would depend on READ_UNCOMMITTED under
EOS. This appears to require that two Interactive Queries are processed
that are related by a record, one that creates it, and one that reads it
back. I would like to better understand this use-case/usage pattern; in
particular, how the Interactive Query creates a StateStore record. *Colt,
could you either reply to this thread or reach out on Slack with more
detail here please?* Depending on your design, it's possible that this
might be mitigated/resolved by having your processor explicitly request an
early Task commit via ProcessorContext#commit().

*Feature flag:*
I have been implementing this KIP in parallel with its design, because I
don't know enough about the internals of Kafka Streams to settle on a
design without first exploring the feasibility. While such a feature flag
might well be possible, I have not yet had the time to attempt to implement
it. I believe the main difficulty will be with offset management, as it
will need a completely different code-path when operating
non-transactionally, in order to detect when the store contains uncommitted
data.

FWIW, I don't think an explicit feature flag is necessary/desirable, but
instead (if possible) we should replace the current restriction with:

> Iff processing.mode = exactly-once(-v2|-beta) and
default.state.isolation.level = READ_UNCOMMITTED, local state will be wiped
and rebuilt from changelogs on-error, as is currently the case in 3.6.0.

This way, there is no change to behaviour at all, until a user explicitly
sets default.state.isolation.level to READ_COMMITTED, which would (under
EOS) no longer require wiping of the store.

The advantage of this over an explicit "enable transactions" feature flag
is that:

   1. It doesn't need to be deprecated and removed in the future once
   transactionality no longer has restrictions and becomes the default.
   2. It allows for the possibility of a future KIP that selectively alters
   the isolation level for individual Interactive Queries.

However, this all assumes that it is indeed possible to make
READ_UNCOMMITTED work under EOS.

In the interests of trying to get this KIP over the line ASAP, I settled on
adding the restriction that READ_UNCOMMITTED would be unavailable under
EOS, with the goal of relaxing this in a future KIP.

If it turns out that this restriction is a blocker, then I'll try to find
the time to explore the possibility of adding a flag.

Regards,
Nick

On Thu, 12 Oct 2023 at 21:32, Sophie Blee-Goldman <ab...@gmail.com>
wrote:

> Hey Nick! First of all thanks for taking up this awesome feature, I'm sure
> every single
> Kafka Streams user and dev would agree that it is sorely needed.
>
> I've just been catching up on the KIP and surrounding discussion, so please
> forgive me
> for any misunderstandings or misinterpretations of the current plan and
> don't hesitate to
> correct me.
>
> Before I jump in, I just want to say that having seen this drag on for so
> long, my singular
> goal in responding is to help this KIP past a perceived impasse so we can
> finally move on
> to voting and implementing it. Long discussions are to be expected for
> major features like
> this but it's completely on us as the Streams devs to make sure there is an
> end in sight
> for any ongoing discussion.
>
> With that said, it's my understanding that the KIP as currently proposed is
> just not tenable
> for Kafka Streams, and would prevent some EOS users from upgrading to the
> version it
> first appears in. Given that we can't predict or guarantee whether any of
> the followup KIPs
> would be completed in the same release cycle as this one, we need to make
> sure that the
> feature is either compatible with all current users or else feature-flagged
> so that they may
> opt in/out.
>
> Therefore, IIUC we need to have either (or both) of these as
> fully-implemented config options:
> 1. default.state.isolation.level
> 2. enable.transactional.state.stores
>
> This way EOS users for whom read_committed semantics are not viable can
> still upgrade,
> and either use the isolation.level config to leverage the new txn state
> stores without sacrificing
> their application semantics, or else simply keep the transactional state
> stores disabled until we
> are able to fully implement the isolation level configuration at either an
> application or query level.
>
> Frankly you are the expert here and know much more about the tradeoffs in
> both semantics and
> effort level of implementing one of these configs vs the other. In my
> opinion, either option would
> be fine and I would leave the decision of which one to include in this KIP
> completely up to you.
> I just don't see a way for the KIP to proceed without some variation of the
> above that would allow
> EOS users to opt-out of read_committed.
>
> (If it's all the same to you, I would recommend 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 that gets released.
> So personally, my choice would be to add the feature flag and leave it off
> by default. If all 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
>
> On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <ni...@gmail.com>
> wrote:
>
> > Hi Bruno,
> >
> > Agreed, I can live with that for now.
> >
> > In an effort to keep the scope of this KIP from expanding, I'm leaning
> > towards just providing a configurable default.state.isolation.level and
> > removing IsolationLevel from the StateStoreContext. This would be
> > compatible with adding support for query-time IsolationLevels in the
> > future, whilst providing a way for users to select an isolation level
> now.
> >
> > The big problem with this, however, is that if a user selects
> > processing.mode
> > = "exactly-once(-v2|-beta)", and default.state.isolation.level =
> > "READ_UNCOMMITTED", we need to guarantee that the data isn't written to
> > disk until commit() is called, but we also need to permit IQ threads to
> > read from the ongoing transaction.
> >
> > A simple solution would be to (temporarily) forbid this combination of
> > configuration, and have default.state.isolation.level automatically
> switch
> > to READ_COMMITTED when processing.mode is anything other than
> > at-least-once. Do you think this would be acceptable?
> >
> > In a later KIP, we can add support for query-time isolation levels and
> > solve this particular problem there, which would relax this restriction.
> >
> > Regards,
> > Nick
> >
> > On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <ca...@apache.org> 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,
> > > Bruno
> > >
> > > On 9/18/23 7:12 PM, Nick Telford wrote:
> > > > 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 <ni...@gmail.com>
> > > wrote:
> > > >
> > > >> Hi everyone,
> > > >>
> > > >> I agree that having IsolationLevel be determined at query-time is
> the
> > > >> ideal design, but there are a few sticking points:
> > > >>
> > > >> 1.
> > > >> There needs to be some way to communicate the IsolationLevel down to
> > the
> > > >> RocksDBStore itself, so that the query can respect it. Since stores
> > are
> > > >> "layered" in functionality (i.e. ChangeLoggingStore, MeteredStore,
> > > etc.),
> > > >> we need some way to deliver that information to the bottom layer.
> For
> > > IQv2,
> > > >> we can use the existing State#query() method, but IQv1 has no way to
> > do
> > > >> this.
> > > >>
> > > >> A simple approach, which would potentially open up other options,
> > would
> > > be
> > > >> to add something like: ReadOnlyKeyValueStore<K, V>
> > > >> readOnlyView(IsolationLevel isolationLevel) to ReadOnlyKeyValueStore
> > > (and
> > > >> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
> > > >>
> > > >> 2.
> > > >> As mentioned above, RocksDB WriteBatches are not thread-safe, which
> > > causes
> > > >> a problem if we want to provide READ_UNCOMMITTED Iterators. I also
> > had a
> > > >> look at RocksDB Transactions[1], but they solve a very different
> > > problem,
> > > >> and have the same thread-safety issue.
> > > >>
> > > >> One possible approach that I mentioned is chaining WriteBatches:
> every
> > > >> time a new Interactive Query is received (i.e. readOnlyView, see
> > above,
> > > >> is called) we "freeze" the existing WriteBatch, and start a new one
> > for
> > > new
> > > >> writes. The Interactive Query queries the "chain" of previous
> > > WriteBatches
> > > >> + the underlying database; while the StreamThread starts writing to
> > the
> > > >> *new* WriteBatch. On-commit, the StreamThread would write *all*
> > > >> WriteBatches in the chain to the database (that have not yet been
> > > written).
> > > >>
> > > >> WriteBatches would be closed/freed only when they have been both
> > > >> committed, and all open Interactive Queries on them have been
> closed.
> > > This
> > > >> would require some reference counting.
> > > >>
> > > >> Obviously a drawback of this approach is the potential for increased
> > > >> memory usage: if an Interactive Query is long-lived, for example by
> > > doing a
> > > >> full scan over a large database, or even just pausing in the middle
> of
> > > an
> > > >> iteration, then the existing chain of WriteBatches could be kept
> > around
> > > for
> > > >> a long time, potentially forever.
> > > >>
> > > >> --
> > > >>
> > > >> A.
> > > >> Going off on a tangent, it looks like in addition to supporting
> > > >> READ_COMMITTED queries, we could go further and support
> > REPEATABLE_READ
> > > >> queries (i.e. where subsequent reads to the same key in the same
> > > >> Interactive Query are guaranteed to yield the same value) by making
> > use
> > > of
> > > >> RocksDB Snapshots[2]. These are fairly lightweight, so the
> performance
> > > >> impact is likely to be negligible, but they do require that the
> > > Interactive
> > > >> Query session can be explicitly closed.
> > > >>
> > > >> This could be achieved if we made the above readOnlyView interface
> > look
> > > >> more like:
> > > >>
> > > >> interface ReadOnlyKeyValueView<K, V> implements
> > ReadOnlyKeyValueStore<K,
> > > >> V>, AutoCloseable {}
> > > >>
> > > >> interface ReadOnlyKeyValueStore<K, V> {
> > > >>      ...
> > > >>      ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel
> > > isolationLevel);
> > > >> }
> > > >>
> > > >> But this would be a breaking change, as existing IQv1 queries are
> > > >> guaranteed to never call store.close(), and therefore these would
> leak
> > > >> memory under REPEATABLE_READ.
> > > >>
> > > >> B.
> > > >> One thing that's notable: MyRocks states that they support
> > > READ_COMMITTED
> > > >> and REPEATABLE_READ, but they make no mention of
> > READ_UNCOMMITTED[3][4].
> > > >> This could be because doing so is technically difficult/impossible
> > using
> > > >> the primitives available in RocksDB.
> > > >>
> > > >> --
> > > >>
> > > >> Lucas, to address your points:
> > > >>
> > > >> U1.
> > > >> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
> > > >> implementations of StateStore that do not support atomic writes.
> > > Obviously
> > > >> in those cases, the guarantees Kafka Streams provides/expects would
> be
> > > >> relaxed. Do you think we should require all implementations to
> support
> > > >> atomic writes?
> > > >>
> > > >> U2.
> > > >> Stores can support multiple IsolationLevels. As we've discussed
> above,
> > > the
> > > >> ideal scenario would be to specify the IsolationLevel at query-time.
> > > >> Failing that, I think the second-best approach is to define the
> > > >> IsolationLevel for *all* queries based on the processing.mode, which
> > is
> > > >> what the default StateStoreContext#isolationLevel() achieves. Would
> > you
> > > >> prefer an alternative?
> > > >>
> > > >> While the existing implementation is equivalent to READ_UNCOMMITTED,
> > > this
> > > >> can yield unexpected results/errors under EOS, if a transaction is
> > > rolled
> > > >> back. While this would be a change in behaviour for users, it would
> > look
> > > >> more like a bug fix than a breaking change. That said, we *could*
> make
> > > it
> > > >> configurable, and default to the existing behaviour
> (READ_UNCOMMITTED)
> > > >> instead of inferring it from the processing.mode?
> > > >>
> > > >> N1, N2.
> > > >> These were only primitives to avoid boxing costs, but since this is
> > not
> > > a
> > > >> performance sensitive area, it should be fine to change if that's
> > > desirable.
> > > >>
> > > >> N3.
> > > >> It's because the store "manages its own offsets", which includes
> both
> > > >> committing the offset, *and providing it* via getCommittedOffset().
> > > >> Personally, I think "managesOffsets" conveys this best, but I don't
> > mind
> > > >> changing it if the nomenclature is unclear.
> > > >>
> > > >> Sorry for the massive emails/essays!
> > > >> --
> > > >> Nick
> > > >>
> > > >> 1: https://github.com/facebook/rocksdb/wiki/Transactions
> > > >> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > > >> 3: https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > > >> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > > >>
> > > >> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > > >> <lb...@confluent.io.invalid> wrote:
> > > >>
> > > >>> Hi Nick,
> > > >>>
> > > >>> since I last read it in April, the KIP has become much cleaner and
> > > >>> easier to read. Great work!
> > > >>>
> > > >>> It feels to me the last big open point is whether we can implement
> > > >>> isolation level as a query parameter. I understand that there are
> > > >>> implementation concerns, but as Colt says, it would be a great
> > > >>> addition, and would also simplify the migration path for this
> change.
> > > >>> Is the implementation problem you mentioned caused by the
> WriteBatch
> > > >>> not having a notion of a snapshot, as the underlying DB iterator
> > does?
> > > >>> In that case, I am not sure a chain of WriteBatches as you propose
> > > >>> would fully solve the problem, but maybe I didn't dig enough into
> the
> > > >>> details to fully understand it.
> > > >>>
> > > >>> If it's not possible to implement it now, would it be an option to
> > > >>> make sure in this KIP that we do not fully close the door on
> > per-query
> > > >>> isolation levels in the interface, as it may be possible to
> implement
> > > >>> the missing primitives in RocksDB or Speedb in the future.
> > > >>>
> > > >>> Understanding:
> > > >>>
> > > >>> * U1) Why is it only "SHOULD" for changelogOffsets to be persisted
> > > >>> atomically with the records?
> > > >>> * U2) Don't understand the default implementation of
> > `isolationLevel`.
> > > >>> The isolation level should be a property of the underlying store,
> and
> > > >>> not be defined by the default config? Existing stores probably
> don't
> > > >>> guarantee READ_COMMITTED, so the default should be to return
> > > >>> READ_UNCOMMITTED.
> > > >>>
> > > >>> Nits:
> > > >>> * N1) Could `getComittedOffset` use an `OptionalLong` return type,
> to
> > > >>> avoid the `null`?
> > > >>> * N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
> > > >>> return type, to avoid the `-1`?
> > > >>> * N3) I don't understand why `managesOffsets` uses the 'manage'
> verb,
> > > >>> whereas all other methods use the "commits" verb. I'd suggest
> > > >>> `commitsOffsets`.
> > > >>>
> > > >>> Either way, it feels this KIP is very close to the finish line, I'm
> > > >>> looking forward to seeing this in production!
> > > >>>
> > > >>> Cheers,
> > > >>> Lucas
> > > >>>
> > > >>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <co...@littlehorse.io>
> > > wrote:
> > > >>>>
> > > >>>>> Making IsolationLevel a query-time constraint, rather than
> linking
> > it
> > > >>> to
> > > >>>> the processing.guarantee.
> > > >>>>
> > > >>>> As I understand it, would this allow even a user of EOS to control
> > > >>> whether
> > > >>>> reading committed or uncommitted records? If so, I am highly in
> > favor
> > > of
> > > >>>> this.
> > > >>>>
> > > >>>> I know that I was one of the early people to point out the current
> > > >>>> shortcoming that IQ reads uncommitted records, but just this
> > morning I
> > > >>>> realized a pattern we use which means that (for certain queries)
> our
> > > >>> system
> > > >>>> needs to be able to read uncommitted records, which is the current
> > > >>> behavior
> > > >>>> of Kafka Streams in EOS.***
> > > >>>>
> > > >>>> If IsolationLevel being a query-time decision allows for this,
> then
> > > that
> > > >>>> would be amazing. I would also vote that the default behavior
> should
> > > be
> > > >>> for
> > > >>>> reading uncommitted records, because it is totally possible for a
> > > valid
> > > >>>> application to depend on that behavior, and breaking it in a minor
> > > >>> release
> > > >>>> might be a bit strong.
> > > >>>>
> > > >>>> *** (Note, for the curious reader....) Our use-case/query pattern
> > is a
> > > >>> bit
> > > >>>> complex, but reading "uncommitted" records is actually safe in our
> > > case
> > > >>>> 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 soon as the processor processes the result.
> If
> > > they
> > > >>>> can't read uncommitted records, they may receive a "201 - Created"
> > > >>>> response, immediately followed by a "404 - 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 <
> > nick.telford@gmail.com>
> > > >>> wrote:
> > > >>>>
> > > >>>>> Addendum:
> > > >>>>>
> > > >>>>> I think we would also face the same problem with the approach
> John
> > > >>> outlined
> > > >>>>> earlier (using the record cache as a transaction buffer and
> > flushing
> > > >>> it
> > > >>>>> straight to SST files). This is because the record cache (the
> > > >>> ThreadCache
> > > >>>>> class) is not thread-safe, so every commit would invalidate open
> IQ
> > > >>>>> Iterators in the same way that RocksDB WriteBatches do.
> > > >>>>> --
> > > >>>>> Nick
> > > >>>>>
> > > >>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> > nick.telford@gmail.com>
> > > >>>>> 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.
> > > >>>>>> 2. Making IsolationLevel a query-time constraint, rather than
> > > >>> linking it
> > > >>>>>> to the processing.guarantee.
> > > >>>>>>
> > > >>>>>> There's a wrinkle that makes this a challenge: Interactive
> Queries
> > > >>> that
> > > >>>>>> open an Iterator, when using transactions and READ_UNCOMMITTED.
> > > >>>>>> The problem is that under READ_UNCOMMITTED, queries need to be
> > able
> > > >>> to
> > > >>>>>> read records from the currently uncommitted transaction buffer
> > > >>>>>> (WriteBatch). This includes for Iterators, which should iterate
> > > >>> both the
> > > >>>>>> transaction buffer and underlying database (using
> > > >>>>>> WriteBatch#iteratorWithBase()).
> > > >>>>>>
> > > >>>>>> The issue is that when the StreamThread commits, it writes the
> > > >>> current
> > > >>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing
> > the
> > > >>>>>> WriteBatch while an Interactive Query holds an open Iterator on
> it
> > > >>> will
> > > >>>>>> invalidate the Iterator. Worse, it turns out that Iterators
> over a
> > > >>>>>> WriteBatch become invalidated not just when the WriteBatch is
> > > >>> cleared,
> > > >>>>> but
> > > >>>>>> also when the Iterators' current key receives a new write.
> > > >>>>>>
> > > >>>>>> Now that I'm writing this, I remember that this is the major
> > reason
> > > >>> that
> > > >>>>> I
> > > >>>>>> switched the original design from having a query-time
> > > >>> IsolationLevel to
> > > >>>>>> having the IsolationLevel linked to the transactionality of the
> > > >>> stores
> > > >>>>>> themselves.
> > > >>>>>>
> > > >>>>>> It *might* be possible to resolve this, by having a "chain" of
> > > >>>>>> WriteBatches, with the StreamThread switching to a new
> WriteBatch
> > > >>>>> whenever
> > > >>>>>> a new Interactive Query attempts to read from the database, but
> > that
> > > >>>>> could
> > > >>>>>> cause some performance problems/memory pressure when subjected
> to
> > a
> > > >>> high
> > > >>>>>> Interactive Query load. It would also reduce the efficiency of
> > > >>>>> WriteBatches
> > > >>>>>> on-commit, as we'd have to write N WriteBatches, where N is the
> > > >>> number of
> > > >>>>>> Interactive Queries since the last commit.
> > > >>>>>>
> > > >>>>>> I realise this is getting into the weeds of the implementation,
> > and
> > > >>> you'd
> > > >>>>>> rather we focus on the API for now, but I think it's important
> to
> > > >>>>> consider
> > > >>>>>> how to implement the desired API, in case we come up with an API
> > > >>> that
> > > >>>>>> cannot be implemented efficiently, or even at all!
> > > >>>>>>
> > > >>>>>> Thoughts?
> > > >>>>>> --
> > > >>>>>> Nick
> > > >>>>>>
> > > >>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <
> cadonna@apache.org>
> > > >>> wrote:
> > > >>>>>>
> > > >>>>>>> Hi Nick,
> > > >>>>>>>
> > > >>>>>>> 6.
> > > >>>>>>> Of course, you are right! My bad!
> > > >>>>>>> Wiping out the state in the downgrading case is fine.
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> 3a.
> > > >>>>>>> Focus on the public facing changes for the KIP. We will manage
> 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.
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> Best,
> > > >>>>>>> Bruno
> > > >>>>>>>
> > > >>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > > >>>>>>>> Hi Bruno,
> > > >>>>>>>>
> > > >>>>>>>> Thanks for getting back to me!
> > > >>>>>>>>
> > > >>>>>>>> 2.
> > > >>>>>>>> The fact that implementations can always track estimated
> memory
> > > >>> usage
> > > >>>>> in
> > > >>>>>>>> the wrapper is a good point. I can remove -1 as an option, and
> > > >>> I'll
> > > >>>>>>> clarify
> > > >>>>>>>> the JavaDoc that 0 is not just for non-transactional stores,
> > > >>> which is
> > > >>>>>>>> currently misleading.
> > > >>>>>>>>
> > > >>>>>>>> 6.
> > > >>>>>>>> The problem with catching the exception in the downgrade
> process
> > > >>> is
> > > >>>>> that
> > > >>>>>>>> would require new code in the Kafka version being downgraded
> to.
> > > >>> Since
> > > >>>>>>>> users could conceivably downgrade to almost *any* older
> version
> > > >>> of
> > > >>>>> Kafka
> > > >>>>>>>> Streams, I'm not sure how we could add that code?
> > > >>>>>>>> The only way I can think of doing it would be to provide a
> > > >>> dedicated
> > > >>>>>>>> downgrade tool, that goes through every local store and
> removes
> > > >>> the
> > > >>>>>>>> offsets column families. But that seems like an unnecessary
> > > >>> amount of
> > > >>>>>>> extra
> > > >>>>>>>> code to maintain just to handle a somewhat niche situation,
> when
> > > >>> the
> > > >>>>>>>> alternative (automatically wipe and restore stores) should be
> > > >>>>>>> acceptable.
> > > >>>>>>>>
> > > >>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've requested.
> > > >>>>>>>>
> > > >>>>>>>> 3a.
> > > >>>>>>>> I agree that IsolationLevel makes more sense at query-time,
> and
> > I
> > > >>>>>>> actually
> > > >>>>>>>> initially attempted to place the IsolationLevel at query-time,
> > > >>> but I
> > > >>>>> ran
> > > >>>>>>>> into some problems:
> > > >>>>>>>> - The key issue is that, under ALOS we're not staging writes
> in
> > > >>>>>>>> transactions, so can't perform writes at the READ_COMMITTED
> > > >>> isolation
> > > >>>>>>>> level. However, this may be addressed if we decide to *always*
> > > >>> use
> > > >>>>>>>> transactions as discussed under 3b.
> > > >>>>>>>> - IQv1 and IQv2 have quite different implementations. I
> remember
> > > >>>>> having
> > > >>>>>>>> some difficulty understanding the IQv1 internals, which made
> it
> > > >>>>>>> difficult
> > > >>>>>>>> to determine what needed to be changed. However, I *think*
> this
> > > >>> can be
> > > >>>>>>>> addressed for both implementations by wrapping the
> RocksDBStore
> > > >>> in an
> > > >>>>>>>> IsolationLevel-dependent wrapper, that overrides read methods
> > > >>> (get,
> > > >>>>>>> etc.)
> > > >>>>>>>> to either read directly from the database or from the ongoing
> > > >>>>>>> transaction.
> > > >>>>>>>> But IQv1 might still be difficult.
> > > >>>>>>>> - If IsolationLevel becomes a query constraint, then all other
> > > >>>>>>> StateStores
> > > >>>>>>>> will need to respect it, including the in-memory stores. This
> > > >>> would
> > > >>>>>>> require
> > > >>>>>>>> us to adapt in-memory stores to stage their writes so they can
> > be
> > > >>>>>>> isolated
> > > >>>>>>>> from READ_COMMITTTED queries. It would also become an
> important
> > > >>>>>>>> consideration for third-party stores on upgrade, as without
> > > >>> changes,
> > > >>>>>>> they
> > > >>>>>>>> would not support READ_COMMITTED queries correctly.
> > > >>>>>>>>
> > > >>>>>>>> Ultimately, I may need some help making the necessary change
> to
> > > >>> IQv1
> > > >>>>> to
> > > >>>>>>>> support this, but I don't think it's fundamentally impossible,
> > > >>> if we
> > > >>>>>>> want
> > > >>>>>>>> to pursue this route.
> > > >>>>>>>>
> > > >>>>>>>> 3b.
> > > >>>>>>>> The main reason I chose to keep ALOS un-transactional was to
> > > >>> minimize
> > > >>>>>>>> behavioural change for most users (I believe most Streams
> users
> > > >>> use
> > > >>>>> the
> > > >>>>>>>> default configuration, which is ALOS). That said, it's clear
> > > >>> that if
> > > >>>>>>> ALOS
> > > >>>>>>>> also used transactional stores, the only change in behaviour
> > > >>> would be
> > > >>>>>>> that
> > > >>>>>>>> it would become *more correct*, which could be considered a
> "bug
> > > >>> fix"
> > > >>>>> by
> > > >>>>>>>> users, rather than a change they need to handle.
> > > >>>>>>>>
> > > >>>>>>>> I believe that performance using transactions (aka. RocksDB
> > > >>>>>>> WriteBatches)
> > > >>>>>>>> should actually be *better* than the un-batched write-path
> that
> > > >>> is
> > > >>>>>>>> currently used[1]. The only "performance" consideration will
> be
> > > >>> the
> > > >>>>>>>> increased memory usage that transactions require. Given the
> > > >>>>> mitigations
> > > >>>>>>> for
> > > >>>>>>>> this memory that we have in place, I would expect that this is
> > > >>> not a
> > > >>>>>>>> problem for most users.
> > > >>>>>>>>
> > > >>>>>>>> If we're happy to do so, we can make ALOS also use
> transactions.
> > > >>>>>>>>
> > > >>>>>>>> Regards,
> > > >>>>>>>> Nick
> > > >>>>>>>>
> > > >>>>>>>> Link 1:
> > > >>>>>>>>
> > > >>>>>
> > > >>>
> > >
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > >>>>>>>>
> > > >>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> > cadonna@apache.org
> > > >>>>
> > > >>>>>>> wrote:
> > > >>>>>>>>
> > > >>>>>>>>> Hi Nick,
> > > >>>>>>>>>
> > > >>>>>>>>> Thanks for the updates and sorry for the delay on my side!
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> 1.
> > > >>>>>>>>> Making the default implementation for flush() a no-op sounds
> > > >>> good to
> > > >>>>>>> me.
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> 2.
> > > >>>>>>>>> I think what was bugging me here is that a third-party state
> > > >>> store
> > > >>>>>>> needs
> > > >>>>>>>>> to implement the state store interface. That means they need
> to
> > > >>>>>>>>> implement a wrapper around the actual state store as we do
> for
> > > >>>>> RocksDB
> > > >>>>>>>>> with RocksDBStore. So, a third-party state store can always
> > > >>> estimate
> > > >>>>>>> the
> > > >>>>>>>>> uncommitted bytes, if it wants, because the wrapper can
> record
> > > >>> the
> > > >>>>>>> added
> > > >>>>>>>>> bytes.
> > > >>>>>>>>> One case I can think of where returning -1 makes sense is
> when
> > > >>>>> Streams
> > > >>>>>>>>> does not need to estimate the size of the write batch and
> > > >>> trigger
> > > >>>>>>>>> extraordinary commits, because the third-party state store
> > > >>> takes care
> > > >>>>>>> of
> > > >>>>>>>>> memory. But in that case the method could also just return 0.
> > > >>> Even
> > > >>>>> that
> > > >>>>>>>>> case would be better solved with a method that returns
> whether
> > > >>> the
> > > >>>>>>> state
> > > >>>>>>>>> store manages itself the memory used for uncommitted bytes or
> > > >>> not.
> > > >>>>>>>>> Said that, I am fine with keeping the -1 return value, I was
> > > >>> just
> > > >>>>>>>>> wondering when and if it will be used.
> > > >>>>>>>>>
> > > >>>>>>>>> Regarding returning 0 for transactional state stores when the
> > > >>> batch
> > > >>>>> is
> > > >>>>>>>>> empty, I was just wondering because you explicitly stated
> > > >>>>>>>>>
> > > >>>>>>>>> "or {@code 0} if this StateStore does not support
> > transactions."
> > > >>>>>>>>>
> > > >>>>>>>>> So it seemed to me returning 0 could only happen for
> > > >>>>> non-transactional
> > > >>>>>>>>> state stores.
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> 3.
> > > >>>>>>>>>
> > > >>>>>>>>> a) What do you think if we move the isolation level to IQ (v1
> > > >>> and
> > > >>>>> v2)?
> > > >>>>>>>>> In the end this is the only component that really needs to
> > > >>> specify
> > > >>>>> the
> > > >>>>>>>>> isolation level. It is similar to the Kafka consumer that can
> > > >>> choose
> > > >>>>>>>>> with what isolation level to read the input topic.
> > > >>>>>>>>> For IQv1 the isolation level should go into
> > > >>> StoreQueryParameters. For
> > > >>>>>>>>> IQv2, I would add it to the Query interface.
> > > >>>>>>>>>
> > > >>>>>>>>> b) Point a) raises the question what should happen during
> > > >>>>> at-least-once
> > > >>>>>>>>> processing when the state store does not use transactions?
> John
> > > >>> in
> > > >>>>> the
> > > >>>>>>>>> past proposed to also use transactions on state stores for
> > > >>>>>>>>> at-least-once. I like that idea, because it avoids
> aggregating
> > > >>> the
> > > >>>>> same
> > > >>>>>>>>> records over and over again in the case of a failure. We had
> a
> > > >>> case
> > > >>>>> in
> > > >>>>>>>>> the past where a Streams applications in at-least-once mode
> was
> > > >>>>> failing
> > > >>>>>>>>> continuously for some reasons I do not remember before
> > > >>> committing the
> > > >>>>>>>>> offsets. After each failover, the app aggregated again and
> > > >>> again the
> > > >>>>>>>>> same records. Of course the aggregate increased to very wrong
> > > >>> values
> > > >>>>>>>>> just because of the failover. With transactions on the state
> > > >>> stores
> > > >>>>> we
> > > >>>>>>>>> could have avoided this. The app would have output the same
> > > >>> aggregate
> > > >>>>>>>>> multiple times (i.e., after each failover) but at least the
> > > >>> value of
> > > >>>>>>> the
> > > >>>>>>>>> aggregate would not depend on the number of failovers.
> > > >>> Outputting the
> > > >>>>>>>>> same aggregate multiple times would be incorrect under
> > > >>> exactly-once
> > > >>>>> but
> > > >>>>>>>>> it is OK for at-least-once.
> > > >>>>>>>>> If it makes sense to add a config to turn on and off
> > > >>> transactions on
> > > >>>>>>>>> state stores under at-least-once or just use transactions in
> > > >>> any case
> > > >>>>>>> is
> > > >>>>>>>>> a question we should also discuss in this KIP. It depends a
> bit
> > > >>> on
> > > >>>>> the
> > > >>>>>>>>> performance trade-off. Maybe to be safe, I would add a
> config.
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> 4.
> > > >>>>>>>>> Your points are all valid. I tend to say to keep the metrics
> > > >>> around
> > > >>>>>>>>> flush() until we remove flush() completely from the
> interface.
> > > >>> Calls
> > > >>>>> to
> > > >>>>>>>>> flush() might still exist since existing processors might
> still
> > > >>> call
> > > >>>>>>>>> flush() explicitly as you mentioned in 1). For sure, we need
> to
> > > >>>>>>> document
> > > >>>>>>>>> how the metrics change due to the transactions in the upgrade
> > > >>> notes.
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> 5.
> > > >>>>>>>>> I see. Then you should describe how the .position files are
> > > >>> handled
> > > >>>>> in
> > > >>>>>>>>> a dedicated section of the KIP or incorporate the description
> > > >>> in the
> > > >>>>>>>>> "Atomic Checkpointing" section instead of only mentioning it
> in
> > > >>> the
> > > >>>>>>>>> "Compatibility, 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:
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>
> > > >>>>>
> > > >>>
> > >
> >
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > >>>>>>>>>
> > > >>>>>>>>> It is a bit brittle, but it works.
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> Best,
> > > >>>>>>>>> Bruno
> > > >>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > > >>>>>>>>>> Hi Bruno,
> > > >>>>>>>>>>
> > > >>>>>>>>>> Thanks for taking the time to review the KIP. I'm back from
> > > >>> leave
> > > >>>>> now
> > > >>>>>>> and
> > > >>>>>>>>>> intend to move this forwards as quickly as I can.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Addressing your points:
> > > >>>>>>>>>>
> > > >>>>>>>>>> 1.
> > > >>>>>>>>>> Because flush() is part of the StateStore API, it's exposed
> to
> > > >>>>> custom
> > > >>>>>>>>>> Processors, which might be making calls to flush(). This was
> > > >>>>> actually
> > > >>>>>>> the
> > > >>>>>>>>>> case in a few integration tests.
> > > >>>>>>>>>> To maintain as much compatibility as possible, I'd prefer
> not
> > > >>> to
> > > >>>>> make
> > > >>>>>>>>> this
> > > >>>>>>>>>> an UnsupportedOperationException, as it will cause
> previously
> > > >>>>> working
> > > >>>>>>>>>> Processors to start throwing exceptions at runtime.
> > > >>>>>>>>>> I agree that it doesn't make sense for it to proxy commit(),
> > > >>> though,
> > > >>>>>>> as
> > > >>>>>>>>>> that would cause it to violate the "StateStores commit only
> > > >>> when the
> > > >>>>>>> Task
> > > >>>>>>>>>> commits" rule.
> > > >>>>>>>>>> Instead, I think we should make this a no-op. That way,
> > > >>> existing
> > > >>>>> user
> > > >>>>>>>>>> Processors will continue to work as-before, without
> violation
> > > >>> of
> > > >>>>> store
> > > >>>>>>>>>> consistency that would be caused by premature flush/commit
> of
> > > >>>>>>> StateStore
> > > >>>>>>>>>> data to disk.
> > > >>>>>>>>>> What do you think?
> > > >>>>>>>>>>
> > > >>>>>>>>>> 2.
> > > >>>>>>>>>> As stated in the JavaDoc, when a StateStore implementation
> is
> > > >>>>>>>>>> transactional, but is unable to estimate the uncommitted
> > memory
> > > >>>>> usage,
> > > >>>>>>>>> the
> > > >>>>>>>>>> method will return -1.
> > > >>>>>>>>>> The intention here is to permit third-party implementations
> > > >>> that may
> > > >>>>>>> not
> > > >>>>>>>>> be
> > > >>>>>>>>>> able to estimate memory usage.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Yes, it will be 0 when nothing has been written to the store
> > > >>> yet. I
> > > >>>>>>>>> thought
> > > >>>>>>>>>> that was implied by "This method will return an
> approximation
> > > >>> of the
> > > >>>>>>>>> memory
> > > >>>>>>>>>> would be freed by the next call to {@link #commit(Map)}" and
> > > >>>>> "@return
> > > >>>>>>> The
> > > >>>>>>>>>> approximate size of all records awaiting {@link
> > #commit(Map)}",
> > > >>>>>>> however,
> > > >>>>>>>>> I
> > > >>>>>>>>>> can add it explicitly to the JavaDoc if you think this is
> > > >>> unclear?
> > > >>>>>>>>>>
> > > >>>>>>>>>> 3.
> > > >>>>>>>>>> I realise this is probably the most contentious point in my
> > > >>> design,
> > > >>>>>>> and
> > > >>>>>>>>> I'm
> > > >>>>>>>>>> open to changing it if I'm unable to convince you of the
> > > >>> benefits.
> > > >>>>>>>>>> Nevertheless, here's my argument:
> > > >>>>>>>>>> The Interactive Query (IQ) API(s) are directly provided
> > > >>> StateStores
> > > >>>>> to
> > > >>>>>>>>>> query, and it may be important for users to programmatically
> > > >>> know
> > > >>>>>>> which
> > > >>>>>>>>>> mode the StateStore is operating under. If we simply provide
> > an
> > > >>>>>>>>>> "eosEnabled" boolean (as used throughout the internal
> streams
> > > >>>>>>> engine), or
> > > >>>>>>>>>> similar, then users will need to understand the operation
> and
> > > >>>>>>>>> consequences
> > > >>>>>>>>>> of each available processing mode and how it pertains to
> their
> > > >>>>>>>>> StateStore.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Interactive Query users aren't the only people that care
> about
> > > >>> the
> > > >>>>>>>>>> processing.mode/IsolationLevel of a StateStore: implementers
> > of
> > > >>>>> custom
> > > >>>>>>>>>> StateStores also need to understand the behaviour expected
> of
> > > >>> their
> > > >>>>>>>>>> implementation. KIP-892 introduces some assumptions into the
> > > >>> Streams
> > > >>>>>>>>> Engine
> > > >>>>>>>>>> about how StateStores operate under each processing mode,
> and
> > > >>> it's
> > > >>>>>>>>>> important that custom implementations adhere to those
> > > >>> assumptions in
> > > >>>>>>>>> order
> > > >>>>>>>>>> to maintain the consistency guarantees.
> > > >>>>>>>>>>
> > > >>>>>>>>>> IsolationLevels provide a high-level contract on the
> behaviour
> > > >>> of
> > > >>>>> the
> > > >>>>>>>>>> StateStore: a user knows that under READ_COMMITTED, they
> will
> > > >>> see
> > > >>>>>>> writes
> > > >>>>>>>>>> only after the Task has committed, and under
> READ_UNCOMMITTED
> > > >>> they
> > > >>>>>>> will
> > > >>>>>>>>> see
> > > >>>>>>>>>> writes immediately. No understanding of the details of each
> > > >>>>>>>>> processing.mode
> > > >>>>>>>>>> is required, either for IQ users or StateStore implementers.
> > > >>>>>>>>>>
> > > >>>>>>>>>> An argument can be made that these contractual guarantees
> can
> > > >>> simply
> > > >>>>>>> be
> > > >>>>>>>>>> documented for the processing.mode (i.e. that exactly-once
> and
> > > >>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and at-least-once
> > > >>> behaves
> > > >>>>>>> like
> > > >>>>>>>>>> READ_UNCOMMITTED), but there are several small issues with
> > > >>> this I'd
> > > >>>>>>>>> prefer
> > > >>>>>>>>>> to avoid:
> > > >>>>>>>>>>
> > > >>>>>>>>>>       - Where would we document these contracts, in a way
> that
> > > >>> is
> > > >>>>>>> difficult
> > > >>>>>>>>>>       for users/implementers to miss/ignore?
> > > >>>>>>>>>>       - It's not clear to users that the processing mode is
> > > >>>>>>> communicating
> > > >>>>>>>>>>       an expectation of read isolation, unless they read the
> > > >>>>>>>>> documentation. Users
> > > >>>>>>>>>>       rarely consult documentation unless they feel they
> need
> > > >>> to, so
> > > >>>>>>> it's
> > > >>>>>>>>> likely
> > > >>>>>>>>>>       this detail would get missed by many users.
> > > >>>>>>>>>>       - It tightly couples processing modes to read
> isolation.
> > > >>> Adding
> > > >>>>>>> new
> > > >>>>>>>>>>       processing modes, or changing the read isolation of
> > > >>> existing
> > > >>>>>>>>> processing
> > > >>>>>>>>>>       modes would be difficult/impossible.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Ultimately, the cost of introducing IsolationLevels is just
> a
> > > >>> single
> > > >>>>>>>>>> method, since we re-use the existing IsolationLevel enum
> from
> > > >>> Kafka.
> > > >>>>>>> This
> > > >>>>>>>>>> gives us a clear place to document the contractual
> guarantees
> > > >>>>> expected
> > > >>>>>>>>>> of/provided by StateStores, that is accessible both by the
> > > >>>>> StateStore
> > > >>>>>>>>>> itself, and by IQ users.
> > > >>>>>>>>>>
> > > >>>>>>>>>> (Writing this I've just realised that the StateStore and IQ
> > > >>> APIs
> > > >>>>>>> actually
> > > >>>>>>>>>> don't provide access to StateStoreContext that IQ users
> would
> > > >>> have
> > > >>>>>>> direct
> > > >>>>>>>>>> access to... Perhaps StateStore should expose
> isolationLevel()
> > > >>>>> itself
> > > >>>>>>>>> too?)
> > > >>>>>>>>>>
> > > >>>>>>>>>> 4.
> > > >>>>>>>>>> Yeah, I'm not comfortable renaming the metrics in-place
> > > >>> either, as
> > > >>>>>>> it's a
> > > >>>>>>>>>> backwards incompatible change. My concern is that, if we
> leave
> > > >>> the
> > > >>>>>>>>> existing
> > > >>>>>>>>>> "flush" metrics in place, they will be confusing to users.
> > > >>> Right
> > > >>>>> now,
> > > >>>>>>>>>> "flush" metrics record explicit flushes to disk, but under
> > > >>> KIP-892,
> > > >>>>>>> even
> > > >>>>>>>>> a
> > > >>>>>>>>>> commit() will not explicitly flush data to disk - RocksDB
> will
> > > >>>>> decide
> > > >>>>>>> on
> > > >>>>>>>>>> when to flush memtables to disk itself.
> > > >>>>>>>>>>
> > > >>>>>>>>>> If we keep the existing "flush" metrics, we'd have two
> > options,
> > > >>>>> which
> > > >>>>>>>>> both
> > > >>>>>>>>>> seem pretty bad to me:
> > > >>>>>>>>>>
> > > >>>>>>>>>>       1. Have them record calls to commit(), which would be
> > > >>>>>>> misleading, as
> > > >>>>>>>>>>       data is no longer explicitly "flushed" to disk by this
> > > >>> call.
> > > >>>>>>>>>>       2. Have them record nothing at all, which is
> equivalent
> > to
> > > >>>>>>> removing
> > > >>>>>>>>> the
> > > >>>>>>>>>>       metrics, except that users will see the metric still
> > > >>> exists and
> > > >>>>>>> so
> > > >>>>>>>>> assume
> > > >>>>>>>>>>       that the metric is correct, and that there's a problem
> > > >>> with
> > > >>>>> their
> > > >>>>>>>>> system
> > > >>>>>>>>>>       when there isn't.
> > > >>>>>>>>>>
> > > >>>>>>>>>> I agree that removing them is also a bad solution, and I'd
> > > >>> like some
> > > >>>>>>>>>> guidance on the best path forward here.
> > > >>>>>>>>>>
> > > >>>>>>>>>> 5.
> > > >>>>>>>>>> Position files are updated on every write to a StateStore.
> > > >>> Since our
> > > >>>>>>>>> writes
> > > >>>>>>>>>> are now buffered until commit(), we can't update the
> Position
> > > >>> file
> > > >>>>>>> until
> > > >>>>>>>>>> commit() has been called, otherwise it would be inconsistent
> > > >>> with
> > > >>>>> the
> > > >>>>>>>>> data
> > > >>>>>>>>>> in the event of a rollback. Consequently, we need to manage
> > > >>> these
> > > >>>>>>> offsets
> > > >>>>>>>>>> the same way we manage the checkpoint offsets, and ensure
> > > >>> they're
> > > >>>>> only
> > > >>>>>>>>>> written on commit().
> > > >>>>>>>>>>
> > > >>>>>>>>>> 6.
> > > >>>>>>>>>> Agreed, although I'm not exactly sure yet what tests to
> write.
> > > >>> How
> > > >>>>>>>>> explicit
> > > >>>>>>>>>> do we need to be here in the KIP?
> > > >>>>>>>>>>
> > > >>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be
> seamless,
> > > >>> and we
> > > >>>>>>>>> should
> > > >>>>>>>>>> definitely add some tests around that. Downgrade, it
> > > >>> transpires,
> > > >>>>> isn't
> > > >>>>>>>>>> currently possible, as the extra column family for offset
> > > >>> storage is
> > > >>>>>>>>>> incompatible with the pre-KIP-892 implementation: when you
> > > >>> open a
> > > >>>>>>> RocksDB
> > > >>>>>>>>>> database, you must open all available column families or
> > > >>> receive an
> > > >>>>>>>>> error.
> > > >>>>>>>>>> What currently happens on downgrade is that it attempts to
> > > >>> open the
> > > >>>>>>>>> store,
> > > >>>>>>>>>> throws an error about the offsets column family not being
> > > >>> opened,
> > > >>>>>>> which
> > > >>>>>>>>>> triggers a wipe and rebuild of the Task. Given that
> downgrades
> > > >>>>> should
> > > >>>>>>> be
> > > >>>>>>>>>> uncommon, I think this is acceptable behaviour, as the
> > > >>> end-state is
> > > >>>>>>>>>> consistent, even if it results in an undesirable state
> > restore.
> > > >>>>>>>>>>
> > > >>>>>>>>>> Should I document the upgrade/downgrade behaviour explicitly
> > > >>> in the
> > > >>>>>>> KIP?
> > > >>>>>>>>>>
> > > >>>>>>>>>> --
> > > >>>>>>>>>>
> > > >>>>>>>>>> Regards,
> > > >>>>>>>>>> Nick
> > > >>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > > >>> cadonna@apache.org>
> > > >>>>>>> wrote:
> > > >>>>>>>>>>
> > > >>>>>>>>>>> Hi Nick!
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Thanks for the updates!
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 1.
> > > >>>>>>>>>>> Why does StateStore#flush() default to
> > > >>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > > >>>>>>>>>>> Since calls to flush() will not exist anymore after this
> KIP
> > > >>> is
> > > >>>>>>>>>>> released, I would rather throw an unsupported operation
> > > >>> exception
> > > >>>>> by
> > > >>>>>>>>>>> default.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 2.
> > > >>>>>>>>>>> When would a state store return -1 from
> > > >>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while being
> > > >>>>>>> transactional?
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Wouldn't StateStore#approximateNumUncommittedBytes() also
> > > >>> return 0
> > > >>>>> if
> > > >>>>>>>>>>> the state store is transactional but nothing has been
> written
> > > >>> to
> > > >>>>> the
> > > >>>>>>>>>>> state store yet?
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 3.
> > > >>>>>>>>>>> Sorry for bringing this up again. Does this KIP really need
> > to
> > > >>>>>>> introduce
> > > >>>>>>>>>>> StateStoreContext#isolationLevel()? StateStoreContext has
> > > >>> already
> > > >>>>>>>>>>> appConfigs() which basically exposes the same information,
> > > >>> i.e., if
> > > >>>>>>> EOS
> > > >>>>>>>>>>> is enabled or not.
> > > >>>>>>>>>>> In one of your previous e-mails you wrote:
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> "My idea was to try to keep the StateStore interface as
> > > >>> loosely
> > > >>>>>>> coupled
> > > >>>>>>>>>>> from the Streams engine as possible, to give implementers
> > more
> > > >>>>>>> freedom,
> > > >>>>>>>>>>> and reduce the amount of internal knowledge required."
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> While I understand the intent, I doubt that it decreases
> the
> > > >>>>>>> coupling of
> > > >>>>>>>>>>> a StateStore interface and the Streams engine.
> READ_COMMITTED
> > > >>> only
> > > >>>>>>>>>>> applies to IQ but not to reads by processors. Thus,
> > > >>> implementers
> > > >>>>>>> need to
> > > >>>>>>>>>>> understand how Streams accesses the state stores.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> I would like to hear what others think about this.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> 4.
> > > >>>>>>>>>>> Great exposing new metrics for transactional state stores!
> > > >>>>> However, I
> > > >>>>>>>>>>> would prefer to add new metrics and deprecate (in the docs)
> > > >>> the 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 also need to adapt/add integration tests
> > > >>> besides
> > > >>>>> unit
> > > >>>>>>>>>>> tests. Additionally, we probably need integration or system
> > > >>> tests
> > > >>>>> to
> > > >>>>>>>>>>> verify that upgrades and downgrades between transactional
> and
> > > >>>>>>>>>>> non-transactional state stores work as expected.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Best,
> > > >>>>>>>>>>> Bruno
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > > >>>>>>>>>>>> One more thing: I noted John's suggestion in the KIP,
> under
> > > >>>>>>> "Rejected
> > > >>>>>>>>>>>> Alternatives". I still think it's an idea worth pursuing,
> > > >>> but I
> > > >>>>>>> believe
> > > >>>>>>>>>>>> that it's out of the scope of this KIP, because it solves
> a
> > > >>>>>>> different
> > > >>>>>>>>> set
> > > >>>>>>>>>>>> of problems to this KIP, and the scope of this one has
> > > >>> already
> > > >>>>> grown
> > > >>>>>>>>>>> quite
> > > >>>>>>>>>>>> large!
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > > >>>>> nick.telford@gmail.com>
> > > >>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>> Hi everyone,
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> I've updated the KIP (
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>
> > > >>>>>
> > > >>>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > >>>>>>>>>>> )
> > > >>>>>>>>>>>>> with the latest changes; mostly bringing back "Atomic
> > > >>>>>>> Checkpointing"
> > > >>>>>>>>>>> (for
> > > >>>>>>>>>>>>> what feels like the 10th time!). I think the one thing
> > > >>> missing is
> > > >>>>>>> some
> > > >>>>>>>>>>>>> changes to metrics (notably the store "flush" metrics
> will
> > > >>> need
> > > >>>>> to
> > > >>>>>>> be
> > > >>>>>>>>>>>>> renamed to "commit").
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> The reason I brought back Atomic Checkpointing was to
> > > >>> decouple
> > > >>>>>>> store
> > > >>>>>>>>>>> flush
> > > >>>>>>>>>>>>> from store commit. This is important, because with
> > > >>> Transactional
> > > >>>>>>>>>>>>> StateStores, we now need to call "flush" on *every* Task
> > > >>> commit,
> > > >>>>>>> and
> > > >>>>>>>>> not
> > > >>>>>>>>>>>>> just when the StateStore is closing, otherwise our
> > > >>> transaction
> > > >>>>>>> buffer
> > > >>>>>>>>>>> will
> > > >>>>>>>>>>>>> never be written and persisted, instead growing
> unbounded!
> > I
> > > >>>>>>>>>>> experimented
> > > >>>>>>>>>>>>> with some simple solutions, like forcing a store flush
> > > >>> whenever
> > > >>>>> the
> > > >>>>>>>>>>>>> transaction buffer was likely to exceed its configured
> > > >>> size, but
> > > >>>>>>> this
> > > >>>>>>>>>>> was
> > > >>>>>>>>>>>>> brittle: it prevented the transaction buffer from being
> > > >>>>> configured
> > > >>>>>>> to
> > > >>>>>>>>> be
> > > >>>>>>>>>>>>> unbounded, and it still would have required explicit
> > > >>> flushes of
> > > >>>>>>>>> RocksDB,
> > > >>>>>>>>>>>>> yielding sub-optimal performance and memory utilization.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right" way to
> > > >>> resolve
> > > >>>>> this
> > > >>>>>>>>>>>>> problem. By ensuring that the changelog offsets that
> > > >>> correspond
> > > >>>>> to
> > > >>>>>>> the
> > > >>>>>>>>>>> most
> > > >>>>>>>>>>>>> recently written records are always atomically written to
> > > >>> the
> > > >>>>>>>>> StateStore
> > > >>>>>>>>>>>>> (by writing them to the same transaction buffer), we can
> > > >>> avoid
> > > >>>>>>>>> forcibly
> > > >>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting RocksDB
> > > >>> flush
> > > >>>>> them
> > > >>>>>>>>> only
> > > >>>>>>>>>>>>> when necessary, without losing any of our consistency
> > > >>> guarantees.
> > > >>>>>>> See
> > > >>>>>>>>>>> the
> > > >>>>>>>>>>>>> updated KIP for more info.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> I have fully implemented these changes, although I'm
> still
> > > >>> not
> > > >>>>>>>>> entirely
> > > >>>>>>>>>>>>> happy with the implementation for segmented StateStores,
> so
> > > >>> I
> > > >>>>> plan
> > > >>>>>>> to
> > > >>>>>>>>>>>>> refactor that. Despite that, all tests pass. If you'd
> like
> > > >>> to try
> > > >>>>>>> out
> > > >>>>>>>>> or
> > > >>>>>>>>>>>>> review this highly experimental and incomplete branch,
> it's
> > > >>>>>>> available
> > > >>>>>>>>>>> here:
> > > >>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0.
> > > >>> Note:
> > > >>>>>>> it's
> > > >>>>>>>>>>> built
> > > >>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base to build
> > > >>> and test
> > > >>>>>>> it
> > > >>>>>>>>> on,
> > > >>>>>>>>>>>>> and to enable easy apples-to-apples comparisons in a live
> > > >>>>>>>>> environment. I
> > > >>>>>>>>>>>>> plan to rebase it against trunk once it's nearer
> completion
> > > >>> and
> > > >>>>> has
> > > >>>>>>>>> been
> > > >>>>>>>>>>>>> proven on our main application.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> I would really appreciate help in reviewing and testing:
> > > >>>>>>>>>>>>> - Segmented (Versioned, Session and Window) stores
> > > >>>>>>>>>>>>> - Global stores
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> As I do not currently use either of these, so my primary
> > > >>> test
> > > >>>>>>>>>>> environment
> > > >>>>>>>>>>>>> doesn't test these areas.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> I'm going on Parental Leave starting next week for a few
> > > >>> weeks,
> > > >>>>> so
> > > >>>>>>>>> will
> > > >>>>>>>>>>>>> not have time to move this forward until late August.
> That
> > > >>> said,
> > > >>>>>>> your
> > > >>>>>>>>>>>>> feedback is welcome and appreciated, I just won't be able
> > to
> > > >>>>>>> respond
> > > >>>>>>>>> as
> > > >>>>>>>>>>>>> quickly as usual.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > > >>>>> nick.telford@gmail.com>
> > > >>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Hi Bruno
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ is not
> > > >>> something
> > > >>>>> I
> > > >>>>>>> had
> > > >>>>>>>>>>>>>> considered.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> What about atomically updating the state store from the
> > > >>>>>>> transaction
> > > >>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
> > > >>> (thus,
> > > >>>>>>>>>>> flushing
> > > >>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> > > >>> number of
> > > >>>>>>>>> commit
> > > >>>>>>>>>>>>>>> intervals?
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting that we
> > > >>> add an
> > > >>>>>>>>>>> additional
> > > >>>>>>>>>>>>>> config for the max number of commit intervals between
> > > >>>>> checkpoints?
> > > >>>>>>>>> That
> > > >>>>>>>>>>>>>> way, we would checkpoint *either* when the transaction
> > > >>> buffers
> > > >>>>> are
> > > >>>>>>>>>>> nearly
> > > >>>>>>>>>>>>>> full, *OR* whenever a certain number of commit intervals
> > > >>> have
> > > >>>>>>>>> elapsed,
> > > >>>>>>>>>>>>>> whichever comes first?
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> That certainly seems reasonable, although this
> re-ignites
> > > >>> an
> > > >>>>>>> earlier
> > > >>>>>>>>>>>>>> debate about whether a config should be measured in
> > > >>> "number of
> > > >>>>>>> commit
> > > >>>>>>>>>>>>>> intervals", instead of just an absolute time.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> FWIW, I realised that this issue is the reason I was
> > > >>> pursuing
> > > >>>>> the
> > > >>>>>>>>>>> Atomic
> > > >>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush from
> > > >>> checkpointing,
> > > >>>>>>>>> which
> > > >>>>>>>>>>>>>> enables us to just checkpoint on every commit without
> any
> > > >>>>>>> performance
> > > >>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the "best"
> > > >>> solution,
> > > >>>>>>> but
> > > >>>>>>>>>>> I'm not
> > > >>>>>>>>>>>>>> sure if this is enough to bring it back into this KIP.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> I'm currently working on moving all the transactional
> > logic
> > > >>>>>>> directly
> > > >>>>>>>>>>> into
> > > >>>>>>>>>>>>>> RocksDBStore itself, which does away with the
> > > >>>>>>>>> StateStore#newTransaction
> > > >>>>>>>>>>>>>> method, and reduces the number of new classes
> introduced,
> > > >>>>>>>>> significantly
> > > >>>>>>>>>>>>>> reducing the complexity. If it works, and the complexity
> > is
> > > >>>>>>>>> drastically
> > > >>>>>>>>>>>>>> reduced, I may try bringing back Atomic Checkpoints into
> > > >>> this
> > > >>>>> KIP.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
> > > >>> cadonna@apache.org>
> > > >>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> As far as I understand, you want to atomically update
> the
> > > >>> state
> > > >>>>>>>>> store
> > > >>>>>>>>>>>>>>> from the transaction buffer, flush the memtable of a
> > state
> > > >>>>> store
> > > >>>>>>> and
> > > >>>>>>>>>>>>>>> write the checkpoint not after the commit time elapsed
> > but
> > > >>>>> after
> > > >>>>>>> the
> > > >>>>>>>>>>>>>>> transaction buffer reached a size that would lead to
> > > >>> exceeding
> > > >>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes before the next
> > > >>> commit
> > > >>>>>>>>>>> interval
> > > >>>>>>>>>>>>>>> ends.
> > > >>>>>>>>>>>>>>> That means, the Kafka transaction would commit every
> > > >>> commit
> > > >>>>>>> interval
> > > >>>>>>>>>>> but
> > > >>>>>>>>>>>>>>> the state store will only be atomically updated roughly
> > > >>> every
> > > >>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of data. Also
> IQ
> > > >>> would
> > > >>>>>>> then
> > > >>>>>>>>>>> only
> > > >>>>>>>>>>>>>>> see new data roughly every
> > > >>>>>>> statestore.transaction.buffer.max.bytes.
> > > >>>>>>>>>>>>>>> After a failure the state store needs to restore up to
> > > >>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Is this correct?
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> What about atomically updating the state store from the
> > > >>>>>>> transaction
> > > >>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
> > > >>> (thus,
> > > >>>>>>>>>>> flushing
> > > >>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> > > >>> number of
> > > >>>>>>>>> commit
> > > >>>>>>>>>>>>>>> intervals? In such a way, we would have the same delay
> > for
> > > >>>>>>> records
> > > >>>>>>>>>>>>>>> appearing in output topics and IQ because both would
> > > >>> appear
> > > >>>>> when
> > > >>>>>>> the
> > > >>>>>>>>>>>>>>> Kafka transaction is committed. However, after a
> failure
> > > >>> the
> > > >>>>>>> state
> > > >>>>>>>>>>> store
> > > >>>>>>>>>>>>>>> still needs to restore up to
> > > >>>>>>> statestore.transaction.buffer.max.bytes
> > > >>>>>>>>>>> and
> > > >>>>>>>>>>>>>>> it might restore data that is already in the state
> store
> > > >>>>> because
> > > >>>>>>> the
> > > >>>>>>>>>>>>>>> checkpoint lags behind the last stable offset (i.e. the
> > > >>> last
> > > >>>>>>>>> committed
> > > >>>>>>>>>>>>>>> offset) of the changelog topics. Restoring data that is
> > > >>> already
> > > >>>>>>> in
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>> state store is idempotent, so eos should not violated.
> > > >>>>>>>>>>>>>>> This solution needs at least one new config to specify
> > > >>> when a
> > > >>>>>>>>>>> checkpoint
> > > >>>>>>>>>>>>>>> should be written.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> A small correction to your previous e-mail that does
> not
> > > >>> change
> > > >>>>>>>>>>> anything
> > > >>>>>>>>>>>>>>> you said: Under alos the default commit interval is 30
> > > >>> seconds,
> > > >>>>>>> not
> > > >>>>>>>>>>> five
> > > >>>>>>>>>>>>>>> seconds.
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > > >>>>>>>>>>>>>>>> Hi everyone,
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> I've begun performance testing my branch on our
> staging
> > > >>>>>>>>> environment,
> > > >>>>>>>>>>>>>>>> putting it through its paces in our non-trivial
> > > >>> application.
> > > >>>>> I'm
> > > >>>>>>>>>>>>>>> already
> > > >>>>>>>>>>>>>>>> observing the same increased flush rate that we saw
> the
> > > >>> last
> > > >>>>>>> time
> > > >>>>>>>>> we
> > > >>>>>>>>>>>>>>>> attempted to use a version of this KIP, but this
> time, I
> > > >>>>> think I
> > > >>>>>>>>> know
> > > >>>>>>>>>>>>>>> why.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at
> > > >>> the end
> > > >>>>>>> of
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>> Task
> > > >>>>>>>>>>>>>>>> commit process, has the following behaviour:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>         - Under ALOS: checkpoint the state stores.
> This
> > > >>>>> includes
> > > >>>>>>>>>>>>>>>>         flushing memtables in RocksDB. This is
> > acceptable
> > > >>>>>>> because the
> > > >>>>>>>>>>>>>>> default
> > > >>>>>>>>>>>>>>>>         commit.interval.ms is 5 seconds, so forcibly
> > > >>> flushing
> > > >>>>>>>>> memtables
> > > >>>>>>>>>>>>>>> every 5
> > > >>>>>>>>>>>>>>>>         seconds is acceptable for most applications.
> > > >>>>>>>>>>>>>>>>         - Under EOS: checkpointing is not done,
> *unless*
> > > >>> it's
> > > >>>>>>> being
> > > >>>>>>>>>>>>>>> forced, due
> > > >>>>>>>>>>>>>>>>         to e.g. the Task closing or being revoked.
> This
> > > >>> means
> > > >>>>>>> that
> > > >>>>>>>>> under
> > > >>>>>>>>>>>>>>> normal
> > > >>>>>>>>>>>>>>>>         processing conditions, the state stores will
> not
> > > >>> be
> > > >>>>>>>>>>> checkpointed,
> > > >>>>>>>>>>>>>>> and will
> > > >>>>>>>>>>>>>>>>         not have memtables flushed at all , unless
> > RocksDB
> > > >>>>>>> decides to
> > > >>>>>>>>>>>>>>> flush them on
> > > >>>>>>>>>>>>>>>>         its own. Checkpointing stores and
> force-flushing
> > > >>> their
> > > >>>>>>>>> memtables
> > > >>>>>>>>>>>>>>> is only
> > > >>>>>>>>>>>>>>>>         done when a Task is being closed.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at
> > least
> > > >>>>> *some*
> > > >>>>>>>>>>> normal
> > > >>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB
> transaction
> > > >>>>> buffers
> > > >>>>>>> to
> > > >>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> state stores, and to ensure the offsets are synced to
> > > >>> disk to
> > > >>>>>>>>> prevent
> > > >>>>>>>>>>>>>>>> restores from getting out of hand. Consequently, my
> > > >>> current
> > > >>>>>>>>>>>>>>> implementation
> > > >>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is
> > > >>> far too
> > > >>>>>>>>>>>>>>> frequent.
> > > >>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records, which
> is a
> > > >>>>> change
> > > >>>>>>> in
> > > >>>>>>>>>>>>>>> flush
> > > >>>>>>>>>>>>>>>> behaviour, potentially causing performance problems
> for
> > > >>> some
> > > >>>>>>>>>>>>>>> applications.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm currently
> > > >>> leaning
> > > >>>>>>>>>>> towards
> > > >>>>>>>>>>>>>>>> using the statestore.transaction.buffer.max.bytes
> > > >>>>> configuration
> > > >>>>>>> to
> > > >>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to exceed it. This
> > > >>> would
> > > >>>>>>>>>>>>>>> complement the
> > > >>>>>>>>>>>>>>>> existing "early Task commit" functionality that this
> > > >>>>>>> configuration
> > > >>>>>>>>>>>>>>>> provides, in the following way:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>         - Currently, we use
> > > >>>>>>> statestore.transaction.buffer.max.bytes
> > > >>>>>>>>> to
> > > >>>>>>>>>>>>>>> force an
> > > >>>>>>>>>>>>>>>>         early Task commit if processing more records
> > would
> > > >>>>> cause
> > > >>>>>>> our
> > > >>>>>>>>>>> state
> > > >>>>>>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>         transactions to exceed the memory assigned to
> > > >>> them.
> > > >>>>>>>>>>>>>>>>         - New functionality: when a Task *does*
> commit,
> > > >>> we will
> > > >>>>>>> not
> > > >>>>>>>>>>>>>>> checkpoint
> > > >>>>>>>>>>>>>>>>         the stores (and hence flush the transaction
> > > >>> buffers)
> > > >>>>>>> unless
> > > >>>>>>>>> we
> > > >>>>>>>>>>>>>>> expect to
> > > >>>>>>>>>>>>>>>>         cross the
> > statestore.transaction.buffer.max.bytes
> > > >>>>>>> threshold
> > > >>>>>>>>>>> before
> > > >>>>>>>>>>>>>>> the next
> > > >>>>>>>>>>>>>>>>         commit
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> I'm also open to suggestions.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> > > >>>>>>> nick.telford@gmail.com
> > > >>>>>>>>>>
> > > >>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Hi Bruno!
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>> By "less predictable for users", I meant in terms of
> > > >>>>>>> understanding
> > > >>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> performance profile under various circumstances. The
> > > >>> more
> > > >>>>>>> complex
> > > >>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> solution, the more difficult it would be for users to
> > > >>>>>>> understand
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> performance they see. For example, spilling records
> to
> > > >>> disk
> > > >>>>>>> when
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> transaction buffer reaches a threshold would, I
> expect,
> > > >>>>> reduce
> > > >>>>>>>>> write
> > > >>>>>>>>>>>>>>>>> throughput. This reduction in write throughput could
> be
> > > >>>>>>>>> unexpected,
> > > >>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>> potentially difficult to diagnose/understand for
> users.
> > > >>>>>>>>>>>>>>>>> At the moment, I think the "early commit" concept is
> > > >>>>> relatively
> > > >>>>>>>>>>>>>>>>> straightforward; it's easy to document, and
> > conceptually
> > > >>>>> fairly
> > > >>>>>>>>>>>>>>> obvious to
> > > >>>>>>>>>>>>>>>>> users. We could probably add a metric to make it
> easier
> > > >>> to
> > > >>>>>>>>>>> understand
> > > >>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>> it happens though.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> 3. (the second one)
> > > >>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an indirect way
> of
> > > >>>>> telling
> > > >>>>>>>>>>>>>>> StateStores
> > > >>>>>>>>>>>>>>>>> whether they should be transactional. READ_COMMITTED
> > > >>>>>>> essentially
> > > >>>>>>>>>>>>>>> requires
> > > >>>>>>>>>>>>>>>>> transactions, because it dictates that two threads
> > > >>> calling
> > > >>>>>>>>>>>>>>>>> `newTransaction()` should not see writes from the
> other
> > > >>>>>>>>> transaction
> > > >>>>>>>>>>>>>>> until
> > > >>>>>>>>>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all
> > > >>> bets are
> > > >>>>>>> off,
> > > >>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>> stores can allow threads to observe written records
> at
> > > >>> any
> > > >>>>>>> time,
> > > >>>>>>>>>>>>>>> which is
> > > >>>>>>>>>>>>>>>>> essentially "no transactions". That said, StateStores
> > > >>> are
> > > >>>>> free
> > > >>>>>>> to
> > > >>>>>>>>>>>>>>> implement
> > > >>>>>>>>>>>>>>>>> these guarantees however they can, which is a bit
> more
> > > >>>>> relaxed
> > > >>>>>>>>> than
> > > >>>>>>>>>>>>>>>>> dictating "you must use transactions". For example,
> > with
> > > >>>>>>> RocksDB
> > > >>>>>>>>> we
> > > >>>>>>>>>>>>>>> would
> > > >>>>>>>>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
> > > >>>>> "transactions",
> > > >>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the database.
> But
> > > >>> with
> > > >>>>>>> other
> > > >>>>>>>>>>>>>>> storage
> > > >>>>>>>>>>>>>>>>> engines, it might be preferable to *always* use
> > > >>> transactions,
> > > >>>>>>> even
> > > >>>>>>>>>>>>>>> when
> > > >>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines that
> don't
> > > >>>>> provide
> > > >>>>>>>>>>>>>>>>> transactions, but the isolation guarantees can be met
> > > >>> using a
> > > >>>>>>>>>>>>>>> different
> > > >>>>>>>>>>>>>>>>> technique.
> > > >>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore interface
> as
> > > >>>>> loosely
> > > >>>>>>>>>>> coupled
> > > >>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > > >>> implementers
> > > >>>>> more
> > > >>>>>>>>>>>>>>> freedom, and
> > > >>>>>>>>>>>>>>>>> reduce the amount of internal knowledge required.
> > > >>>>>>>>>>>>>>>>> That said, I understand that "IsolationLevel" might
> not
> > > >>> be
> > > >>>>> the
> > > >>>>>>>>> right
> > > >>>>>>>>>>>>>>>>> abstraction, and we can always make it much more
> > > >>> explicit if
> > > >>>>>>>>>>>>>>> required, e.g.
> > > >>>>>>>>>>>>>>>>> boolean transactional()
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> 7-8.
> > > >>>>>>>>>>>>>>>>> I can make these changes either later today or
> > tomorrow.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Small update:
> > > >>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of
> > > >>> issues
> > > >>>>>>> that
> > > >>>>>>>>>>>>>>> needed
> > > >>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass, which is
> > > >>>>> promising,
> > > >>>>>>> but
> > > >>>>>>>>>>> it
> > > >>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>> need to undergo some performance testing. I haven't
> > > >>> (yet)
> > > >>>>>>> worked
> > > >>>>>>>>> on
> > > >>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I would
> > > >>> expect
> > > >>>>> that,
> > > >>>>>>>>>>>>>>>>> behaviourally, it should make no difference. The
> branch
> > > >>> is
> > > >>>>>>>>> available
> > > >>>>>>>>>>>>>>> at
> > > >>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c
> if
> > > >>>>> anyone
> > > >>>>>>> is
> > > >>>>>>>>>>>>>>>>> interested in taking an early look.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> > > >>>>>>> cadonna@apache.org>
> > > >>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Hi Nick,
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually also my
> > > >>> point. I
> > > >>>>>>>>>>> understood
> > > >>>>>>>>>>>>>>>>>> that John was proposing the ingestion path as a way
> to
> > > >>> avoid
> > > >>>>>>> the
> > > >>>>>>>>>>>>>>> early
> > > >>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>>>>> I agree with John here, that actually it is public
> > > >>> API. My
> > > >>>>>>>>> question
> > > >>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>> how this usage pattern affects normal processing.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>> My concern is that checking for the size of the
> > > >>> transaction
> > > >>>>>>>>> buffer
> > > >>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>> maybe triggering an early commit affects the whole
> > > >>>>> processing
> > > >>>>>>> of
> > > >>>>>>>>>>>>>>> Kafka
> > > >>>>>>>>>>>>>>>>>> Streams. The transactionality of a state store is
> not
> > > >>>>>>> confined to
> > > >>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> state store itself, but spills over and changes the
> > > >>> behavior
> > > >>>>>>> of
> > > >>>>>>>>>>> other
> > > >>>>>>>>>>>>>>>>>> parts of the system. I agree with you that it is a
> > > >>> decent
> > > >>>>>>>>>>>>>>> compromise. I
> > > >>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and list the
> > > >>> options to
> > > >>>>>>>>>>> overcome
> > > >>>>>>>>>>>>>>>>>> them. I also agree with you that all options seem
> > quite
> > > >>>>> heavy
> > > >>>>>>>>>>>>>>> compared
> > > >>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you mean
> with
> > > >>> "less
> > > >>>>>>>>>>>>>>> predictable
> > > >>>>>>>>>>>>>>>>>> for users", though.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> I found the discussions about the alternatives
> really
> > > >>>>>>>>> interesting.
> > > >>>>>>>>>>>>>>> But I
> > > >>>>>>>>>>>>>>>>>> also think that your plan sounds good and we should
> > > >>> continue
> > > >>>>>>> with
> > > >>>>>>>>>>> it!
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on June
> 20th:
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind putting
> > > >>> isolation
> > > >>>>>>>>> level
> > > >>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>>> the state store context. Thanks! Should that also
> be a
> > > >>> way
> > > >>>>> to
> > > >>>>>>>>> give
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> the state store the opportunity to decide whether to
> > > >>> turn on
> > > >>>>>>>>>>>>>>>>>> transactions or not?
> > > >>>>>>>>>>>>>>>>>> With my comment, I was more concerned about how do
> you
> > > >>> know
> > > >>>>>>> if a
> > > >>>>>>>>>>>>>>>>>> checkpoint file needs to be written under EOS, if
> you
> > > >>> do not
> > > >>>>>>>>> have a
> > > >>>>>>>>>>>>>>> way
> > > >>>>>>>>>>>>>>>>>> to know if the state store is transactional or not.
> If
> > > >>> a
> > > >>>>> state
> > > >>>>>>>>>>> store
> > > >>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be written
> > > >>> during
> > > >>>>>>> normal
> > > >>>>>>>>>>>>>>>>>> processing under EOS. If the state store is not
> > > >>>>> transactional,
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> checkpoint file must not be written under EOS.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> 7.
> > > >>>>>>>>>>>>>>>>>> My point was about not only considering the bytes in
> > > >>> memory
> > > >>>>> in
> > > >>>>>>>>>>> config
> > > >>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes
> that
> > > >>> might
> > > >>>>> be
> > > >>>>>>>>>>>>>>> spilled
> > > >>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering whether you
> should
> > > >>>>> remove
> > > >>>>>>> the
> > > >>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory bytes to be
> used
> > > >>> to
> > > >>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My thinking
> > > >>> was
> > > >>>>> that
> > > >>>>>>>>> even
> > > >>>>>>>>>>>>>>> if a
> > > >>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to disk,
> limiting
> > > >>> the
> > > >>>>>>>>> overall
> > > >>>>>>>>>>>>>>> bytes
> > > >>>>>>>>>>>>>>>>>> might make sense. Thinking about it again and
> > > >>> considering
> > > >>>>> the
> > > >>>>>>>>>>> recent
> > > >>>>>>>>>>>>>>>>>> discussions, it does not make too much sense
> anymore.
> > > >>>>>>>>>>>>>>>>>> I like the name
> > > >>> statestore.transaction.buffer.max.bytes that
> > > >>>>>>> you
> > > >>>>>>>>>>>>>>> proposed.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> 8.
> > > >>>>>>>>>>>>>>>>>> A high-level description (without implementation
> > > >>> details) of
> > > >>>>>>> how
> > > >>>>>>>>>>>>>>> Kafka
> > > >>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
> > > >>> transactions,
> > > >>>>>>> state
> > > >>>>>>>>>>> store
> > > >>>>>>>>>>>>>>>>>> transactions and checkpointing would be great. Would
> > be
> > > >>>>> great
> > > >>>>>>> if
> > > >>>>>>>>>>> you
> > > >>>>>>>>>>>>>>>>>> could also add some sentences about the behavior in
> > > >>> case of
> > > >>>>> a
> > > >>>>>>>>>>>>>>> failure.
> > > >>>>>>>>>>>>>>>>>> For instance how does a transactional state store
> > > >>> recover
> > > >>>>>>> after a
> > > >>>>>>>>>>>>>>>>>> failure or what happens with the transaction buffer,
> > > >>> etc.
> > > >>>>>>> (that
> > > >>>>>>>>> is
> > > >>>>>>>>>>>>>>> what
> > > >>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Best,
> > > >>>>>>>>>>>>>>>>>> Bruno
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > > >>>>>>>>>>>>>>>>>>> Hi Bruno,
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> 1.
> > > >>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> > > >>> WriteBatchWithIndex
> > > >>>>>>>>>>>>>>> transactions
> > > >>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
> > > >>> configured
> > > >>>>>>> memory
> > > >>>>>>>>>>>>>>> needs to
> > > >>>>>>>>>>>>>>>>>>> trigger an early commit?
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> 2.
> > > >>>>>>>>>>>>>>>>>>> This is one of my big concerns. Ultimately, any
> > > >>> approach
> > > >>>>>>> based
> > > >>>>>>>>> on
> > > >>>>>>>>>>>>>>>>>> cracking
> > > >>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in ways it's
> not
> > > >>> really
> > > >>>>>>>>>>> designed
> > > >>>>>>>>>>>>>>>>>> for is
> > > >>>>>>>>>>>>>>>>>>> likely to have some unforseen performance or
> > > >>> consistency
> > > >>>>>>> issues.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> 3.
> > > >>>>>>>>>>>>>>>>>>> What's your motivation for removing these early
> > > >>> commits?
> > > >>>>>>> While
> > > >>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>> ideal, I
> > > >>>>>>>>>>>>>>>>>>> think they're a decent compromise to ensure
> > > >>> consistency
> > > >>>>>>> whilst
> > > >>>>>>>>>>>>>>>>>> maintaining
> > > >>>>>>>>>>>>>>>>>>> good and predictable performance.
> > > >>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
> > > >>> complicated, and
> > > >>>>>>> might
> > > >>>>>>>>>>>>>>>>>> actually
> > > >>>>>>>>>>>>>>>>>>> make behaviour less predictable for users as a
> > > >>> consequence.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is
> > > >>> growing a
> > > >>>>>>> bit
> > > >>>>>>>>>>> out
> > > >>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas for
> future
> > > >>>>>>>>>>> improvements, I
> > > >>>>>>>>>>>>>>>>>> think
> > > >>>>>>>>>>>>>>>>>>> it's important to narrow the scope down to a design
> > > >>> that
> > > >>>>>>>>> achieves
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> most
> > > >>>>>>>>>>>>>>>>>>> pressing objectives (constant sized restorations
> > > >>> during
> > > >>>>> dirty
> > > >>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that this KIP
> > > >>> produces
> > > >>>>>>> can
> > > >>>>>>>>>>>>>>>>>> ultimately
> > > >>>>>>>>>>>>>>>>>>> be changed in the future, especially if the bulk of
> > > >>> it is
> > > >>>>>>>>> internal
> > > >>>>>>>>>>>>>>>>>>> behaviour.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> I'm going to spend some time next week trying to
> > > >>> re-work
> > > >>>>> the
> > > >>>>>>>>>>>>>>> original
> > > >>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
> > > >>> newTransaction()
> > > >>>>>>>>> method,
> > > >>>>>>>>>>>>>>> such
> > > >>>>>>>>>>>>>>>>>> that
> > > >>>>>>>>>>>>>>>>>>> it's just an implementation detail of RocksDBStore.
> > > >>> That
> > > >>>>>>> way, if
> > > >>>>>>>>>>> we
> > > >>>>>>>>>>>>>>>>>> want to
> > > >>>>>>>>>>>>>>>>>>> replace WBWI with something in the future, like the
> > > >>> SST
> > > >>>>> file
> > > >>>>>>>>>>>>>>> management
> > > >>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with little/no
> > API
> > > >>>>>>> changes.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Regards,
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Nick
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>
> > > >>
> > > >
> > >
> >
>

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

Posted by Sophie Blee-Goldman <ab...@gmail.com>.
Hey Nick! First of all thanks for taking up this awesome feature, I'm sure
every single
Kafka Streams user and dev would agree that it is sorely needed.

I've just been catching up on the KIP and surrounding discussion, so please
forgive me
for any misunderstandings or misinterpretations of the current plan and
don't hesitate to
correct me.

Before I jump in, I just want to say that having seen this drag on for so
long, my singular
goal in responding is to help this KIP past a perceived impasse so we can
finally move on
to voting and implementing it. Long discussions are to be expected for
major features like
this but it's completely on us as the Streams devs to make sure there is an
end in sight
for any ongoing discussion.

With that said, it's my understanding that the KIP as currently proposed is
just not tenable
for Kafka Streams, and would prevent some EOS users from upgrading to the
version it
first appears in. Given that we can't predict or guarantee whether any of
the followup KIPs
would be completed in the same release cycle as this one, we need to make
sure that the
feature is either compatible with all current users or else feature-flagged
so that they may
opt in/out.

Therefore, IIUC we need to have either (or both) of these as
fully-implemented config options:
1. default.state.isolation.level
2. enable.transactional.state.stores

This way EOS users for whom read_committed semantics are not viable can
still upgrade,
and either use the isolation.level config to leverage the new txn state
stores without sacrificing
their application semantics, or else simply keep the transactional state
stores disabled until we
are able to fully implement the isolation level configuration at either an
application or query level.

Frankly you are the expert here and know much more about the tradeoffs in
both semantics and
effort level of implementing one of these configs vs the other. In my
opinion, either option would
be fine and I would leave the decision of which one to include in this KIP
completely up to you.
I just don't see a way for the KIP to proceed without some variation of the
above that would allow
EOS users to opt-out of read_committed.

(If it's all the same to you, I would recommend 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 that gets released.
So personally, my choice would be to add the feature flag and leave it off
by default. If all 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

On Tue, Sep 19, 2023 at 1:57 AM Nick Telford <ni...@gmail.com> wrote:

> Hi Bruno,
>
> Agreed, I can live with that for now.
>
> In an effort to keep the scope of this KIP from expanding, I'm leaning
> towards just providing a configurable default.state.isolation.level and
> removing IsolationLevel from the StateStoreContext. This would be
> compatible with adding support for query-time IsolationLevels in the
> future, whilst providing a way for users to select an isolation level now.
>
> The big problem with this, however, is that if a user selects
> processing.mode
> = "exactly-once(-v2|-beta)", and default.state.isolation.level =
> "READ_UNCOMMITTED", we need to guarantee that the data isn't written to
> disk until commit() is called, but we also need to permit IQ threads to
> read from the ongoing transaction.
>
> A simple solution would be to (temporarily) forbid this combination of
> configuration, and have default.state.isolation.level automatically switch
> to READ_COMMITTED when processing.mode is anything other than
> at-least-once. Do you think this would be acceptable?
>
> In a later KIP, we can add support for query-time isolation levels and
> solve this particular problem there, which would relax this restriction.
>
> Regards,
> Nick
>
> On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <ca...@apache.org> 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,
> > Bruno
> >
> > On 9/18/23 7:12 PM, Nick Telford wrote:
> > > 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 <ni...@gmail.com>
> > wrote:
> > >
> > >> Hi everyone,
> > >>
> > >> I agree that having IsolationLevel be determined at query-time is the
> > >> ideal design, but there are a few sticking points:
> > >>
> > >> 1.
> > >> There needs to be some way to communicate the IsolationLevel down to
> the
> > >> RocksDBStore itself, so that the query can respect it. Since stores
> are
> > >> "layered" in functionality (i.e. ChangeLoggingStore, MeteredStore,
> > etc.),
> > >> we need some way to deliver that information to the bottom layer. For
> > IQv2,
> > >> we can use the existing State#query() method, but IQv1 has no way to
> do
> > >> this.
> > >>
> > >> A simple approach, which would potentially open up other options,
> would
> > be
> > >> to add something like: ReadOnlyKeyValueStore<K, V>
> > >> readOnlyView(IsolationLevel isolationLevel) to ReadOnlyKeyValueStore
> > (and
> > >> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
> > >>
> > >> 2.
> > >> As mentioned above, RocksDB WriteBatches are not thread-safe, which
> > causes
> > >> a problem if we want to provide READ_UNCOMMITTED Iterators. I also
> had a
> > >> look at RocksDB Transactions[1], but they solve a very different
> > problem,
> > >> and have the same thread-safety issue.
> > >>
> > >> One possible approach that I mentioned is chaining WriteBatches: every
> > >> time a new Interactive Query is received (i.e. readOnlyView, see
> above,
> > >> is called) we "freeze" the existing WriteBatch, and start a new one
> for
> > new
> > >> writes. The Interactive Query queries the "chain" of previous
> > WriteBatches
> > >> + the underlying database; while the StreamThread starts writing to
> the
> > >> *new* WriteBatch. On-commit, the StreamThread would write *all*
> > >> WriteBatches in the chain to the database (that have not yet been
> > written).
> > >>
> > >> WriteBatches would be closed/freed only when they have been both
> > >> committed, and all open Interactive Queries on them have been closed.
> > This
> > >> would require some reference counting.
> > >>
> > >> Obviously a drawback of this approach is the potential for increased
> > >> memory usage: if an Interactive Query is long-lived, for example by
> > doing a
> > >> full scan over a large database, or even just pausing in the middle of
> > an
> > >> iteration, then the existing chain of WriteBatches could be kept
> around
> > for
> > >> a long time, potentially forever.
> > >>
> > >> --
> > >>
> > >> A.
> > >> Going off on a tangent, it looks like in addition to supporting
> > >> READ_COMMITTED queries, we could go further and support
> REPEATABLE_READ
> > >> queries (i.e. where subsequent reads to the same key in the same
> > >> Interactive Query are guaranteed to yield the same value) by making
> use
> > of
> > >> RocksDB Snapshots[2]. These are fairly lightweight, so the performance
> > >> impact is likely to be negligible, but they do require that the
> > Interactive
> > >> Query session can be explicitly closed.
> > >>
> > >> This could be achieved if we made the above readOnlyView interface
> look
> > >> more like:
> > >>
> > >> interface ReadOnlyKeyValueView<K, V> implements
> ReadOnlyKeyValueStore<K,
> > >> V>, AutoCloseable {}
> > >>
> > >> interface ReadOnlyKeyValueStore<K, V> {
> > >>      ...
> > >>      ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel
> > isolationLevel);
> > >> }
> > >>
> > >> But this would be a breaking change, as existing IQv1 queries are
> > >> guaranteed to never call store.close(), and therefore these would leak
> > >> memory under REPEATABLE_READ.
> > >>
> > >> B.
> > >> One thing that's notable: MyRocks states that they support
> > READ_COMMITTED
> > >> and REPEATABLE_READ, but they make no mention of
> READ_UNCOMMITTED[3][4].
> > >> This could be because doing so is technically difficult/impossible
> using
> > >> the primitives available in RocksDB.
> > >>
> > >> --
> > >>
> > >> Lucas, to address your points:
> > >>
> > >> U1.
> > >> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
> > >> implementations of StateStore that do not support atomic writes.
> > Obviously
> > >> in those cases, the guarantees Kafka Streams provides/expects would be
> > >> relaxed. Do you think we should require all implementations to support
> > >> atomic writes?
> > >>
> > >> U2.
> > >> Stores can support multiple IsolationLevels. As we've discussed above,
> > the
> > >> ideal scenario would be to specify the IsolationLevel at query-time.
> > >> Failing that, I think the second-best approach is to define the
> > >> IsolationLevel for *all* queries based on the processing.mode, which
> is
> > >> what the default StateStoreContext#isolationLevel() achieves. Would
> you
> > >> prefer an alternative?
> > >>
> > >> While the existing implementation is equivalent to READ_UNCOMMITTED,
> > this
> > >> can yield unexpected results/errors under EOS, if a transaction is
> > rolled
> > >> back. While this would be a change in behaviour for users, it would
> look
> > >> more like a bug fix than a breaking change. That said, we *could* make
> > it
> > >> configurable, and default to the existing behaviour (READ_UNCOMMITTED)
> > >> instead of inferring it from the processing.mode?
> > >>
> > >> N1, N2.
> > >> These were only primitives to avoid boxing costs, but since this is
> not
> > a
> > >> performance sensitive area, it should be fine to change if that's
> > desirable.
> > >>
> > >> N3.
> > >> It's because the store "manages its own offsets", which includes both
> > >> committing the offset, *and providing it* via getCommittedOffset().
> > >> Personally, I think "managesOffsets" conveys this best, but I don't
> mind
> > >> changing it if the nomenclature is unclear.
> > >>
> > >> Sorry for the massive emails/essays!
> > >> --
> > >> Nick
> > >>
> > >> 1: https://github.com/facebook/rocksdb/wiki/Transactions
> > >> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> > >> 3: https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> > >> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
> > >>
> > >> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> > >> <lb...@confluent.io.invalid> wrote:
> > >>
> > >>> Hi Nick,
> > >>>
> > >>> since I last read it in April, the KIP has become much cleaner and
> > >>> easier to read. Great work!
> > >>>
> > >>> It feels to me the last big open point is whether we can implement
> > >>> isolation level as a query parameter. I understand that there are
> > >>> implementation concerns, but as Colt says, it would be a great
> > >>> addition, and would also simplify the migration path for this change.
> > >>> Is the implementation problem you mentioned caused by the WriteBatch
> > >>> not having a notion of a snapshot, as the underlying DB iterator
> does?
> > >>> In that case, I am not sure a chain of WriteBatches as you propose
> > >>> would fully solve the problem, but maybe I didn't dig enough into the
> > >>> details to fully understand it.
> > >>>
> > >>> If it's not possible to implement it now, would it be an option to
> > >>> make sure in this KIP that we do not fully close the door on
> per-query
> > >>> isolation levels in the interface, as it may be possible to implement
> > >>> the missing primitives in RocksDB or Speedb in the future.
> > >>>
> > >>> Understanding:
> > >>>
> > >>> * U1) Why is it only "SHOULD" for changelogOffsets to be persisted
> > >>> atomically with the records?
> > >>> * U2) Don't understand the default implementation of
> `isolationLevel`.
> > >>> The isolation level should be a property of the underlying store, and
> > >>> not be defined by the default config? Existing stores probably don't
> > >>> guarantee READ_COMMITTED, so the default should be to return
> > >>> READ_UNCOMMITTED.
> > >>>
> > >>> Nits:
> > >>> * N1) Could `getComittedOffset` use an `OptionalLong` return type, to
> > >>> avoid the `null`?
> > >>> * N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
> > >>> return type, to avoid the `-1`?
> > >>> * N3) I don't understand why `managesOffsets` uses the 'manage' verb,
> > >>> whereas all other methods use the "commits" verb. I'd suggest
> > >>> `commitsOffsets`.
> > >>>
> > >>> Either way, it feels this KIP is very close to the finish line, I'm
> > >>> looking forward to seeing this in production!
> > >>>
> > >>> Cheers,
> > >>> Lucas
> > >>>
> > >>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <co...@littlehorse.io>
> > wrote:
> > >>>>
> > >>>>> Making IsolationLevel a query-time constraint, rather than linking
> it
> > >>> to
> > >>>> the processing.guarantee.
> > >>>>
> > >>>> As I understand it, would this allow even a user of EOS to control
> > >>> whether
> > >>>> reading committed or uncommitted records? If so, I am highly in
> favor
> > of
> > >>>> this.
> > >>>>
> > >>>> I know that I was one of the early people to point out the current
> > >>>> shortcoming that IQ reads uncommitted records, but just this
> morning I
> > >>>> realized a pattern we use which means that (for certain queries) our
> > >>> system
> > >>>> needs to be able to read uncommitted records, which is the current
> > >>> behavior
> > >>>> of Kafka Streams in EOS.***
> > >>>>
> > >>>> If IsolationLevel being a query-time decision allows for this, then
> > that
> > >>>> would be amazing. I would also vote that the default behavior should
> > be
> > >>> for
> > >>>> reading uncommitted records, because it is totally possible for a
> > valid
> > >>>> application to depend on that behavior, and breaking it in a minor
> > >>> release
> > >>>> might be a bit strong.
> > >>>>
> > >>>> *** (Note, for the curious reader....) Our use-case/query pattern
> is a
> > >>> bit
> > >>>> complex, but reading "uncommitted" records is actually safe in our
> > case
> > >>>> 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 soon as the processor processes the result. If
> > they
> > >>>> can't read uncommitted records, they may receive a "201 - Created"
> > >>>> response, immediately followed by a "404 - 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 <
> nick.telford@gmail.com>
> > >>> wrote:
> > >>>>
> > >>>>> Addendum:
> > >>>>>
> > >>>>> I think we would also face the same problem with the approach John
> > >>> outlined
> > >>>>> earlier (using the record cache as a transaction buffer and
> flushing
> > >>> it
> > >>>>> straight to SST files). This is because the record cache (the
> > >>> ThreadCache
> > >>>>> class) is not thread-safe, so every commit would invalidate open IQ
> > >>>>> Iterators in the same way that RocksDB WriteBatches do.
> > >>>>> --
> > >>>>> Nick
> > >>>>>
> > >>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <
> nick.telford@gmail.com>
> > >>>>> 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.
> > >>>>>> 2. Making IsolationLevel a query-time constraint, rather than
> > >>> linking it
> > >>>>>> to the processing.guarantee.
> > >>>>>>
> > >>>>>> There's a wrinkle that makes this a challenge: Interactive Queries
> > >>> that
> > >>>>>> open an Iterator, when using transactions and READ_UNCOMMITTED.
> > >>>>>> The problem is that under READ_UNCOMMITTED, queries need to be
> able
> > >>> to
> > >>>>>> read records from the currently uncommitted transaction buffer
> > >>>>>> (WriteBatch). This includes for Iterators, which should iterate
> > >>> both the
> > >>>>>> transaction buffer and underlying database (using
> > >>>>>> WriteBatch#iteratorWithBase()).
> > >>>>>>
> > >>>>>> The issue is that when the StreamThread commits, it writes the
> > >>> current
> > >>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing
> the
> > >>>>>> WriteBatch while an Interactive Query holds an open Iterator on it
> > >>> will
> > >>>>>> invalidate the Iterator. Worse, it turns out that Iterators over a
> > >>>>>> WriteBatch become invalidated not just when the WriteBatch is
> > >>> cleared,
> > >>>>> but
> > >>>>>> also when the Iterators' current key receives a new write.
> > >>>>>>
> > >>>>>> Now that I'm writing this, I remember that this is the major
> reason
> > >>> that
> > >>>>> I
> > >>>>>> switched the original design from having a query-time
> > >>> IsolationLevel to
> > >>>>>> having the IsolationLevel linked to the transactionality of the
> > >>> stores
> > >>>>>> themselves.
> > >>>>>>
> > >>>>>> It *might* be possible to resolve this, by having a "chain" of
> > >>>>>> WriteBatches, with the StreamThread switching to a new WriteBatch
> > >>>>> whenever
> > >>>>>> a new Interactive Query attempts to read from the database, but
> that
> > >>>>> could
> > >>>>>> cause some performance problems/memory pressure when subjected to
> a
> > >>> high
> > >>>>>> Interactive Query load. It would also reduce the efficiency of
> > >>>>> WriteBatches
> > >>>>>> on-commit, as we'd have to write N WriteBatches, where N is the
> > >>> number of
> > >>>>>> Interactive Queries since the last commit.
> > >>>>>>
> > >>>>>> I realise this is getting into the weeds of the implementation,
> and
> > >>> you'd
> > >>>>>> rather we focus on the API for now, but I think it's important to
> > >>>>> consider
> > >>>>>> how to implement the desired API, in case we come up with an API
> > >>> that
> > >>>>>> cannot be implemented efficiently, or even at all!
> > >>>>>>
> > >>>>>> Thoughts?
> > >>>>>> --
> > >>>>>> Nick
> > >>>>>>
> > >>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <ca...@apache.org>
> > >>> wrote:
> > >>>>>>
> > >>>>>>> Hi Nick,
> > >>>>>>>
> > >>>>>>> 6.
> > >>>>>>> Of course, you are right! My bad!
> > >>>>>>> Wiping out the state in the downgrading case is fine.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> 3a.
> > >>>>>>> Focus on the public facing changes for the KIP. We will manage 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.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> Best,
> > >>>>>>> Bruno
> > >>>>>>>
> > >>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> > >>>>>>>> Hi Bruno,
> > >>>>>>>>
> > >>>>>>>> Thanks for getting back to me!
> > >>>>>>>>
> > >>>>>>>> 2.
> > >>>>>>>> The fact that implementations can always track estimated memory
> > >>> usage
> > >>>>> in
> > >>>>>>>> the wrapper is a good point. I can remove -1 as an option, and
> > >>> I'll
> > >>>>>>> clarify
> > >>>>>>>> the JavaDoc that 0 is not just for non-transactional stores,
> > >>> which is
> > >>>>>>>> currently misleading.
> > >>>>>>>>
> > >>>>>>>> 6.
> > >>>>>>>> The problem with catching the exception in the downgrade process
> > >>> is
> > >>>>> that
> > >>>>>>>> would require new code in the Kafka version being downgraded to.
> > >>> Since
> > >>>>>>>> users could conceivably downgrade to almost *any* older version
> > >>> of
> > >>>>> Kafka
> > >>>>>>>> Streams, I'm not sure how we could add that code?
> > >>>>>>>> The only way I can think of doing it would be to provide a
> > >>> dedicated
> > >>>>>>>> downgrade tool, that goes through every local store and removes
> > >>> the
> > >>>>>>>> offsets column families. But that seems like an unnecessary
> > >>> amount of
> > >>>>>>> extra
> > >>>>>>>> code to maintain just to handle a somewhat niche situation, when
> > >>> the
> > >>>>>>>> alternative (automatically wipe and restore stores) should be
> > >>>>>>> acceptable.
> > >>>>>>>>
> > >>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've requested.
> > >>>>>>>>
> > >>>>>>>> 3a.
> > >>>>>>>> I agree that IsolationLevel makes more sense at query-time, and
> I
> > >>>>>>> actually
> > >>>>>>>> initially attempted to place the IsolationLevel at query-time,
> > >>> but I
> > >>>>> ran
> > >>>>>>>> into some problems:
> > >>>>>>>> - The key issue is that, under ALOS we're not staging writes in
> > >>>>>>>> transactions, so can't perform writes at the READ_COMMITTED
> > >>> isolation
> > >>>>>>>> level. However, this may be addressed if we decide to *always*
> > >>> use
> > >>>>>>>> transactions as discussed under 3b.
> > >>>>>>>> - IQv1 and IQv2 have quite different implementations. I remember
> > >>>>> having
> > >>>>>>>> some difficulty understanding the IQv1 internals, which made it
> > >>>>>>> difficult
> > >>>>>>>> to determine what needed to be changed. However, I *think* this
> > >>> can be
> > >>>>>>>> addressed for both implementations by wrapping the RocksDBStore
> > >>> in an
> > >>>>>>>> IsolationLevel-dependent wrapper, that overrides read methods
> > >>> (get,
> > >>>>>>> etc.)
> > >>>>>>>> to either read directly from the database or from the ongoing
> > >>>>>>> transaction.
> > >>>>>>>> But IQv1 might still be difficult.
> > >>>>>>>> - If IsolationLevel becomes a query constraint, then all other
> > >>>>>>> StateStores
> > >>>>>>>> will need to respect it, including the in-memory stores. This
> > >>> would
> > >>>>>>> require
> > >>>>>>>> us to adapt in-memory stores to stage their writes so they can
> be
> > >>>>>>> isolated
> > >>>>>>>> from READ_COMMITTTED queries. It would also become an important
> > >>>>>>>> consideration for third-party stores on upgrade, as without
> > >>> changes,
> > >>>>>>> they
> > >>>>>>>> would not support READ_COMMITTED queries correctly.
> > >>>>>>>>
> > >>>>>>>> Ultimately, I may need some help making the necessary change to
> > >>> IQv1
> > >>>>> to
> > >>>>>>>> support this, but I don't think it's fundamentally impossible,
> > >>> if we
> > >>>>>>> want
> > >>>>>>>> to pursue this route.
> > >>>>>>>>
> > >>>>>>>> 3b.
> > >>>>>>>> The main reason I chose to keep ALOS un-transactional was to
> > >>> minimize
> > >>>>>>>> behavioural change for most users (I believe most Streams users
> > >>> use
> > >>>>> the
> > >>>>>>>> default configuration, which is ALOS). That said, it's clear
> > >>> that if
> > >>>>>>> ALOS
> > >>>>>>>> also used transactional stores, the only change in behaviour
> > >>> would be
> > >>>>>>> that
> > >>>>>>>> it would become *more correct*, which could be considered a "bug
> > >>> fix"
> > >>>>> by
> > >>>>>>>> users, rather than a change they need to handle.
> > >>>>>>>>
> > >>>>>>>> I believe that performance using transactions (aka. RocksDB
> > >>>>>>> WriteBatches)
> > >>>>>>>> should actually be *better* than the un-batched write-path that
> > >>> is
> > >>>>>>>> currently used[1]. The only "performance" consideration will be
> > >>> the
> > >>>>>>>> increased memory usage that transactions require. Given the
> > >>>>> mitigations
> > >>>>>>> for
> > >>>>>>>> this memory that we have in place, I would expect that this is
> > >>> not a
> > >>>>>>>> problem for most users.
> > >>>>>>>>
> > >>>>>>>> If we're happy to do so, we can make ALOS also use transactions.
> > >>>>>>>>
> > >>>>>>>> Regards,
> > >>>>>>>> Nick
> > >>>>>>>>
> > >>>>>>>> Link 1:
> > >>>>>>>>
> > >>>>>
> > >>>
> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > >>>>>>>>
> > >>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <
> cadonna@apache.org
> > >>>>
> > >>>>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> Hi Nick,
> > >>>>>>>>>
> > >>>>>>>>> Thanks for the updates and sorry for the delay on my side!
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> 1.
> > >>>>>>>>> Making the default implementation for flush() a no-op sounds
> > >>> good to
> > >>>>>>> me.
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> 2.
> > >>>>>>>>> I think what was bugging me here is that a third-party state
> > >>> store
> > >>>>>>> needs
> > >>>>>>>>> to implement the state store interface. That means they need to
> > >>>>>>>>> implement a wrapper around the actual state store as we do for
> > >>>>> RocksDB
> > >>>>>>>>> with RocksDBStore. So, a third-party state store can always
> > >>> estimate
> > >>>>>>> the
> > >>>>>>>>> uncommitted bytes, if it wants, because the wrapper can record
> > >>> the
> > >>>>>>> added
> > >>>>>>>>> bytes.
> > >>>>>>>>> One case I can think of where returning -1 makes sense is when
> > >>>>> Streams
> > >>>>>>>>> does not need to estimate the size of the write batch and
> > >>> trigger
> > >>>>>>>>> extraordinary commits, because the third-party state store
> > >>> takes care
> > >>>>>>> of
> > >>>>>>>>> memory. But in that case the method could also just return 0.
> > >>> Even
> > >>>>> that
> > >>>>>>>>> case would be better solved with a method that returns whether
> > >>> the
> > >>>>>>> state
> > >>>>>>>>> store manages itself the memory used for uncommitted bytes or
> > >>> not.
> > >>>>>>>>> Said that, I am fine with keeping the -1 return value, I was
> > >>> just
> > >>>>>>>>> wondering when and if it will be used.
> > >>>>>>>>>
> > >>>>>>>>> Regarding returning 0 for transactional state stores when the
> > >>> batch
> > >>>>> is
> > >>>>>>>>> empty, I was just wondering because you explicitly stated
> > >>>>>>>>>
> > >>>>>>>>> "or {@code 0} if this StateStore does not support
> transactions."
> > >>>>>>>>>
> > >>>>>>>>> So it seemed to me returning 0 could only happen for
> > >>>>> non-transactional
> > >>>>>>>>> state stores.
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> 3.
> > >>>>>>>>>
> > >>>>>>>>> a) What do you think if we move the isolation level to IQ (v1
> > >>> and
> > >>>>> v2)?
> > >>>>>>>>> In the end this is the only component that really needs to
> > >>> specify
> > >>>>> the
> > >>>>>>>>> isolation level. It is similar to the Kafka consumer that can
> > >>> choose
> > >>>>>>>>> with what isolation level to read the input topic.
> > >>>>>>>>> For IQv1 the isolation level should go into
> > >>> StoreQueryParameters. For
> > >>>>>>>>> IQv2, I would add it to the Query interface.
> > >>>>>>>>>
> > >>>>>>>>> b) Point a) raises the question what should happen during
> > >>>>> at-least-once
> > >>>>>>>>> processing when the state store does not use transactions? John
> > >>> in
> > >>>>> the
> > >>>>>>>>> past proposed to also use transactions on state stores for
> > >>>>>>>>> at-least-once. I like that idea, because it avoids aggregating
> > >>> the
> > >>>>> same
> > >>>>>>>>> records over and over again in the case of a failure. We had a
> > >>> case
> > >>>>> in
> > >>>>>>>>> the past where a Streams applications in at-least-once mode was
> > >>>>> failing
> > >>>>>>>>> continuously for some reasons I do not remember before
> > >>> committing the
> > >>>>>>>>> offsets. After each failover, the app aggregated again and
> > >>> again the
> > >>>>>>>>> same records. Of course the aggregate increased to very wrong
> > >>> values
> > >>>>>>>>> just because of the failover. With transactions on the state
> > >>> stores
> > >>>>> we
> > >>>>>>>>> could have avoided this. The app would have output the same
> > >>> aggregate
> > >>>>>>>>> multiple times (i.e., after each failover) but at least the
> > >>> value of
> > >>>>>>> the
> > >>>>>>>>> aggregate would not depend on the number of failovers.
> > >>> Outputting the
> > >>>>>>>>> same aggregate multiple times would be incorrect under
> > >>> exactly-once
> > >>>>> but
> > >>>>>>>>> it is OK for at-least-once.
> > >>>>>>>>> If it makes sense to add a config to turn on and off
> > >>> transactions on
> > >>>>>>>>> state stores under at-least-once or just use transactions in
> > >>> any case
> > >>>>>>> is
> > >>>>>>>>> a question we should also discuss in this KIP. It depends a bit
> > >>> on
> > >>>>> the
> > >>>>>>>>> performance trade-off. Maybe to be safe, I would add a config.
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> 4.
> > >>>>>>>>> Your points are all valid. I tend to say to keep the metrics
> > >>> around
> > >>>>>>>>> flush() until we remove flush() completely from the interface.
> > >>> Calls
> > >>>>> to
> > >>>>>>>>> flush() might still exist since existing processors might still
> > >>> call
> > >>>>>>>>> flush() explicitly as you mentioned in 1). For sure, we need to
> > >>>>>>> document
> > >>>>>>>>> how the metrics change due to the transactions in the upgrade
> > >>> notes.
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> 5.
> > >>>>>>>>> I see. Then you should describe how the .position files are
> > >>> handled
> > >>>>> in
> > >>>>>>>>> a dedicated section of the KIP or incorporate the description
> > >>> in the
> > >>>>>>>>> "Atomic Checkpointing" section instead of only mentioning it in
> > >>> the
> > >>>>>>>>> "Compatibility, 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:
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>>
> >
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > >>>>>>>>>
> > >>>>>>>>> It is a bit brittle, but it works.
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> Best,
> > >>>>>>>>> Bruno
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> > >>>>>>>>>> Hi Bruno,
> > >>>>>>>>>>
> > >>>>>>>>>> Thanks for taking the time to review the KIP. I'm back from
> > >>> leave
> > >>>>> now
> > >>>>>>> and
> > >>>>>>>>>> intend to move this forwards as quickly as I can.
> > >>>>>>>>>>
> > >>>>>>>>>> Addressing your points:
> > >>>>>>>>>>
> > >>>>>>>>>> 1.
> > >>>>>>>>>> Because flush() is part of the StateStore API, it's exposed to
> > >>>>> custom
> > >>>>>>>>>> Processors, which might be making calls to flush(). This was
> > >>>>> actually
> > >>>>>>> the
> > >>>>>>>>>> case in a few integration tests.
> > >>>>>>>>>> To maintain as much compatibility as possible, I'd prefer not
> > >>> to
> > >>>>> make
> > >>>>>>>>> this
> > >>>>>>>>>> an UnsupportedOperationException, as it will cause previously
> > >>>>> working
> > >>>>>>>>>> Processors to start throwing exceptions at runtime.
> > >>>>>>>>>> I agree that it doesn't make sense for it to proxy commit(),
> > >>> though,
> > >>>>>>> as
> > >>>>>>>>>> that would cause it to violate the "StateStores commit only
> > >>> when the
> > >>>>>>> Task
> > >>>>>>>>>> commits" rule.
> > >>>>>>>>>> Instead, I think we should make this a no-op. That way,
> > >>> existing
> > >>>>> user
> > >>>>>>>>>> Processors will continue to work as-before, without violation
> > >>> of
> > >>>>> store
> > >>>>>>>>>> consistency that would be caused by premature flush/commit of
> > >>>>>>> StateStore
> > >>>>>>>>>> data to disk.
> > >>>>>>>>>> What do you think?
> > >>>>>>>>>>
> > >>>>>>>>>> 2.
> > >>>>>>>>>> As stated in the JavaDoc, when a StateStore implementation is
> > >>>>>>>>>> transactional, but is unable to estimate the uncommitted
> memory
> > >>>>> usage,
> > >>>>>>>>> the
> > >>>>>>>>>> method will return -1.
> > >>>>>>>>>> The intention here is to permit third-party implementations
> > >>> that may
> > >>>>>>> not
> > >>>>>>>>> be
> > >>>>>>>>>> able to estimate memory usage.
> > >>>>>>>>>>
> > >>>>>>>>>> Yes, it will be 0 when nothing has been written to the store
> > >>> yet. I
> > >>>>>>>>> thought
> > >>>>>>>>>> that was implied by "This method will return an approximation
> > >>> of the
> > >>>>>>>>> memory
> > >>>>>>>>>> would be freed by the next call to {@link #commit(Map)}" and
> > >>>>> "@return
> > >>>>>>> The
> > >>>>>>>>>> approximate size of all records awaiting {@link
> #commit(Map)}",
> > >>>>>>> however,
> > >>>>>>>>> I
> > >>>>>>>>>> can add it explicitly to the JavaDoc if you think this is
> > >>> unclear?
> > >>>>>>>>>>
> > >>>>>>>>>> 3.
> > >>>>>>>>>> I realise this is probably the most contentious point in my
> > >>> design,
> > >>>>>>> and
> > >>>>>>>>> I'm
> > >>>>>>>>>> open to changing it if I'm unable to convince you of the
> > >>> benefits.
> > >>>>>>>>>> Nevertheless, here's my argument:
> > >>>>>>>>>> The Interactive Query (IQ) API(s) are directly provided
> > >>> StateStores
> > >>>>> to
> > >>>>>>>>>> query, and it may be important for users to programmatically
> > >>> know
> > >>>>>>> which
> > >>>>>>>>>> mode the StateStore is operating under. If we simply provide
> an
> > >>>>>>>>>> "eosEnabled" boolean (as used throughout the internal streams
> > >>>>>>> engine), or
> > >>>>>>>>>> similar, then users will need to understand the operation and
> > >>>>>>>>> consequences
> > >>>>>>>>>> of each available processing mode and how it pertains to their
> > >>>>>>>>> StateStore.
> > >>>>>>>>>>
> > >>>>>>>>>> Interactive Query users aren't the only people that care about
> > >>> the
> > >>>>>>>>>> processing.mode/IsolationLevel of a StateStore: implementers
> of
> > >>>>> custom
> > >>>>>>>>>> StateStores also need to understand the behaviour expected of
> > >>> their
> > >>>>>>>>>> implementation. KIP-892 introduces some assumptions into the
> > >>> Streams
> > >>>>>>>>> Engine
> > >>>>>>>>>> about how StateStores operate under each processing mode, and
> > >>> it's
> > >>>>>>>>>> important that custom implementations adhere to those
> > >>> assumptions in
> > >>>>>>>>> order
> > >>>>>>>>>> to maintain the consistency guarantees.
> > >>>>>>>>>>
> > >>>>>>>>>> IsolationLevels provide a high-level contract on the behaviour
> > >>> of
> > >>>>> the
> > >>>>>>>>>> StateStore: a user knows that under READ_COMMITTED, they will
> > >>> see
> > >>>>>>> writes
> > >>>>>>>>>> only after the Task has committed, and under READ_UNCOMMITTED
> > >>> they
> > >>>>>>> will
> > >>>>>>>>> see
> > >>>>>>>>>> writes immediately. No understanding of the details of each
> > >>>>>>>>> processing.mode
> > >>>>>>>>>> is required, either for IQ users or StateStore implementers.
> > >>>>>>>>>>
> > >>>>>>>>>> An argument can be made that these contractual guarantees can
> > >>> simply
> > >>>>>>> be
> > >>>>>>>>>> documented for the processing.mode (i.e. that exactly-once and
> > >>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and at-least-once
> > >>> behaves
> > >>>>>>> like
> > >>>>>>>>>> READ_UNCOMMITTED), but there are several small issues with
> > >>> this I'd
> > >>>>>>>>> prefer
> > >>>>>>>>>> to avoid:
> > >>>>>>>>>>
> > >>>>>>>>>>       - Where would we document these contracts, in a way that
> > >>> is
> > >>>>>>> difficult
> > >>>>>>>>>>       for users/implementers to miss/ignore?
> > >>>>>>>>>>       - It's not clear to users that the processing mode is
> > >>>>>>> communicating
> > >>>>>>>>>>       an expectation of read isolation, unless they read the
> > >>>>>>>>> documentation. Users
> > >>>>>>>>>>       rarely consult documentation unless they feel they need
> > >>> to, so
> > >>>>>>> it's
> > >>>>>>>>> likely
> > >>>>>>>>>>       this detail would get missed by many users.
> > >>>>>>>>>>       - It tightly couples processing modes to read isolation.
> > >>> Adding
> > >>>>>>> new
> > >>>>>>>>>>       processing modes, or changing the read isolation of
> > >>> existing
> > >>>>>>>>> processing
> > >>>>>>>>>>       modes would be difficult/impossible.
> > >>>>>>>>>>
> > >>>>>>>>>> Ultimately, the cost of introducing IsolationLevels is just a
> > >>> single
> > >>>>>>>>>> method, since we re-use the existing IsolationLevel enum from
> > >>> Kafka.
> > >>>>>>> This
> > >>>>>>>>>> gives us a clear place to document the contractual guarantees
> > >>>>> expected
> > >>>>>>>>>> of/provided by StateStores, that is accessible both by the
> > >>>>> StateStore
> > >>>>>>>>>> itself, and by IQ users.
> > >>>>>>>>>>
> > >>>>>>>>>> (Writing this I've just realised that the StateStore and IQ
> > >>> APIs
> > >>>>>>> actually
> > >>>>>>>>>> don't provide access to StateStoreContext that IQ users would
> > >>> have
> > >>>>>>> direct
> > >>>>>>>>>> access to... Perhaps StateStore should expose isolationLevel()
> > >>>>> itself
> > >>>>>>>>> too?)
> > >>>>>>>>>>
> > >>>>>>>>>> 4.
> > >>>>>>>>>> Yeah, I'm not comfortable renaming the metrics in-place
> > >>> either, as
> > >>>>>>> it's a
> > >>>>>>>>>> backwards incompatible change. My concern is that, if we leave
> > >>> the
> > >>>>>>>>> existing
> > >>>>>>>>>> "flush" metrics in place, they will be confusing to users.
> > >>> Right
> > >>>>> now,
> > >>>>>>>>>> "flush" metrics record explicit flushes to disk, but under
> > >>> KIP-892,
> > >>>>>>> even
> > >>>>>>>>> a
> > >>>>>>>>>> commit() will not explicitly flush data to disk - RocksDB will
> > >>>>> decide
> > >>>>>>> on
> > >>>>>>>>>> when to flush memtables to disk itself.
> > >>>>>>>>>>
> > >>>>>>>>>> If we keep the existing "flush" metrics, we'd have two
> options,
> > >>>>> which
> > >>>>>>>>> both
> > >>>>>>>>>> seem pretty bad to me:
> > >>>>>>>>>>
> > >>>>>>>>>>       1. Have them record calls to commit(), which would be
> > >>>>>>> misleading, as
> > >>>>>>>>>>       data is no longer explicitly "flushed" to disk by this
> > >>> call.
> > >>>>>>>>>>       2. Have them record nothing at all, which is equivalent
> to
> > >>>>>>> removing
> > >>>>>>>>> the
> > >>>>>>>>>>       metrics, except that users will see the metric still
> > >>> exists and
> > >>>>>>> so
> > >>>>>>>>> assume
> > >>>>>>>>>>       that the metric is correct, and that there's a problem
> > >>> with
> > >>>>> their
> > >>>>>>>>> system
> > >>>>>>>>>>       when there isn't.
> > >>>>>>>>>>
> > >>>>>>>>>> I agree that removing them is also a bad solution, and I'd
> > >>> like some
> > >>>>>>>>>> guidance on the best path forward here.
> > >>>>>>>>>>
> > >>>>>>>>>> 5.
> > >>>>>>>>>> Position files are updated on every write to a StateStore.
> > >>> Since our
> > >>>>>>>>> writes
> > >>>>>>>>>> are now buffered until commit(), we can't update the Position
> > >>> file
> > >>>>>>> until
> > >>>>>>>>>> commit() has been called, otherwise it would be inconsistent
> > >>> with
> > >>>>> the
> > >>>>>>>>> data
> > >>>>>>>>>> in the event of a rollback. Consequently, we need to manage
> > >>> these
> > >>>>>>> offsets
> > >>>>>>>>>> the same way we manage the checkpoint offsets, and ensure
> > >>> they're
> > >>>>> only
> > >>>>>>>>>> written on commit().
> > >>>>>>>>>>
> > >>>>>>>>>> 6.
> > >>>>>>>>>> Agreed, although I'm not exactly sure yet what tests to write.
> > >>> How
> > >>>>>>>>> explicit
> > >>>>>>>>>> do we need to be here in the KIP?
> > >>>>>>>>>>
> > >>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be seamless,
> > >>> and we
> > >>>>>>>>> should
> > >>>>>>>>>> definitely add some tests around that. Downgrade, it
> > >>> transpires,
> > >>>>> isn't
> > >>>>>>>>>> currently possible, as the extra column family for offset
> > >>> storage is
> > >>>>>>>>>> incompatible with the pre-KIP-892 implementation: when you
> > >>> open a
> > >>>>>>> RocksDB
> > >>>>>>>>>> database, you must open all available column families or
> > >>> receive an
> > >>>>>>>>> error.
> > >>>>>>>>>> What currently happens on downgrade is that it attempts to
> > >>> open the
> > >>>>>>>>> store,
> > >>>>>>>>>> throws an error about the offsets column family not being
> > >>> opened,
> > >>>>>>> which
> > >>>>>>>>>> triggers a wipe and rebuild of the Task. Given that downgrades
> > >>>>> should
> > >>>>>>> be
> > >>>>>>>>>> uncommon, I think this is acceptable behaviour, as the
> > >>> end-state is
> > >>>>>>>>>> consistent, even if it results in an undesirable state
> restore.
> > >>>>>>>>>>
> > >>>>>>>>>> Should I document the upgrade/downgrade behaviour explicitly
> > >>> in the
> > >>>>>>> KIP?
> > >>>>>>>>>>
> > >>>>>>>>>> --
> > >>>>>>>>>>
> > >>>>>>>>>> Regards,
> > >>>>>>>>>> Nick
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> > >>> cadonna@apache.org>
> > >>>>>>> wrote:
> > >>>>>>>>>>
> > >>>>>>>>>>> Hi Nick!
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thanks for the updates!
> > >>>>>>>>>>>
> > >>>>>>>>>>> 1.
> > >>>>>>>>>>> Why does StateStore#flush() default to
> > >>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> > >>>>>>>>>>> Since calls to flush() will not exist anymore after this KIP
> > >>> is
> > >>>>>>>>>>> released, I would rather throw an unsupported operation
> > >>> exception
> > >>>>> by
> > >>>>>>>>>>> default.
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> 2.
> > >>>>>>>>>>> When would a state store return -1 from
> > >>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while being
> > >>>>>>> transactional?
> > >>>>>>>>>>>
> > >>>>>>>>>>> Wouldn't StateStore#approximateNumUncommittedBytes() also
> > >>> return 0
> > >>>>> if
> > >>>>>>>>>>> the state store is transactional but nothing has been written
> > >>> to
> > >>>>> the
> > >>>>>>>>>>> state store yet?
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> 3.
> > >>>>>>>>>>> Sorry for bringing this up again. Does this KIP really need
> to
> > >>>>>>> introduce
> > >>>>>>>>>>> StateStoreContext#isolationLevel()? StateStoreContext has
> > >>> already
> > >>>>>>>>>>> appConfigs() which basically exposes the same information,
> > >>> i.e., if
> > >>>>>>> EOS
> > >>>>>>>>>>> is enabled or not.
> > >>>>>>>>>>> In one of your previous e-mails you wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>> "My idea was to try to keep the StateStore interface as
> > >>> loosely
> > >>>>>>> coupled
> > >>>>>>>>>>> from the Streams engine as possible, to give implementers
> more
> > >>>>>>> freedom,
> > >>>>>>>>>>> and reduce the amount of internal knowledge required."
> > >>>>>>>>>>>
> > >>>>>>>>>>> While I understand the intent, I doubt that it decreases the
> > >>>>>>> coupling of
> > >>>>>>>>>>> a StateStore interface and the Streams engine. READ_COMMITTED
> > >>> only
> > >>>>>>>>>>> applies to IQ but not to reads by processors. Thus,
> > >>> implementers
> > >>>>>>> need to
> > >>>>>>>>>>> understand how Streams accesses the state stores.
> > >>>>>>>>>>>
> > >>>>>>>>>>> I would like to hear what others think about this.
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> 4.
> > >>>>>>>>>>> Great exposing new metrics for transactional state stores!
> > >>>>> However, I
> > >>>>>>>>>>> would prefer to add new metrics and deprecate (in the docs)
> > >>> the 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 also need to adapt/add integration tests
> > >>> besides
> > >>>>> unit
> > >>>>>>>>>>> tests. Additionally, we probably need integration or system
> > >>> tests
> > >>>>> to
> > >>>>>>>>>>> verify that upgrades and downgrades between transactional and
> > >>>>>>>>>>> non-transactional state stores work as expected.
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> Best,
> > >>>>>>>>>>> Bruno
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > >>>>>>>>>>>> One more thing: I noted John's suggestion in the KIP, under
> > >>>>>>> "Rejected
> > >>>>>>>>>>>> Alternatives". I still think it's an idea worth pursuing,
> > >>> but I
> > >>>>>>> believe
> > >>>>>>>>>>>> that it's out of the scope of this KIP, because it solves a
> > >>>>>>> different
> > >>>>>>>>> set
> > >>>>>>>>>>>> of problems to this KIP, and the scope of this one has
> > >>> already
> > >>>>> grown
> > >>>>>>>>>>> quite
> > >>>>>>>>>>>> large!
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > >>>>> nick.telford@gmail.com>
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>> Hi everyone,
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> I've updated the KIP (
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>
> > >>>>>
> > >>>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > >>>>>>>>>>> )
> > >>>>>>>>>>>>> with the latest changes; mostly bringing back "Atomic
> > >>>>>>> Checkpointing"
> > >>>>>>>>>>> (for
> > >>>>>>>>>>>>> what feels like the 10th time!). I think the one thing
> > >>> missing is
> > >>>>>>> some
> > >>>>>>>>>>>>> changes to metrics (notably the store "flush" metrics will
> > >>> need
> > >>>>> to
> > >>>>>>> be
> > >>>>>>>>>>>>> renamed to "commit").
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> The reason I brought back Atomic Checkpointing was to
> > >>> decouple
> > >>>>>>> store
> > >>>>>>>>>>> flush
> > >>>>>>>>>>>>> from store commit. This is important, because with
> > >>> Transactional
> > >>>>>>>>>>>>> StateStores, we now need to call "flush" on *every* Task
> > >>> commit,
> > >>>>>>> and
> > >>>>>>>>> not
> > >>>>>>>>>>>>> just when the StateStore is closing, otherwise our
> > >>> transaction
> > >>>>>>> buffer
> > >>>>>>>>>>> will
> > >>>>>>>>>>>>> never be written and persisted, instead growing unbounded!
> I
> > >>>>>>>>>>> experimented
> > >>>>>>>>>>>>> with some simple solutions, like forcing a store flush
> > >>> whenever
> > >>>>> the
> > >>>>>>>>>>>>> transaction buffer was likely to exceed its configured
> > >>> size, but
> > >>>>>>> this
> > >>>>>>>>>>> was
> > >>>>>>>>>>>>> brittle: it prevented the transaction buffer from being
> > >>>>> configured
> > >>>>>>> to
> > >>>>>>>>> be
> > >>>>>>>>>>>>> unbounded, and it still would have required explicit
> > >>> flushes of
> > >>>>>>>>> RocksDB,
> > >>>>>>>>>>>>> yielding sub-optimal performance and memory utilization.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right" way to
> > >>> resolve
> > >>>>> this
> > >>>>>>>>>>>>> problem. By ensuring that the changelog offsets that
> > >>> correspond
> > >>>>> to
> > >>>>>>> the
> > >>>>>>>>>>> most
> > >>>>>>>>>>>>> recently written records are always atomically written to
> > >>> the
> > >>>>>>>>> StateStore
> > >>>>>>>>>>>>> (by writing them to the same transaction buffer), we can
> > >>> avoid
> > >>>>>>>>> forcibly
> > >>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting RocksDB
> > >>> flush
> > >>>>> them
> > >>>>>>>>> only
> > >>>>>>>>>>>>> when necessary, without losing any of our consistency
> > >>> guarantees.
> > >>>>>>> See
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>> updated KIP for more info.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> I have fully implemented these changes, although I'm still
> > >>> not
> > >>>>>>>>> entirely
> > >>>>>>>>>>>>> happy with the implementation for segmented StateStores, so
> > >>> I
> > >>>>> plan
> > >>>>>>> to
> > >>>>>>>>>>>>> refactor that. Despite that, all tests pass. If you'd like
> > >>> to try
> > >>>>>>> out
> > >>>>>>>>> or
> > >>>>>>>>>>>>> review this highly experimental and incomplete branch, it's
> > >>>>>>> available
> > >>>>>>>>>>> here:
> > >>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0.
> > >>> Note:
> > >>>>>>> it's
> > >>>>>>>>>>> built
> > >>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base to build
> > >>> and test
> > >>>>>>> it
> > >>>>>>>>> on,
> > >>>>>>>>>>>>> and to enable easy apples-to-apples comparisons in a live
> > >>>>>>>>> environment. I
> > >>>>>>>>>>>>> plan to rebase it against trunk once it's nearer completion
> > >>> and
> > >>>>> has
> > >>>>>>>>> been
> > >>>>>>>>>>>>> proven on our main application.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> I would really appreciate help in reviewing and testing:
> > >>>>>>>>>>>>> - Segmented (Versioned, Session and Window) stores
> > >>>>>>>>>>>>> - Global stores
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> As I do not currently use either of these, so my primary
> > >>> test
> > >>>>>>>>>>> environment
> > >>>>>>>>>>>>> doesn't test these areas.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> I'm going on Parental Leave starting next week for a few
> > >>> weeks,
> > >>>>> so
> > >>>>>>>>> will
> > >>>>>>>>>>>>> not have time to move this forward until late August. That
> > >>> said,
> > >>>>>>> your
> > >>>>>>>>>>>>> feedback is welcome and appreciated, I just won't be able
> to
> > >>>>>>> respond
> > >>>>>>>>> as
> > >>>>>>>>>>>>> quickly as usual.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > >>>>> nick.telford@gmail.com>
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Hi Bruno
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ is not
> > >>> something
> > >>>>> I
> > >>>>>>> had
> > >>>>>>>>>>>>>> considered.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> What about atomically updating the state store from the
> > >>>>>>> transaction
> > >>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
> > >>> (thus,
> > >>>>>>>>>>> flushing
> > >>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> > >>> number of
> > >>>>>>>>> commit
> > >>>>>>>>>>>>>>> intervals?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting that we
> > >>> add an
> > >>>>>>>>>>> additional
> > >>>>>>>>>>>>>> config for the max number of commit intervals between
> > >>>>> checkpoints?
> > >>>>>>>>> That
> > >>>>>>>>>>>>>> way, we would checkpoint *either* when the transaction
> > >>> buffers
> > >>>>> are
> > >>>>>>>>>>> nearly
> > >>>>>>>>>>>>>> full, *OR* whenever a certain number of commit intervals
> > >>> have
> > >>>>>>>>> elapsed,
> > >>>>>>>>>>>>>> whichever comes first?
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> That certainly seems reasonable, although this re-ignites
> > >>> an
> > >>>>>>> earlier
> > >>>>>>>>>>>>>> debate about whether a config should be measured in
> > >>> "number of
> > >>>>>>> commit
> > >>>>>>>>>>>>>> intervals", instead of just an absolute time.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> FWIW, I realised that this issue is the reason I was
> > >>> pursuing
> > >>>>> the
> > >>>>>>>>>>> Atomic
> > >>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush from
> > >>> checkpointing,
> > >>>>>>>>> which
> > >>>>>>>>>>>>>> enables us to just checkpoint on every commit without any
> > >>>>>>> performance
> > >>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the "best"
> > >>> solution,
> > >>>>>>> but
> > >>>>>>>>>>> I'm not
> > >>>>>>>>>>>>>> sure if this is enough to bring it back into this KIP.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> I'm currently working on moving all the transactional
> logic
> > >>>>>>> directly
> > >>>>>>>>>>> into
> > >>>>>>>>>>>>>> RocksDBStore itself, which does away with the
> > >>>>>>>>> StateStore#newTransaction
> > >>>>>>>>>>>>>> method, and reduces the number of new classes introduced,
> > >>>>>>>>> significantly
> > >>>>>>>>>>>>>> reducing the complexity. If it works, and the complexity
> is
> > >>>>>>>>> drastically
> > >>>>>>>>>>>>>> reduced, I may try bringing back Atomic Checkpoints into
> > >>> this
> > >>>>> KIP.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
> > >>> cadonna@apache.org>
> > >>>>>>>>> wrote:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Hi Nick,
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> As far as I understand, you want to atomically update the
> > >>> state
> > >>>>>>>>> store
> > >>>>>>>>>>>>>>> from the transaction buffer, flush the memtable of a
> state
> > >>>>> store
> > >>>>>>> and
> > >>>>>>>>>>>>>>> write the checkpoint not after the commit time elapsed
> but
> > >>>>> after
> > >>>>>>> the
> > >>>>>>>>>>>>>>> transaction buffer reached a size that would lead to
> > >>> exceeding
> > >>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes before the next
> > >>> commit
> > >>>>>>>>>>> interval
> > >>>>>>>>>>>>>>> ends.
> > >>>>>>>>>>>>>>> That means, the Kafka transaction would commit every
> > >>> commit
> > >>>>>>> interval
> > >>>>>>>>>>> but
> > >>>>>>>>>>>>>>> the state store will only be atomically updated roughly
> > >>> every
> > >>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ
> > >>> would
> > >>>>>>> then
> > >>>>>>>>>>> only
> > >>>>>>>>>>>>>>> see new data roughly every
> > >>>>>>> statestore.transaction.buffer.max.bytes.
> > >>>>>>>>>>>>>>> After a failure the state store needs to restore up to
> > >>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Is this correct?
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> What about atomically updating the state store from the
> > >>>>>>> transaction
> > >>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
> > >>> (thus,
> > >>>>>>>>>>> flushing
> > >>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> > >>> number of
> > >>>>>>>>> commit
> > >>>>>>>>>>>>>>> intervals? In such a way, we would have the same delay
> for
> > >>>>>>> records
> > >>>>>>>>>>>>>>> appearing in output topics and IQ because both would
> > >>> appear
> > >>>>> when
> > >>>>>>> the
> > >>>>>>>>>>>>>>> Kafka transaction is committed. However, after a failure
> > >>> the
> > >>>>>>> state
> > >>>>>>>>>>> store
> > >>>>>>>>>>>>>>> still needs to restore up to
> > >>>>>>> statestore.transaction.buffer.max.bytes
> > >>>>>>>>>>> and
> > >>>>>>>>>>>>>>> it might restore data that is already in the state store
> > >>>>> because
> > >>>>>>> the
> > >>>>>>>>>>>>>>> checkpoint lags behind the last stable offset (i.e. the
> > >>> last
> > >>>>>>>>> committed
> > >>>>>>>>>>>>>>> offset) of the changelog topics. Restoring data that is
> > >>> already
> > >>>>>>> in
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>> state store is idempotent, so eos should not violated.
> > >>>>>>>>>>>>>>> This solution needs at least one new config to specify
> > >>> when a
> > >>>>>>>>>>> checkpoint
> > >>>>>>>>>>>>>>> should be written.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> A small correction to your previous e-mail that does not
> > >>> change
> > >>>>>>>>>>> anything
> > >>>>>>>>>>>>>>> you said: Under alos the default commit interval is 30
> > >>> seconds,
> > >>>>>>> not
> > >>>>>>>>>>> five
> > >>>>>>>>>>>>>>> seconds.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > >>>>>>>>>>>>>>>> Hi everyone,
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I've begun performance testing my branch on our staging
> > >>>>>>>>> environment,
> > >>>>>>>>>>>>>>>> putting it through its paces in our non-trivial
> > >>> application.
> > >>>>> I'm
> > >>>>>>>>>>>>>>> already
> > >>>>>>>>>>>>>>>> observing the same increased flush rate that we saw the
> > >>> last
> > >>>>>>> time
> > >>>>>>>>> we
> > >>>>>>>>>>>>>>>> attempted to use a version of this KIP, but this time, I
> > >>>>> think I
> > >>>>>>>>> know
> > >>>>>>>>>>>>>>> why.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at
> > >>> the end
> > >>>>>>> of
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>> Task
> > >>>>>>>>>>>>>>>> commit process, has the following behaviour:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>         - Under ALOS: checkpoint the state stores. This
> > >>>>> includes
> > >>>>>>>>>>>>>>>>         flushing memtables in RocksDB. This is
> acceptable
> > >>>>>>> because the
> > >>>>>>>>>>>>>>> default
> > >>>>>>>>>>>>>>>>         commit.interval.ms is 5 seconds, so forcibly
> > >>> flushing
> > >>>>>>>>> memtables
> > >>>>>>>>>>>>>>> every 5
> > >>>>>>>>>>>>>>>>         seconds is acceptable for most applications.
> > >>>>>>>>>>>>>>>>         - Under EOS: checkpointing is not done, *unless*
> > >>> it's
> > >>>>>>> being
> > >>>>>>>>>>>>>>> forced, due
> > >>>>>>>>>>>>>>>>         to e.g. the Task closing or being revoked. This
> > >>> means
> > >>>>>>> that
> > >>>>>>>>> under
> > >>>>>>>>>>>>>>> normal
> > >>>>>>>>>>>>>>>>         processing conditions, the state stores will not
> > >>> be
> > >>>>>>>>>>> checkpointed,
> > >>>>>>>>>>>>>>> and will
> > >>>>>>>>>>>>>>>>         not have memtables flushed at all , unless
> RocksDB
> > >>>>>>> decides to
> > >>>>>>>>>>>>>>> flush them on
> > >>>>>>>>>>>>>>>>         its own. Checkpointing stores and force-flushing
> > >>> their
> > >>>>>>>>> memtables
> > >>>>>>>>>>>>>>> is only
> > >>>>>>>>>>>>>>>>         done when a Task is being closed.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at
> least
> > >>>>> *some*
> > >>>>>>>>>>> normal
> > >>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB transaction
> > >>>>> buffers
> > >>>>>>> to
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>>>>> state stores, and to ensure the offsets are synced to
> > >>> disk to
> > >>>>>>>>> prevent
> > >>>>>>>>>>>>>>>> restores from getting out of hand. Consequently, my
> > >>> current
> > >>>>>>>>>>>>>>> implementation
> > >>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is
> > >>> far too
> > >>>>>>>>>>>>>>> frequent.
> > >>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records, which is a
> > >>>>> change
> > >>>>>>> in
> > >>>>>>>>>>>>>>> flush
> > >>>>>>>>>>>>>>>> behaviour, potentially causing performance problems for
> > >>> some
> > >>>>>>>>>>>>>>> applications.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm currently
> > >>> leaning
> > >>>>>>>>>>> towards
> > >>>>>>>>>>>>>>>> using the statestore.transaction.buffer.max.bytes
> > >>>>> configuration
> > >>>>>>> to
> > >>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to exceed it. This
> > >>> would
> > >>>>>>>>>>>>>>> complement the
> > >>>>>>>>>>>>>>>> existing "early Task commit" functionality that this
> > >>>>>>> configuration
> > >>>>>>>>>>>>>>>> provides, in the following way:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>         - Currently, we use
> > >>>>>>> statestore.transaction.buffer.max.bytes
> > >>>>>>>>> to
> > >>>>>>>>>>>>>>> force an
> > >>>>>>>>>>>>>>>>         early Task commit if processing more records
> would
> > >>>>> cause
> > >>>>>>> our
> > >>>>>>>>>>> state
> > >>>>>>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>         transactions to exceed the memory assigned to
> > >>> them.
> > >>>>>>>>>>>>>>>>         - New functionality: when a Task *does* commit,
> > >>> we will
> > >>>>>>> not
> > >>>>>>>>>>>>>>> checkpoint
> > >>>>>>>>>>>>>>>>         the stores (and hence flush the transaction
> > >>> buffers)
> > >>>>>>> unless
> > >>>>>>>>> we
> > >>>>>>>>>>>>>>> expect to
> > >>>>>>>>>>>>>>>>         cross the
> statestore.transaction.buffer.max.bytes
> > >>>>>>> threshold
> > >>>>>>>>>>> before
> > >>>>>>>>>>>>>>> the next
> > >>>>>>>>>>>>>>>>         commit
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> I'm also open to suggestions.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> > >>>>>>> nick.telford@gmail.com
> > >>>>>>>>>>
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Hi Bruno!
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>> By "less predictable for users", I meant in terms of
> > >>>>>>> understanding
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> performance profile under various circumstances. The
> > >>> more
> > >>>>>>> complex
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> solution, the more difficult it would be for users to
> > >>>>>>> understand
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>>> performance they see. For example, spilling records to
> > >>> disk
> > >>>>>>> when
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>>> transaction buffer reaches a threshold would, I expect,
> > >>>>> reduce
> > >>>>>>>>> write
> > >>>>>>>>>>>>>>>>> throughput. This reduction in write throughput could be
> > >>>>>>>>> unexpected,
> > >>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>> potentially difficult to diagnose/understand for users.
> > >>>>>>>>>>>>>>>>> At the moment, I think the "early commit" concept is
> > >>>>> relatively
> > >>>>>>>>>>>>>>>>> straightforward; it's easy to document, and
> conceptually
> > >>>>> fairly
> > >>>>>>>>>>>>>>> obvious to
> > >>>>>>>>>>>>>>>>> users. We could probably add a metric to make it easier
> > >>> to
> > >>>>>>>>>>> understand
> > >>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>> it happens though.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> 3. (the second one)
> > >>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an indirect way of
> > >>>>> telling
> > >>>>>>>>>>>>>>> StateStores
> > >>>>>>>>>>>>>>>>> whether they should be transactional. READ_COMMITTED
> > >>>>>>> essentially
> > >>>>>>>>>>>>>>> requires
> > >>>>>>>>>>>>>>>>> transactions, because it dictates that two threads
> > >>> calling
> > >>>>>>>>>>>>>>>>> `newTransaction()` should not see writes from the other
> > >>>>>>>>> transaction
> > >>>>>>>>>>>>>>> until
> > >>>>>>>>>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all
> > >>> bets are
> > >>>>>>> off,
> > >>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>> stores can allow threads to observe written records at
> > >>> any
> > >>>>>>> time,
> > >>>>>>>>>>>>>>> which is
> > >>>>>>>>>>>>>>>>> essentially "no transactions". That said, StateStores
> > >>> are
> > >>>>> free
> > >>>>>>> to
> > >>>>>>>>>>>>>>> implement
> > >>>>>>>>>>>>>>>>> these guarantees however they can, which is a bit more
> > >>>>> relaxed
> > >>>>>>>>> than
> > >>>>>>>>>>>>>>>>> dictating "you must use transactions". For example,
> with
> > >>>>>>> RocksDB
> > >>>>>>>>> we
> > >>>>>>>>>>>>>>> would
> > >>>>>>>>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
> > >>>>> "transactions",
> > >>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But
> > >>> with
> > >>>>>>> other
> > >>>>>>>>>>>>>>> storage
> > >>>>>>>>>>>>>>>>> engines, it might be preferable to *always* use
> > >>> transactions,
> > >>>>>>> even
> > >>>>>>>>>>>>>>> when
> > >>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines that don't
> > >>>>> provide
> > >>>>>>>>>>>>>>>>> transactions, but the isolation guarantees can be met
> > >>> using a
> > >>>>>>>>>>>>>>> different
> > >>>>>>>>>>>>>>>>> technique.
> > >>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore interface as
> > >>>>> loosely
> > >>>>>>>>>>> coupled
> > >>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> > >>> implementers
> > >>>>> more
> > >>>>>>>>>>>>>>> freedom, and
> > >>>>>>>>>>>>>>>>> reduce the amount of internal knowledge required.
> > >>>>>>>>>>>>>>>>> That said, I understand that "IsolationLevel" might not
> > >>> be
> > >>>>> the
> > >>>>>>>>> right
> > >>>>>>>>>>>>>>>>> abstraction, and we can always make it much more
> > >>> explicit if
> > >>>>>>>>>>>>>>> required, e.g.
> > >>>>>>>>>>>>>>>>> boolean transactional()
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> 7-8.
> > >>>>>>>>>>>>>>>>> I can make these changes either later today or
> tomorrow.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Small update:
> > >>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of
> > >>> issues
> > >>>>>>> that
> > >>>>>>>>>>>>>>> needed
> > >>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass, which is
> > >>>>> promising,
> > >>>>>>> but
> > >>>>>>>>>>> it
> > >>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>> need to undergo some performance testing. I haven't
> > >>> (yet)
> > >>>>>>> worked
> > >>>>>>>>> on
> > >>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I would
> > >>> expect
> > >>>>> that,
> > >>>>>>>>>>>>>>>>> behaviourally, it should make no difference. The branch
> > >>> is
> > >>>>>>>>> available
> > >>>>>>>>>>>>>>> at
> > >>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if
> > >>>>> anyone
> > >>>>>>> is
> > >>>>>>>>>>>>>>>>> interested in taking an early look.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> > >>>>>>> cadonna@apache.org>
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Hi Nick,
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually also my
> > >>> point. I
> > >>>>>>>>>>> understood
> > >>>>>>>>>>>>>>>>>> that John was proposing the ingestion path as a way to
> > >>> avoid
> > >>>>>>> the
> > >>>>>>>>>>>>>>> early
> > >>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>>>>> I agree with John here, that actually it is public
> > >>> API. My
> > >>>>>>>>> question
> > >>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>> how this usage pattern affects normal processing.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>> My concern is that checking for the size of the
> > >>> transaction
> > >>>>>>>>> buffer
> > >>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>> maybe triggering an early commit affects the whole
> > >>>>> processing
> > >>>>>>> of
> > >>>>>>>>>>>>>>> Kafka
> > >>>>>>>>>>>>>>>>>> Streams. The transactionality of a state store is not
> > >>>>>>> confined to
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> state store itself, but spills over and changes the
> > >>> behavior
> > >>>>>>> of
> > >>>>>>>>>>> other
> > >>>>>>>>>>>>>>>>>> parts of the system. I agree with you that it is a
> > >>> decent
> > >>>>>>>>>>>>>>> compromise. I
> > >>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and list the
> > >>> options to
> > >>>>>>>>>>> overcome
> > >>>>>>>>>>>>>>>>>> them. I also agree with you that all options seem
> quite
> > >>>>> heavy
> > >>>>>>>>>>>>>>> compared
> > >>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you mean with
> > >>> "less
> > >>>>>>>>>>>>>>> predictable
> > >>>>>>>>>>>>>>>>>> for users", though.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I found the discussions about the alternatives really
> > >>>>>>>>> interesting.
> > >>>>>>>>>>>>>>> But I
> > >>>>>>>>>>>>>>>>>> also think that your plan sounds good and we should
> > >>> continue
> > >>>>>>> with
> > >>>>>>>>>>> it!
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind putting
> > >>> isolation
> > >>>>>>>>> level
> > >>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>> the state store context. Thanks! Should that also be a
> > >>> way
> > >>>>> to
> > >>>>>>>>> give
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> the state store the opportunity to decide whether to
> > >>> turn on
> > >>>>>>>>>>>>>>>>>> transactions or not?
> > >>>>>>>>>>>>>>>>>> With my comment, I was more concerned about how do you
> > >>> know
> > >>>>>>> if a
> > >>>>>>>>>>>>>>>>>> checkpoint file needs to be written under EOS, if you
> > >>> do not
> > >>>>>>>>> have a
> > >>>>>>>>>>>>>>> way
> > >>>>>>>>>>>>>>>>>> to know if the state store is transactional or not. If
> > >>> a
> > >>>>> state
> > >>>>>>>>>>> store
> > >>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be written
> > >>> during
> > >>>>>>> normal
> > >>>>>>>>>>>>>>>>>> processing under EOS. If the state store is not
> > >>>>> transactional,
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> checkpoint file must not be written under EOS.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> 7.
> > >>>>>>>>>>>>>>>>>> My point was about not only considering the bytes in
> > >>> memory
> > >>>>> in
> > >>>>>>>>>>> config
> > >>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that
> > >>> might
> > >>>>> be
> > >>>>>>>>>>>>>>> spilled
> > >>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering whether you should
> > >>>>> remove
> > >>>>>>> the
> > >>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory bytes to be used
> > >>> to
> > >>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My thinking
> > >>> was
> > >>>>> that
> > >>>>>>>>> even
> > >>>>>>>>>>>>>>> if a
> > >>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to disk, limiting
> > >>> the
> > >>>>>>>>> overall
> > >>>>>>>>>>>>>>> bytes
> > >>>>>>>>>>>>>>>>>> might make sense. Thinking about it again and
> > >>> considering
> > >>>>> the
> > >>>>>>>>>>> recent
> > >>>>>>>>>>>>>>>>>> discussions, it does not make too much sense anymore.
> > >>>>>>>>>>>>>>>>>> I like the name
> > >>> statestore.transaction.buffer.max.bytes that
> > >>>>>>> you
> > >>>>>>>>>>>>>>> proposed.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> 8.
> > >>>>>>>>>>>>>>>>>> A high-level description (without implementation
> > >>> details) of
> > >>>>>>> how
> > >>>>>>>>>>>>>>> Kafka
> > >>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
> > >>> transactions,
> > >>>>>>> state
> > >>>>>>>>>>> store
> > >>>>>>>>>>>>>>>>>> transactions and checkpointing would be great. Would
> be
> > >>>>> great
> > >>>>>>> if
> > >>>>>>>>>>> you
> > >>>>>>>>>>>>>>>>>> could also add some sentences about the behavior in
> > >>> case of
> > >>>>> a
> > >>>>>>>>>>>>>>> failure.
> > >>>>>>>>>>>>>>>>>> For instance how does a transactional state store
> > >>> recover
> > >>>>>>> after a
> > >>>>>>>>>>>>>>>>>> failure or what happens with the transaction buffer,
> > >>> etc.
> > >>>>>>> (that
> > >>>>>>>>> is
> > >>>>>>>>>>>>>>> what
> > >>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Best,
> > >>>>>>>>>>>>>>>>>> Bruno
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > >>>>>>>>>>>>>>>>>>> Hi Bruno,
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> 1.
> > >>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> > >>> WriteBatchWithIndex
> > >>>>>>>>>>>>>>> transactions
> > >>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
> > >>> configured
> > >>>>>>> memory
> > >>>>>>>>>>>>>>> needs to
> > >>>>>>>>>>>>>>>>>>> trigger an early commit?
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> 2.
> > >>>>>>>>>>>>>>>>>>> This is one of my big concerns. Ultimately, any
> > >>> approach
> > >>>>>>> based
> > >>>>>>>>> on
> > >>>>>>>>>>>>>>>>>> cracking
> > >>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in ways it's not
> > >>> really
> > >>>>>>>>>>> designed
> > >>>>>>>>>>>>>>>>>> for is
> > >>>>>>>>>>>>>>>>>>> likely to have some unforseen performance or
> > >>> consistency
> > >>>>>>> issues.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> 3.
> > >>>>>>>>>>>>>>>>>>> What's your motivation for removing these early
> > >>> commits?
> > >>>>>>> While
> > >>>>>>>>> not
> > >>>>>>>>>>>>>>>>>> ideal, I
> > >>>>>>>>>>>>>>>>>>> think they're a decent compromise to ensure
> > >>> consistency
> > >>>>>>> whilst
> > >>>>>>>>>>>>>>>>>> maintaining
> > >>>>>>>>>>>>>>>>>>> good and predictable performance.
> > >>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
> > >>> complicated, and
> > >>>>>>> might
> > >>>>>>>>>>>>>>>>>> actually
> > >>>>>>>>>>>>>>>>>>> make behaviour less predictable for users as a
> > >>> consequence.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is
> > >>> growing a
> > >>>>>>> bit
> > >>>>>>>>>>> out
> > >>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas for future
> > >>>>>>>>>>> improvements, I
> > >>>>>>>>>>>>>>>>>> think
> > >>>>>>>>>>>>>>>>>>> it's important to narrow the scope down to a design
> > >>> that
> > >>>>>>>>> achieves
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> most
> > >>>>>>>>>>>>>>>>>>> pressing objectives (constant sized restorations
> > >>> during
> > >>>>> dirty
> > >>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that this KIP
> > >>> produces
> > >>>>>>> can
> > >>>>>>>>>>>>>>>>>> ultimately
> > >>>>>>>>>>>>>>>>>>> be changed in the future, especially if the bulk of
> > >>> it is
> > >>>>>>>>> internal
> > >>>>>>>>>>>>>>>>>>> behaviour.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> I'm going to spend some time next week trying to
> > >>> re-work
> > >>>>> the
> > >>>>>>>>>>>>>>> original
> > >>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
> > >>> newTransaction()
> > >>>>>>>>> method,
> > >>>>>>>>>>>>>>> such
> > >>>>>>>>>>>>>>>>>> that
> > >>>>>>>>>>>>>>>>>>> it's just an implementation detail of RocksDBStore.
> > >>> That
> > >>>>>>> way, if
> > >>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>> want to
> > >>>>>>>>>>>>>>>>>>> replace WBWI with something in the future, like the
> > >>> SST
> > >>>>> file
> > >>>>>>>>>>>>>>> management
> > >>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with little/no
> API
> > >>>>>>> changes.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Regards,
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Nick
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>
> > >>
> > >
> >
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Bruno,

Agreed, I can live with that for now.

In an effort to keep the scope of this KIP from expanding, I'm leaning
towards just providing a configurable default.state.isolation.level and
removing IsolationLevel from the StateStoreContext. This would be
compatible with adding support for query-time IsolationLevels in the
future, whilst providing a way for users to select an isolation level now.

The big problem with this, however, is that if a user selects processing.mode
= "exactly-once(-v2|-beta)", and default.state.isolation.level =
"READ_UNCOMMITTED", we need to guarantee that the data isn't written to
disk until commit() is called, but we also need to permit IQ threads to
read from the ongoing transaction.

A simple solution would be to (temporarily) forbid this combination of
configuration, and have default.state.isolation.level automatically switch
to READ_COMMITTED when processing.mode is anything other than
at-least-once. Do you think this would be acceptable?

In a later KIP, we can add support for query-time isolation levels and
solve this particular problem there, which would relax this restriction.

Regards,
Nick

On Tue, 19 Sept 2023 at 09:30, Bruno Cadonna <ca...@apache.org> 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,
> Bruno
>
> On 9/18/23 7:12 PM, Nick Telford wrote:
> > 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 <ni...@gmail.com>
> wrote:
> >
> >> Hi everyone,
> >>
> >> I agree that having IsolationLevel be determined at query-time is the
> >> ideal design, but there are a few sticking points:
> >>
> >> 1.
> >> There needs to be some way to communicate the IsolationLevel down to the
> >> RocksDBStore itself, so that the query can respect it. Since stores are
> >> "layered" in functionality (i.e. ChangeLoggingStore, MeteredStore,
> etc.),
> >> we need some way to deliver that information to the bottom layer. For
> IQv2,
> >> we can use the existing State#query() method, but IQv1 has no way to do
> >> this.
> >>
> >> A simple approach, which would potentially open up other options, would
> be
> >> to add something like: ReadOnlyKeyValueStore<K, V>
> >> readOnlyView(IsolationLevel isolationLevel) to ReadOnlyKeyValueStore
> (and
> >> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
> >>
> >> 2.
> >> As mentioned above, RocksDB WriteBatches are not thread-safe, which
> causes
> >> a problem if we want to provide READ_UNCOMMITTED Iterators. I also had a
> >> look at RocksDB Transactions[1], but they solve a very different
> problem,
> >> and have the same thread-safety issue.
> >>
> >> One possible approach that I mentioned is chaining WriteBatches: every
> >> time a new Interactive Query is received (i.e. readOnlyView, see above,
> >> is called) we "freeze" the existing WriteBatch, and start a new one for
> new
> >> writes. The Interactive Query queries the "chain" of previous
> WriteBatches
> >> + the underlying database; while the StreamThread starts writing to the
> >> *new* WriteBatch. On-commit, the StreamThread would write *all*
> >> WriteBatches in the chain to the database (that have not yet been
> written).
> >>
> >> WriteBatches would be closed/freed only when they have been both
> >> committed, and all open Interactive Queries on them have been closed.
> This
> >> would require some reference counting.
> >>
> >> Obviously a drawback of this approach is the potential for increased
> >> memory usage: if an Interactive Query is long-lived, for example by
> doing a
> >> full scan over a large database, or even just pausing in the middle of
> an
> >> iteration, then the existing chain of WriteBatches could be kept around
> for
> >> a long time, potentially forever.
> >>
> >> --
> >>
> >> A.
> >> Going off on a tangent, it looks like in addition to supporting
> >> READ_COMMITTED queries, we could go further and support REPEATABLE_READ
> >> queries (i.e. where subsequent reads to the same key in the same
> >> Interactive Query are guaranteed to yield the same value) by making use
> of
> >> RocksDB Snapshots[2]. These are fairly lightweight, so the performance
> >> impact is likely to be negligible, but they do require that the
> Interactive
> >> Query session can be explicitly closed.
> >>
> >> This could be achieved if we made the above readOnlyView interface look
> >> more like:
> >>
> >> interface ReadOnlyKeyValueView<K, V> implements ReadOnlyKeyValueStore<K,
> >> V>, AutoCloseable {}
> >>
> >> interface ReadOnlyKeyValueStore<K, V> {
> >>      ...
> >>      ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel
> isolationLevel);
> >> }
> >>
> >> But this would be a breaking change, as existing IQv1 queries are
> >> guaranteed to never call store.close(), and therefore these would leak
> >> memory under REPEATABLE_READ.
> >>
> >> B.
> >> One thing that's notable: MyRocks states that they support
> READ_COMMITTED
> >> and REPEATABLE_READ, but they make no mention of READ_UNCOMMITTED[3][4].
> >> This could be because doing so is technically difficult/impossible using
> >> the primitives available in RocksDB.
> >>
> >> --
> >>
> >> Lucas, to address your points:
> >>
> >> U1.
> >> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
> >> implementations of StateStore that do not support atomic writes.
> Obviously
> >> in those cases, the guarantees Kafka Streams provides/expects would be
> >> relaxed. Do you think we should require all implementations to support
> >> atomic writes?
> >>
> >> U2.
> >> Stores can support multiple IsolationLevels. As we've discussed above,
> the
> >> ideal scenario would be to specify the IsolationLevel at query-time.
> >> Failing that, I think the second-best approach is to define the
> >> IsolationLevel for *all* queries based on the processing.mode, which is
> >> what the default StateStoreContext#isolationLevel() achieves. Would you
> >> prefer an alternative?
> >>
> >> While the existing implementation is equivalent to READ_UNCOMMITTED,
> this
> >> can yield unexpected results/errors under EOS, if a transaction is
> rolled
> >> back. While this would be a change in behaviour for users, it would look
> >> more like a bug fix than a breaking change. That said, we *could* make
> it
> >> configurable, and default to the existing behaviour (READ_UNCOMMITTED)
> >> instead of inferring it from the processing.mode?
> >>
> >> N1, N2.
> >> These were only primitives to avoid boxing costs, but since this is not
> a
> >> performance sensitive area, it should be fine to change if that's
> desirable.
> >>
> >> N3.
> >> It's because the store "manages its own offsets", which includes both
> >> committing the offset, *and providing it* via getCommittedOffset().
> >> Personally, I think "managesOffsets" conveys this best, but I don't mind
> >> changing it if the nomenclature is unclear.
> >>
> >> Sorry for the massive emails/essays!
> >> --
> >> Nick
> >>
> >> 1: https://github.com/facebook/rocksdb/wiki/Transactions
> >> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> >> 3: https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> >> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
> >>
> >> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> >> <lb...@confluent.io.invalid> wrote:
> >>
> >>> Hi Nick,
> >>>
> >>> since I last read it in April, the KIP has become much cleaner and
> >>> easier to read. Great work!
> >>>
> >>> It feels to me the last big open point is whether we can implement
> >>> isolation level as a query parameter. I understand that there are
> >>> implementation concerns, but as Colt says, it would be a great
> >>> addition, and would also simplify the migration path for this change.
> >>> Is the implementation problem you mentioned caused by the WriteBatch
> >>> not having a notion of a snapshot, as the underlying DB iterator does?
> >>> In that case, I am not sure a chain of WriteBatches as you propose
> >>> would fully solve the problem, but maybe I didn't dig enough into the
> >>> details to fully understand it.
> >>>
> >>> If it's not possible to implement it now, would it be an option to
> >>> make sure in this KIP that we do not fully close the door on per-query
> >>> isolation levels in the interface, as it may be possible to implement
> >>> the missing primitives in RocksDB or Speedb in the future.
> >>>
> >>> Understanding:
> >>>
> >>> * U1) Why is it only "SHOULD" for changelogOffsets to be persisted
> >>> atomically with the records?
> >>> * U2) Don't understand the default implementation of `isolationLevel`.
> >>> The isolation level should be a property of the underlying store, and
> >>> not be defined by the default config? Existing stores probably don't
> >>> guarantee READ_COMMITTED, so the default should be to return
> >>> READ_UNCOMMITTED.
> >>>
> >>> Nits:
> >>> * N1) Could `getComittedOffset` use an `OptionalLong` return type, to
> >>> avoid the `null`?
> >>> * N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
> >>> return type, to avoid the `-1`?
> >>> * N3) I don't understand why `managesOffsets` uses the 'manage' verb,
> >>> whereas all other methods use the "commits" verb. I'd suggest
> >>> `commitsOffsets`.
> >>>
> >>> Either way, it feels this KIP is very close to the finish line, I'm
> >>> looking forward to seeing this in production!
> >>>
> >>> Cheers,
> >>> Lucas
> >>>
> >>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <co...@littlehorse.io>
> wrote:
> >>>>
> >>>>> Making IsolationLevel a query-time constraint, rather than linking it
> >>> to
> >>>> the processing.guarantee.
> >>>>
> >>>> As I understand it, would this allow even a user of EOS to control
> >>> whether
> >>>> reading committed or uncommitted records? If so, I am highly in favor
> of
> >>>> this.
> >>>>
> >>>> I know that I was one of the early people to point out the current
> >>>> shortcoming that IQ reads uncommitted records, but just this morning I
> >>>> realized a pattern we use which means that (for certain queries) our
> >>> system
> >>>> needs to be able to read uncommitted records, which is the current
> >>> behavior
> >>>> of Kafka Streams in EOS.***
> >>>>
> >>>> If IsolationLevel being a query-time decision allows for this, then
> that
> >>>> would be amazing. I would also vote that the default behavior should
> be
> >>> for
> >>>> reading uncommitted records, because it is totally possible for a
> valid
> >>>> application to depend on that behavior, and breaking it in a minor
> >>> release
> >>>> might be a bit strong.
> >>>>
> >>>> *** (Note, for the curious reader....) Our use-case/query pattern is a
> >>> bit
> >>>> complex, but reading "uncommitted" records is actually safe in our
> case
> >>>> 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 soon as the processor processes the result. If
> they
> >>>> can't read uncommitted records, they may receive a "201 - Created"
> >>>> response, immediately followed by a "404 - 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 <ni...@gmail.com>
> >>> wrote:
> >>>>
> >>>>> Addendum:
> >>>>>
> >>>>> I think we would also face the same problem with the approach John
> >>> outlined
> >>>>> earlier (using the record cache as a transaction buffer and flushing
> >>> it
> >>>>> straight to SST files). This is because the record cache (the
> >>> ThreadCache
> >>>>> class) is not thread-safe, so every commit would invalidate open IQ
> >>>>> Iterators in the same way that RocksDB WriteBatches do.
> >>>>> --
> >>>>> Nick
> >>>>>
> >>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <ni...@gmail.com>
> >>>>> 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.
> >>>>>> 2. Making IsolationLevel a query-time constraint, rather than
> >>> linking it
> >>>>>> to the processing.guarantee.
> >>>>>>
> >>>>>> There's a wrinkle that makes this a challenge: Interactive Queries
> >>> that
> >>>>>> open an Iterator, when using transactions and READ_UNCOMMITTED.
> >>>>>> The problem is that under READ_UNCOMMITTED, queries need to be able
> >>> to
> >>>>>> read records from the currently uncommitted transaction buffer
> >>>>>> (WriteBatch). This includes for Iterators, which should iterate
> >>> both the
> >>>>>> transaction buffer and underlying database (using
> >>>>>> WriteBatch#iteratorWithBase()).
> >>>>>>
> >>>>>> The issue is that when the StreamThread commits, it writes the
> >>> current
> >>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing the
> >>>>>> WriteBatch while an Interactive Query holds an open Iterator on it
> >>> will
> >>>>>> invalidate the Iterator. Worse, it turns out that Iterators over a
> >>>>>> WriteBatch become invalidated not just when the WriteBatch is
> >>> cleared,
> >>>>> but
> >>>>>> also when the Iterators' current key receives a new write.
> >>>>>>
> >>>>>> Now that I'm writing this, I remember that this is the major reason
> >>> that
> >>>>> I
> >>>>>> switched the original design from having a query-time
> >>> IsolationLevel to
> >>>>>> having the IsolationLevel linked to the transactionality of the
> >>> stores
> >>>>>> themselves.
> >>>>>>
> >>>>>> It *might* be possible to resolve this, by having a "chain" of
> >>>>>> WriteBatches, with the StreamThread switching to a new WriteBatch
> >>>>> whenever
> >>>>>> a new Interactive Query attempts to read from the database, but that
> >>>>> could
> >>>>>> cause some performance problems/memory pressure when subjected to a
> >>> high
> >>>>>> Interactive Query load. It would also reduce the efficiency of
> >>>>> WriteBatches
> >>>>>> on-commit, as we'd have to write N WriteBatches, where N is the
> >>> number of
> >>>>>> Interactive Queries since the last commit.
> >>>>>>
> >>>>>> I realise this is getting into the weeds of the implementation, and
> >>> you'd
> >>>>>> rather we focus on the API for now, but I think it's important to
> >>>>> consider
> >>>>>> how to implement the desired API, in case we come up with an API
> >>> that
> >>>>>> cannot be implemented efficiently, or even at all!
> >>>>>>
> >>>>>> Thoughts?
> >>>>>> --
> >>>>>> Nick
> >>>>>>
> >>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <ca...@apache.org>
> >>> wrote:
> >>>>>>
> >>>>>>> Hi Nick,
> >>>>>>>
> >>>>>>> 6.
> >>>>>>> Of course, you are right! My bad!
> >>>>>>> Wiping out the state in the downgrading case is fine.
> >>>>>>>
> >>>>>>>
> >>>>>>> 3a.
> >>>>>>> Focus on the public facing changes for the KIP. We will manage 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.
> >>>>>>>
> >>>>>>>
> >>>>>>> Best,
> >>>>>>> Bruno
> >>>>>>>
> >>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
> >>>>>>>> Hi Bruno,
> >>>>>>>>
> >>>>>>>> Thanks for getting back to me!
> >>>>>>>>
> >>>>>>>> 2.
> >>>>>>>> The fact that implementations can always track estimated memory
> >>> usage
> >>>>> in
> >>>>>>>> the wrapper is a good point. I can remove -1 as an option, and
> >>> I'll
> >>>>>>> clarify
> >>>>>>>> the JavaDoc that 0 is not just for non-transactional stores,
> >>> which is
> >>>>>>>> currently misleading.
> >>>>>>>>
> >>>>>>>> 6.
> >>>>>>>> The problem with catching the exception in the downgrade process
> >>> is
> >>>>> that
> >>>>>>>> would require new code in the Kafka version being downgraded to.
> >>> Since
> >>>>>>>> users could conceivably downgrade to almost *any* older version
> >>> of
> >>>>> Kafka
> >>>>>>>> Streams, I'm not sure how we could add that code?
> >>>>>>>> The only way I can think of doing it would be to provide a
> >>> dedicated
> >>>>>>>> downgrade tool, that goes through every local store and removes
> >>> the
> >>>>>>>> offsets column families. But that seems like an unnecessary
> >>> amount of
> >>>>>>> extra
> >>>>>>>> code to maintain just to handle a somewhat niche situation, when
> >>> the
> >>>>>>>> alternative (automatically wipe and restore stores) should be
> >>>>>>> acceptable.
> >>>>>>>>
> >>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've requested.
> >>>>>>>>
> >>>>>>>> 3a.
> >>>>>>>> I agree that IsolationLevel makes more sense at query-time, and I
> >>>>>>> actually
> >>>>>>>> initially attempted to place the IsolationLevel at query-time,
> >>> but I
> >>>>> ran
> >>>>>>>> into some problems:
> >>>>>>>> - The key issue is that, under ALOS we're not staging writes in
> >>>>>>>> transactions, so can't perform writes at the READ_COMMITTED
> >>> isolation
> >>>>>>>> level. However, this may be addressed if we decide to *always*
> >>> use
> >>>>>>>> transactions as discussed under 3b.
> >>>>>>>> - IQv1 and IQv2 have quite different implementations. I remember
> >>>>> having
> >>>>>>>> some difficulty understanding the IQv1 internals, which made it
> >>>>>>> difficult
> >>>>>>>> to determine what needed to be changed. However, I *think* this
> >>> can be
> >>>>>>>> addressed for both implementations by wrapping the RocksDBStore
> >>> in an
> >>>>>>>> IsolationLevel-dependent wrapper, that overrides read methods
> >>> (get,
> >>>>>>> etc.)
> >>>>>>>> to either read directly from the database or from the ongoing
> >>>>>>> transaction.
> >>>>>>>> But IQv1 might still be difficult.
> >>>>>>>> - If IsolationLevel becomes a query constraint, then all other
> >>>>>>> StateStores
> >>>>>>>> will need to respect it, including the in-memory stores. This
> >>> would
> >>>>>>> require
> >>>>>>>> us to adapt in-memory stores to stage their writes so they can be
> >>>>>>> isolated
> >>>>>>>> from READ_COMMITTTED queries. It would also become an important
> >>>>>>>> consideration for third-party stores on upgrade, as without
> >>> changes,
> >>>>>>> they
> >>>>>>>> would not support READ_COMMITTED queries correctly.
> >>>>>>>>
> >>>>>>>> Ultimately, I may need some help making the necessary change to
> >>> IQv1
> >>>>> to
> >>>>>>>> support this, but I don't think it's fundamentally impossible,
> >>> if we
> >>>>>>> want
> >>>>>>>> to pursue this route.
> >>>>>>>>
> >>>>>>>> 3b.
> >>>>>>>> The main reason I chose to keep ALOS un-transactional was to
> >>> minimize
> >>>>>>>> behavioural change for most users (I believe most Streams users
> >>> use
> >>>>> the
> >>>>>>>> default configuration, which is ALOS). That said, it's clear
> >>> that if
> >>>>>>> ALOS
> >>>>>>>> also used transactional stores, the only change in behaviour
> >>> would be
> >>>>>>> that
> >>>>>>>> it would become *more correct*, which could be considered a "bug
> >>> fix"
> >>>>> by
> >>>>>>>> users, rather than a change they need to handle.
> >>>>>>>>
> >>>>>>>> I believe that performance using transactions (aka. RocksDB
> >>>>>>> WriteBatches)
> >>>>>>>> should actually be *better* than the un-batched write-path that
> >>> is
> >>>>>>>> currently used[1]. The only "performance" consideration will be
> >>> the
> >>>>>>>> increased memory usage that transactions require. Given the
> >>>>> mitigations
> >>>>>>> for
> >>>>>>>> this memory that we have in place, I would expect that this is
> >>> not a
> >>>>>>>> problem for most users.
> >>>>>>>>
> >>>>>>>> If we're happy to do so, we can make ALOS also use transactions.
> >>>>>>>>
> >>>>>>>> Regards,
> >>>>>>>> Nick
> >>>>>>>>
> >>>>>>>> Link 1:
> >>>>>>>>
> >>>>>
> >>>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>
> >>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <cadonna@apache.org
> >>>>
> >>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi Nick,
> >>>>>>>>>
> >>>>>>>>> Thanks for the updates and sorry for the delay on my side!
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 1.
> >>>>>>>>> Making the default implementation for flush() a no-op sounds
> >>> good to
> >>>>>>> me.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 2.
> >>>>>>>>> I think what was bugging me here is that a third-party state
> >>> store
> >>>>>>> needs
> >>>>>>>>> to implement the state store interface. That means they need to
> >>>>>>>>> implement a wrapper around the actual state store as we do for
> >>>>> RocksDB
> >>>>>>>>> with RocksDBStore. So, a third-party state store can always
> >>> estimate
> >>>>>>> the
> >>>>>>>>> uncommitted bytes, if it wants, because the wrapper can record
> >>> the
> >>>>>>> added
> >>>>>>>>> bytes.
> >>>>>>>>> One case I can think of where returning -1 makes sense is when
> >>>>> Streams
> >>>>>>>>> does not need to estimate the size of the write batch and
> >>> trigger
> >>>>>>>>> extraordinary commits, because the third-party state store
> >>> takes care
> >>>>>>> of
> >>>>>>>>> memory. But in that case the method could also just return 0.
> >>> Even
> >>>>> that
> >>>>>>>>> case would be better solved with a method that returns whether
> >>> the
> >>>>>>> state
> >>>>>>>>> store manages itself the memory used for uncommitted bytes or
> >>> not.
> >>>>>>>>> Said that, I am fine with keeping the -1 return value, I was
> >>> just
> >>>>>>>>> wondering when and if it will be used.
> >>>>>>>>>
> >>>>>>>>> Regarding returning 0 for transactional state stores when the
> >>> batch
> >>>>> is
> >>>>>>>>> empty, I was just wondering because you explicitly stated
> >>>>>>>>>
> >>>>>>>>> "or {@code 0} if this StateStore does not support transactions."
> >>>>>>>>>
> >>>>>>>>> So it seemed to me returning 0 could only happen for
> >>>>> non-transactional
> >>>>>>>>> state stores.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 3.
> >>>>>>>>>
> >>>>>>>>> a) What do you think if we move the isolation level to IQ (v1
> >>> and
> >>>>> v2)?
> >>>>>>>>> In the end this is the only component that really needs to
> >>> specify
> >>>>> the
> >>>>>>>>> isolation level. It is similar to the Kafka consumer that can
> >>> choose
> >>>>>>>>> with what isolation level to read the input topic.
> >>>>>>>>> For IQv1 the isolation level should go into
> >>> StoreQueryParameters. For
> >>>>>>>>> IQv2, I would add it to the Query interface.
> >>>>>>>>>
> >>>>>>>>> b) Point a) raises the question what should happen during
> >>>>> at-least-once
> >>>>>>>>> processing when the state store does not use transactions? John
> >>> in
> >>>>> the
> >>>>>>>>> past proposed to also use transactions on state stores for
> >>>>>>>>> at-least-once. I like that idea, because it avoids aggregating
> >>> the
> >>>>> same
> >>>>>>>>> records over and over again in the case of a failure. We had a
> >>> case
> >>>>> in
> >>>>>>>>> the past where a Streams applications in at-least-once mode was
> >>>>> failing
> >>>>>>>>> continuously for some reasons I do not remember before
> >>> committing the
> >>>>>>>>> offsets. After each failover, the app aggregated again and
> >>> again the
> >>>>>>>>> same records. Of course the aggregate increased to very wrong
> >>> values
> >>>>>>>>> just because of the failover. With transactions on the state
> >>> stores
> >>>>> we
> >>>>>>>>> could have avoided this. The app would have output the same
> >>> aggregate
> >>>>>>>>> multiple times (i.e., after each failover) but at least the
> >>> value of
> >>>>>>> the
> >>>>>>>>> aggregate would not depend on the number of failovers.
> >>> Outputting the
> >>>>>>>>> same aggregate multiple times would be incorrect under
> >>> exactly-once
> >>>>> but
> >>>>>>>>> it is OK for at-least-once.
> >>>>>>>>> If it makes sense to add a config to turn on and off
> >>> transactions on
> >>>>>>>>> state stores under at-least-once or just use transactions in
> >>> any case
> >>>>>>> is
> >>>>>>>>> a question we should also discuss in this KIP. It depends a bit
> >>> on
> >>>>> the
> >>>>>>>>> performance trade-off. Maybe to be safe, I would add a config.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 4.
> >>>>>>>>> Your points are all valid. I tend to say to keep the metrics
> >>> around
> >>>>>>>>> flush() until we remove flush() completely from the interface.
> >>> Calls
> >>>>> to
> >>>>>>>>> flush() might still exist since existing processors might still
> >>> call
> >>>>>>>>> flush() explicitly as you mentioned in 1). For sure, we need to
> >>>>>>> document
> >>>>>>>>> how the metrics change due to the transactions in the upgrade
> >>> notes.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 5.
> >>>>>>>>> I see. Then you should describe how the .position files are
> >>> handled
> >>>>> in
> >>>>>>>>> a dedicated section of the KIP or incorporate the description
> >>> in the
> >>>>>>>>> "Atomic Checkpointing" section instead of only mentioning it in
> >>> the
> >>>>>>>>> "Compatibility, 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:
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>
> >>>
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> >>>>>>>>>
> >>>>>>>>> It is a bit brittle, but it works.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Best,
> >>>>>>>>> Bruno
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
> >>>>>>>>>> Hi Bruno,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for taking the time to review the KIP. I'm back from
> >>> leave
> >>>>> now
> >>>>>>> and
> >>>>>>>>>> intend to move this forwards as quickly as I can.
> >>>>>>>>>>
> >>>>>>>>>> Addressing your points:
> >>>>>>>>>>
> >>>>>>>>>> 1.
> >>>>>>>>>> Because flush() is part of the StateStore API, it's exposed to
> >>>>> custom
> >>>>>>>>>> Processors, which might be making calls to flush(). This was
> >>>>> actually
> >>>>>>> the
> >>>>>>>>>> case in a few integration tests.
> >>>>>>>>>> To maintain as much compatibility as possible, I'd prefer not
> >>> to
> >>>>> make
> >>>>>>>>> this
> >>>>>>>>>> an UnsupportedOperationException, as it will cause previously
> >>>>> working
> >>>>>>>>>> Processors to start throwing exceptions at runtime.
> >>>>>>>>>> I agree that it doesn't make sense for it to proxy commit(),
> >>> though,
> >>>>>>> as
> >>>>>>>>>> that would cause it to violate the "StateStores commit only
> >>> when the
> >>>>>>> Task
> >>>>>>>>>> commits" rule.
> >>>>>>>>>> Instead, I think we should make this a no-op. That way,
> >>> existing
> >>>>> user
> >>>>>>>>>> Processors will continue to work as-before, without violation
> >>> of
> >>>>> store
> >>>>>>>>>> consistency that would be caused by premature flush/commit of
> >>>>>>> StateStore
> >>>>>>>>>> data to disk.
> >>>>>>>>>> What do you think?
> >>>>>>>>>>
> >>>>>>>>>> 2.
> >>>>>>>>>> As stated in the JavaDoc, when a StateStore implementation is
> >>>>>>>>>> transactional, but is unable to estimate the uncommitted memory
> >>>>> usage,
> >>>>>>>>> the
> >>>>>>>>>> method will return -1.
> >>>>>>>>>> The intention here is to permit third-party implementations
> >>> that may
> >>>>>>> not
> >>>>>>>>> be
> >>>>>>>>>> able to estimate memory usage.
> >>>>>>>>>>
> >>>>>>>>>> Yes, it will be 0 when nothing has been written to the store
> >>> yet. I
> >>>>>>>>> thought
> >>>>>>>>>> that was implied by "This method will return an approximation
> >>> of the
> >>>>>>>>> memory
> >>>>>>>>>> would be freed by the next call to {@link #commit(Map)}" and
> >>>>> "@return
> >>>>>>> The
> >>>>>>>>>> approximate size of all records awaiting {@link #commit(Map)}",
> >>>>>>> however,
> >>>>>>>>> I
> >>>>>>>>>> can add it explicitly to the JavaDoc if you think this is
> >>> unclear?
> >>>>>>>>>>
> >>>>>>>>>> 3.
> >>>>>>>>>> I realise this is probably the most contentious point in my
> >>> design,
> >>>>>>> and
> >>>>>>>>> I'm
> >>>>>>>>>> open to changing it if I'm unable to convince you of the
> >>> benefits.
> >>>>>>>>>> Nevertheless, here's my argument:
> >>>>>>>>>> The Interactive Query (IQ) API(s) are directly provided
> >>> StateStores
> >>>>> to
> >>>>>>>>>> query, and it may be important for users to programmatically
> >>> know
> >>>>>>> which
> >>>>>>>>>> mode the StateStore is operating under. If we simply provide an
> >>>>>>>>>> "eosEnabled" boolean (as used throughout the internal streams
> >>>>>>> engine), or
> >>>>>>>>>> similar, then users will need to understand the operation and
> >>>>>>>>> consequences
> >>>>>>>>>> of each available processing mode and how it pertains to their
> >>>>>>>>> StateStore.
> >>>>>>>>>>
> >>>>>>>>>> Interactive Query users aren't the only people that care about
> >>> the
> >>>>>>>>>> processing.mode/IsolationLevel of a StateStore: implementers of
> >>>>> custom
> >>>>>>>>>> StateStores also need to understand the behaviour expected of
> >>> their
> >>>>>>>>>> implementation. KIP-892 introduces some assumptions into the
> >>> Streams
> >>>>>>>>> Engine
> >>>>>>>>>> about how StateStores operate under each processing mode, and
> >>> it's
> >>>>>>>>>> important that custom implementations adhere to those
> >>> assumptions in
> >>>>>>>>> order
> >>>>>>>>>> to maintain the consistency guarantees.
> >>>>>>>>>>
> >>>>>>>>>> IsolationLevels provide a high-level contract on the behaviour
> >>> of
> >>>>> the
> >>>>>>>>>> StateStore: a user knows that under READ_COMMITTED, they will
> >>> see
> >>>>>>> writes
> >>>>>>>>>> only after the Task has committed, and under READ_UNCOMMITTED
> >>> they
> >>>>>>> will
> >>>>>>>>> see
> >>>>>>>>>> writes immediately. No understanding of the details of each
> >>>>>>>>> processing.mode
> >>>>>>>>>> is required, either for IQ users or StateStore implementers.
> >>>>>>>>>>
> >>>>>>>>>> An argument can be made that these contractual guarantees can
> >>> simply
> >>>>>>> be
> >>>>>>>>>> documented for the processing.mode (i.e. that exactly-once and
> >>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and at-least-once
> >>> behaves
> >>>>>>> like
> >>>>>>>>>> READ_UNCOMMITTED), but there are several small issues with
> >>> this I'd
> >>>>>>>>> prefer
> >>>>>>>>>> to avoid:
> >>>>>>>>>>
> >>>>>>>>>>       - Where would we document these contracts, in a way that
> >>> is
> >>>>>>> difficult
> >>>>>>>>>>       for users/implementers to miss/ignore?
> >>>>>>>>>>       - It's not clear to users that the processing mode is
> >>>>>>> communicating
> >>>>>>>>>>       an expectation of read isolation, unless they read the
> >>>>>>>>> documentation. Users
> >>>>>>>>>>       rarely consult documentation unless they feel they need
> >>> to, so
> >>>>>>> it's
> >>>>>>>>> likely
> >>>>>>>>>>       this detail would get missed by many users.
> >>>>>>>>>>       - It tightly couples processing modes to read isolation.
> >>> Adding
> >>>>>>> new
> >>>>>>>>>>       processing modes, or changing the read isolation of
> >>> existing
> >>>>>>>>> processing
> >>>>>>>>>>       modes would be difficult/impossible.
> >>>>>>>>>>
> >>>>>>>>>> Ultimately, the cost of introducing IsolationLevels is just a
> >>> single
> >>>>>>>>>> method, since we re-use the existing IsolationLevel enum from
> >>> Kafka.
> >>>>>>> This
> >>>>>>>>>> gives us a clear place to document the contractual guarantees
> >>>>> expected
> >>>>>>>>>> of/provided by StateStores, that is accessible both by the
> >>>>> StateStore
> >>>>>>>>>> itself, and by IQ users.
> >>>>>>>>>>
> >>>>>>>>>> (Writing this I've just realised that the StateStore and IQ
> >>> APIs
> >>>>>>> actually
> >>>>>>>>>> don't provide access to StateStoreContext that IQ users would
> >>> have
> >>>>>>> direct
> >>>>>>>>>> access to... Perhaps StateStore should expose isolationLevel()
> >>>>> itself
> >>>>>>>>> too?)
> >>>>>>>>>>
> >>>>>>>>>> 4.
> >>>>>>>>>> Yeah, I'm not comfortable renaming the metrics in-place
> >>> either, as
> >>>>>>> it's a
> >>>>>>>>>> backwards incompatible change. My concern is that, if we leave
> >>> the
> >>>>>>>>> existing
> >>>>>>>>>> "flush" metrics in place, they will be confusing to users.
> >>> Right
> >>>>> now,
> >>>>>>>>>> "flush" metrics record explicit flushes to disk, but under
> >>> KIP-892,
> >>>>>>> even
> >>>>>>>>> a
> >>>>>>>>>> commit() will not explicitly flush data to disk - RocksDB will
> >>>>> decide
> >>>>>>> on
> >>>>>>>>>> when to flush memtables to disk itself.
> >>>>>>>>>>
> >>>>>>>>>> If we keep the existing "flush" metrics, we'd have two options,
> >>>>> which
> >>>>>>>>> both
> >>>>>>>>>> seem pretty bad to me:
> >>>>>>>>>>
> >>>>>>>>>>       1. Have them record calls to commit(), which would be
> >>>>>>> misleading, as
> >>>>>>>>>>       data is no longer explicitly "flushed" to disk by this
> >>> call.
> >>>>>>>>>>       2. Have them record nothing at all, which is equivalent to
> >>>>>>> removing
> >>>>>>>>> the
> >>>>>>>>>>       metrics, except that users will see the metric still
> >>> exists and
> >>>>>>> so
> >>>>>>>>> assume
> >>>>>>>>>>       that the metric is correct, and that there's a problem
> >>> with
> >>>>> their
> >>>>>>>>> system
> >>>>>>>>>>       when there isn't.
> >>>>>>>>>>
> >>>>>>>>>> I agree that removing them is also a bad solution, and I'd
> >>> like some
> >>>>>>>>>> guidance on the best path forward here.
> >>>>>>>>>>
> >>>>>>>>>> 5.
> >>>>>>>>>> Position files are updated on every write to a StateStore.
> >>> Since our
> >>>>>>>>> writes
> >>>>>>>>>> are now buffered until commit(), we can't update the Position
> >>> file
> >>>>>>> until
> >>>>>>>>>> commit() has been called, otherwise it would be inconsistent
> >>> with
> >>>>> the
> >>>>>>>>> data
> >>>>>>>>>> in the event of a rollback. Consequently, we need to manage
> >>> these
> >>>>>>> offsets
> >>>>>>>>>> the same way we manage the checkpoint offsets, and ensure
> >>> they're
> >>>>> only
> >>>>>>>>>> written on commit().
> >>>>>>>>>>
> >>>>>>>>>> 6.
> >>>>>>>>>> Agreed, although I'm not exactly sure yet what tests to write.
> >>> How
> >>>>>>>>> explicit
> >>>>>>>>>> do we need to be here in the KIP?
> >>>>>>>>>>
> >>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be seamless,
> >>> and we
> >>>>>>>>> should
> >>>>>>>>>> definitely add some tests around that. Downgrade, it
> >>> transpires,
> >>>>> isn't
> >>>>>>>>>> currently possible, as the extra column family for offset
> >>> storage is
> >>>>>>>>>> incompatible with the pre-KIP-892 implementation: when you
> >>> open a
> >>>>>>> RocksDB
> >>>>>>>>>> database, you must open all available column families or
> >>> receive an
> >>>>>>>>> error.
> >>>>>>>>>> What currently happens on downgrade is that it attempts to
> >>> open the
> >>>>>>>>> store,
> >>>>>>>>>> throws an error about the offsets column family not being
> >>> opened,
> >>>>>>> which
> >>>>>>>>>> triggers a wipe and rebuild of the Task. Given that downgrades
> >>>>> should
> >>>>>>> be
> >>>>>>>>>> uncommon, I think this is acceptable behaviour, as the
> >>> end-state is
> >>>>>>>>>> consistent, even if it results in an undesirable state restore.
> >>>>>>>>>>
> >>>>>>>>>> Should I document the upgrade/downgrade behaviour explicitly
> >>> in the
> >>>>>>> KIP?
> >>>>>>>>>>
> >>>>>>>>>> --
> >>>>>>>>>>
> >>>>>>>>>> Regards,
> >>>>>>>>>> Nick
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
> >>> cadonna@apache.org>
> >>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Nick!
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks for the updates!
> >>>>>>>>>>>
> >>>>>>>>>>> 1.
> >>>>>>>>>>> Why does StateStore#flush() default to
> >>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
> >>>>>>>>>>> Since calls to flush() will not exist anymore after this KIP
> >>> is
> >>>>>>>>>>> released, I would rather throw an unsupported operation
> >>> exception
> >>>>> by
> >>>>>>>>>>> default.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 2.
> >>>>>>>>>>> When would a state store return -1 from
> >>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while being
> >>>>>>> transactional?
> >>>>>>>>>>>
> >>>>>>>>>>> Wouldn't StateStore#approximateNumUncommittedBytes() also
> >>> return 0
> >>>>> if
> >>>>>>>>>>> the state store is transactional but nothing has been written
> >>> to
> >>>>> the
> >>>>>>>>>>> state store yet?
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 3.
> >>>>>>>>>>> Sorry for bringing this up again. Does this KIP really need to
> >>>>>>> introduce
> >>>>>>>>>>> StateStoreContext#isolationLevel()? StateStoreContext has
> >>> already
> >>>>>>>>>>> appConfigs() which basically exposes the same information,
> >>> i.e., if
> >>>>>>> EOS
> >>>>>>>>>>> is enabled or not.
> >>>>>>>>>>> In one of your previous e-mails you wrote:
> >>>>>>>>>>>
> >>>>>>>>>>> "My idea was to try to keep the StateStore interface as
> >>> loosely
> >>>>>>> coupled
> >>>>>>>>>>> from the Streams engine as possible, to give implementers more
> >>>>>>> freedom,
> >>>>>>>>>>> and reduce the amount of internal knowledge required."
> >>>>>>>>>>>
> >>>>>>>>>>> While I understand the intent, I doubt that it decreases the
> >>>>>>> coupling of
> >>>>>>>>>>> a StateStore interface and the Streams engine. READ_COMMITTED
> >>> only
> >>>>>>>>>>> applies to IQ but not to reads by processors. Thus,
> >>> implementers
> >>>>>>> need to
> >>>>>>>>>>> understand how Streams accesses the state stores.
> >>>>>>>>>>>
> >>>>>>>>>>> I would like to hear what others think about this.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 4.
> >>>>>>>>>>> Great exposing new metrics for transactional state stores!
> >>>>> However, I
> >>>>>>>>>>> would prefer to add new metrics and deprecate (in the docs)
> >>> the 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 also need to adapt/add integration tests
> >>> besides
> >>>>> unit
> >>>>>>>>>>> tests. Additionally, we probably need integration or system
> >>> tests
> >>>>> to
> >>>>>>>>>>> verify that upgrades and downgrades between transactional and
> >>>>>>>>>>> non-transactional state stores work as expected.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Best,
> >>>>>>>>>>> Bruno
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> >>>>>>>>>>>> One more thing: I noted John's suggestion in the KIP, under
> >>>>>>> "Rejected
> >>>>>>>>>>>> Alternatives". I still think it's an idea worth pursuing,
> >>> but I
> >>>>>>> believe
> >>>>>>>>>>>> that it's out of the scope of this KIP, because it solves a
> >>>>>>> different
> >>>>>>>>> set
> >>>>>>>>>>>> of problems to this KIP, and the scope of this one has
> >>> already
> >>>>> grown
> >>>>>>>>>>> quite
> >>>>>>>>>>>> large!
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> >>>>> nick.telford@gmail.com>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I've updated the KIP (
> >>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>> )
> >>>>>>>>>>>>> with the latest changes; mostly bringing back "Atomic
> >>>>>>> Checkpointing"
> >>>>>>>>>>> (for
> >>>>>>>>>>>>> what feels like the 10th time!). I think the one thing
> >>> missing is
> >>>>>>> some
> >>>>>>>>>>>>> changes to metrics (notably the store "flush" metrics will
> >>> need
> >>>>> to
> >>>>>>> be
> >>>>>>>>>>>>> renamed to "commit").
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> The reason I brought back Atomic Checkpointing was to
> >>> decouple
> >>>>>>> store
> >>>>>>>>>>> flush
> >>>>>>>>>>>>> from store commit. This is important, because with
> >>> Transactional
> >>>>>>>>>>>>> StateStores, we now need to call "flush" on *every* Task
> >>> commit,
> >>>>>>> and
> >>>>>>>>> not
> >>>>>>>>>>>>> just when the StateStore is closing, otherwise our
> >>> transaction
> >>>>>>> buffer
> >>>>>>>>>>> will
> >>>>>>>>>>>>> never be written and persisted, instead growing unbounded! I
> >>>>>>>>>>> experimented
> >>>>>>>>>>>>> with some simple solutions, like forcing a store flush
> >>> whenever
> >>>>> the
> >>>>>>>>>>>>> transaction buffer was likely to exceed its configured
> >>> size, but
> >>>>>>> this
> >>>>>>>>>>> was
> >>>>>>>>>>>>> brittle: it prevented the transaction buffer from being
> >>>>> configured
> >>>>>>> to
> >>>>>>>>> be
> >>>>>>>>>>>>> unbounded, and it still would have required explicit
> >>> flushes of
> >>>>>>>>> RocksDB,
> >>>>>>>>>>>>> yielding sub-optimal performance and memory utilization.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right" way to
> >>> resolve
> >>>>> this
> >>>>>>>>>>>>> problem. By ensuring that the changelog offsets that
> >>> correspond
> >>>>> to
> >>>>>>> the
> >>>>>>>>>>> most
> >>>>>>>>>>>>> recently written records are always atomically written to
> >>> the
> >>>>>>>>> StateStore
> >>>>>>>>>>>>> (by writing them to the same transaction buffer), we can
> >>> avoid
> >>>>>>>>> forcibly
> >>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting RocksDB
> >>> flush
> >>>>> them
> >>>>>>>>> only
> >>>>>>>>>>>>> when necessary, without losing any of our consistency
> >>> guarantees.
> >>>>>>> See
> >>>>>>>>>>> the
> >>>>>>>>>>>>> updated KIP for more info.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I have fully implemented these changes, although I'm still
> >>> not
> >>>>>>>>> entirely
> >>>>>>>>>>>>> happy with the implementation for segmented StateStores, so
> >>> I
> >>>>> plan
> >>>>>>> to
> >>>>>>>>>>>>> refactor that. Despite that, all tests pass. If you'd like
> >>> to try
> >>>>>>> out
> >>>>>>>>> or
> >>>>>>>>>>>>> review this highly experimental and incomplete branch, it's
> >>>>>>> available
> >>>>>>>>>>> here:
> >>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0.
> >>> Note:
> >>>>>>> it's
> >>>>>>>>>>> built
> >>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base to build
> >>> and test
> >>>>>>> it
> >>>>>>>>> on,
> >>>>>>>>>>>>> and to enable easy apples-to-apples comparisons in a live
> >>>>>>>>> environment. I
> >>>>>>>>>>>>> plan to rebase it against trunk once it's nearer completion
> >>> and
> >>>>> has
> >>>>>>>>> been
> >>>>>>>>>>>>> proven on our main application.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I would really appreciate help in reviewing and testing:
> >>>>>>>>>>>>> - Segmented (Versioned, Session and Window) stores
> >>>>>>>>>>>>> - Global stores
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> As I do not currently use either of these, so my primary
> >>> test
> >>>>>>>>>>> environment
> >>>>>>>>>>>>> doesn't test these areas.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I'm going on Parental Leave starting next week for a few
> >>> weeks,
> >>>>> so
> >>>>>>>>> will
> >>>>>>>>>>>>> not have time to move this forward until late August. That
> >>> said,
> >>>>>>> your
> >>>>>>>>>>>>> feedback is welcome and appreciated, I just won't be able to
> >>>>>>> respond
> >>>>>>>>> as
> >>>>>>>>>>>>> quickly as usual.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> >>>>> nick.telford@gmail.com>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi Bruno
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ is not
> >>> something
> >>>>> I
> >>>>>>> had
> >>>>>>>>>>>>>> considered.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> What about atomically updating the state store from the
> >>>>>>> transaction
> >>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
> >>> (thus,
> >>>>>>>>>>> flushing
> >>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> >>> number of
> >>>>>>>>> commit
> >>>>>>>>>>>>>>> intervals?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting that we
> >>> add an
> >>>>>>>>>>> additional
> >>>>>>>>>>>>>> config for the max number of commit intervals between
> >>>>> checkpoints?
> >>>>>>>>> That
> >>>>>>>>>>>>>> way, we would checkpoint *either* when the transaction
> >>> buffers
> >>>>> are
> >>>>>>>>>>> nearly
> >>>>>>>>>>>>>> full, *OR* whenever a certain number of commit intervals
> >>> have
> >>>>>>>>> elapsed,
> >>>>>>>>>>>>>> whichever comes first?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> That certainly seems reasonable, although this re-ignites
> >>> an
> >>>>>>> earlier
> >>>>>>>>>>>>>> debate about whether a config should be measured in
> >>> "number of
> >>>>>>> commit
> >>>>>>>>>>>>>> intervals", instead of just an absolute time.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> FWIW, I realised that this issue is the reason I was
> >>> pursuing
> >>>>> the
> >>>>>>>>>>> Atomic
> >>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush from
> >>> checkpointing,
> >>>>>>>>> which
> >>>>>>>>>>>>>> enables us to just checkpoint on every commit without any
> >>>>>>> performance
> >>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the "best"
> >>> solution,
> >>>>>>> but
> >>>>>>>>>>> I'm not
> >>>>>>>>>>>>>> sure if this is enough to bring it back into this KIP.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I'm currently working on moving all the transactional logic
> >>>>>>> directly
> >>>>>>>>>>> into
> >>>>>>>>>>>>>> RocksDBStore itself, which does away with the
> >>>>>>>>> StateStore#newTransaction
> >>>>>>>>>>>>>> method, and reduces the number of new classes introduced,
> >>>>>>>>> significantly
> >>>>>>>>>>>>>> reducing the complexity. If it works, and the complexity is
> >>>>>>>>> drastically
> >>>>>>>>>>>>>> reduced, I may try bringing back Atomic Checkpoints into
> >>> this
> >>>>> KIP.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
> >>> cadonna@apache.org>
> >>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> As far as I understand, you want to atomically update the
> >>> state
> >>>>>>>>> store
> >>>>>>>>>>>>>>> from the transaction buffer, flush the memtable of a state
> >>>>> store
> >>>>>>> and
> >>>>>>>>>>>>>>> write the checkpoint not after the commit time elapsed but
> >>>>> after
> >>>>>>> the
> >>>>>>>>>>>>>>> transaction buffer reached a size that would lead to
> >>> exceeding
> >>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes before the next
> >>> commit
> >>>>>>>>>>> interval
> >>>>>>>>>>>>>>> ends.
> >>>>>>>>>>>>>>> That means, the Kafka transaction would commit every
> >>> commit
> >>>>>>> interval
> >>>>>>>>>>> but
> >>>>>>>>>>>>>>> the state store will only be atomically updated roughly
> >>> every
> >>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ
> >>> would
> >>>>>>> then
> >>>>>>>>>>> only
> >>>>>>>>>>>>>>> see new data roughly every
> >>>>>>> statestore.transaction.buffer.max.bytes.
> >>>>>>>>>>>>>>> After a failure the state store needs to restore up to
> >>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Is this correct?
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> What about atomically updating the state store from the
> >>>>>>> transaction
> >>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
> >>> (thus,
> >>>>>>>>>>> flushing
> >>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
> >>> number of
> >>>>>>>>> commit
> >>>>>>>>>>>>>>> intervals? In such a way, we would have the same delay for
> >>>>>>> records
> >>>>>>>>>>>>>>> appearing in output topics and IQ because both would
> >>> appear
> >>>>> when
> >>>>>>> the
> >>>>>>>>>>>>>>> Kafka transaction is committed. However, after a failure
> >>> the
> >>>>>>> state
> >>>>>>>>>>> store
> >>>>>>>>>>>>>>> still needs to restore up to
> >>>>>>> statestore.transaction.buffer.max.bytes
> >>>>>>>>>>> and
> >>>>>>>>>>>>>>> it might restore data that is already in the state store
> >>>>> because
> >>>>>>> the
> >>>>>>>>>>>>>>> checkpoint lags behind the last stable offset (i.e. the
> >>> last
> >>>>>>>>> committed
> >>>>>>>>>>>>>>> offset) of the changelog topics. Restoring data that is
> >>> already
> >>>>>>> in
> >>>>>>>>> the
> >>>>>>>>>>>>>>> state store is idempotent, so eos should not violated.
> >>>>>>>>>>>>>>> This solution needs at least one new config to specify
> >>> when a
> >>>>>>>>>>> checkpoint
> >>>>>>>>>>>>>>> should be written.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> A small correction to your previous e-mail that does not
> >>> change
> >>>>>>>>>>> anything
> >>>>>>>>>>>>>>> you said: Under alos the default commit interval is 30
> >>> seconds,
> >>>>>>> not
> >>>>>>>>>>> five
> >>>>>>>>>>>>>>> seconds.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> >>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I've begun performance testing my branch on our staging
> >>>>>>>>> environment,
> >>>>>>>>>>>>>>>> putting it through its paces in our non-trivial
> >>> application.
> >>>>> I'm
> >>>>>>>>>>>>>>> already
> >>>>>>>>>>>>>>>> observing the same increased flush rate that we saw the
> >>> last
> >>>>>>> time
> >>>>>>>>> we
> >>>>>>>>>>>>>>>> attempted to use a version of this KIP, but this time, I
> >>>>> think I
> >>>>>>>>> know
> >>>>>>>>>>>>>>> why.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at
> >>> the end
> >>>>>>> of
> >>>>>>>>> the
> >>>>>>>>>>>>>>> Task
> >>>>>>>>>>>>>>>> commit process, has the following behaviour:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         - Under ALOS: checkpoint the state stores. This
> >>>>> includes
> >>>>>>>>>>>>>>>>         flushing memtables in RocksDB. This is acceptable
> >>>>>>> because the
> >>>>>>>>>>>>>>> default
> >>>>>>>>>>>>>>>>         commit.interval.ms is 5 seconds, so forcibly
> >>> flushing
> >>>>>>>>> memtables
> >>>>>>>>>>>>>>> every 5
> >>>>>>>>>>>>>>>>         seconds is acceptable for most applications.
> >>>>>>>>>>>>>>>>         - Under EOS: checkpointing is not done, *unless*
> >>> it's
> >>>>>>> being
> >>>>>>>>>>>>>>> forced, due
> >>>>>>>>>>>>>>>>         to e.g. the Task closing or being revoked. This
> >>> means
> >>>>>>> that
> >>>>>>>>> under
> >>>>>>>>>>>>>>> normal
> >>>>>>>>>>>>>>>>         processing conditions, the state stores will not
> >>> be
> >>>>>>>>>>> checkpointed,
> >>>>>>>>>>>>>>> and will
> >>>>>>>>>>>>>>>>         not have memtables flushed at all , unless RocksDB
> >>>>>>> decides to
> >>>>>>>>>>>>>>> flush them on
> >>>>>>>>>>>>>>>>         its own. Checkpointing stores and force-flushing
> >>> their
> >>>>>>>>> memtables
> >>>>>>>>>>>>>>> is only
> >>>>>>>>>>>>>>>>         done when a Task is being closed.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least
> >>>>> *some*
> >>>>>>>>>>> normal
> >>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB transaction
> >>>>> buffers
> >>>>>>> to
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> state stores, and to ensure the offsets are synced to
> >>> disk to
> >>>>>>>>> prevent
> >>>>>>>>>>>>>>>> restores from getting out of hand. Consequently, my
> >>> current
> >>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is
> >>> far too
> >>>>>>>>>>>>>>> frequent.
> >>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records, which is a
> >>>>> change
> >>>>>>> in
> >>>>>>>>>>>>>>> flush
> >>>>>>>>>>>>>>>> behaviour, potentially causing performance problems for
> >>> some
> >>>>>>>>>>>>>>> applications.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm currently
> >>> leaning
> >>>>>>>>>>> towards
> >>>>>>>>>>>>>>>> using the statestore.transaction.buffer.max.bytes
> >>>>> configuration
> >>>>>>> to
> >>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to exceed it. This
> >>> would
> >>>>>>>>>>>>>>> complement the
> >>>>>>>>>>>>>>>> existing "early Task commit" functionality that this
> >>>>>>> configuration
> >>>>>>>>>>>>>>>> provides, in the following way:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>         - Currently, we use
> >>>>>>> statestore.transaction.buffer.max.bytes
> >>>>>>>>> to
> >>>>>>>>>>>>>>> force an
> >>>>>>>>>>>>>>>>         early Task commit if processing more records would
> >>>>> cause
> >>>>>>> our
> >>>>>>>>>>> state
> >>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>         transactions to exceed the memory assigned to
> >>> them.
> >>>>>>>>>>>>>>>>         - New functionality: when a Task *does* commit,
> >>> we will
> >>>>>>> not
> >>>>>>>>>>>>>>> checkpoint
> >>>>>>>>>>>>>>>>         the stores (and hence flush the transaction
> >>> buffers)
> >>>>>>> unless
> >>>>>>>>> we
> >>>>>>>>>>>>>>> expect to
> >>>>>>>>>>>>>>>>         cross the statestore.transaction.buffer.max.bytes
> >>>>>>> threshold
> >>>>>>>>>>> before
> >>>>>>>>>>>>>>> the next
> >>>>>>>>>>>>>>>>         commit
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I'm also open to suggestions.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> >>>>>>> nick.telford@gmail.com
> >>>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi Bruno!
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>> By "less predictable for users", I meant in terms of
> >>>>>>> understanding
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> performance profile under various circumstances. The
> >>> more
> >>>>>>> complex
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> solution, the more difficult it would be for users to
> >>>>>>> understand
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>> performance they see. For example, spilling records to
> >>> disk
> >>>>>>> when
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>> transaction buffer reaches a threshold would, I expect,
> >>>>> reduce
> >>>>>>>>> write
> >>>>>>>>>>>>>>>>> throughput. This reduction in write throughput could be
> >>>>>>>>> unexpected,
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> potentially difficult to diagnose/understand for users.
> >>>>>>>>>>>>>>>>> At the moment, I think the "early commit" concept is
> >>>>> relatively
> >>>>>>>>>>>>>>>>> straightforward; it's easy to document, and conceptually
> >>>>> fairly
> >>>>>>>>>>>>>>> obvious to
> >>>>>>>>>>>>>>>>> users. We could probably add a metric to make it easier
> >>> to
> >>>>>>>>>>> understand
> >>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>> it happens though.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 3. (the second one)
> >>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an indirect way of
> >>>>> telling
> >>>>>>>>>>>>>>> StateStores
> >>>>>>>>>>>>>>>>> whether they should be transactional. READ_COMMITTED
> >>>>>>> essentially
> >>>>>>>>>>>>>>> requires
> >>>>>>>>>>>>>>>>> transactions, because it dictates that two threads
> >>> calling
> >>>>>>>>>>>>>>>>> `newTransaction()` should not see writes from the other
> >>>>>>>>> transaction
> >>>>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all
> >>> bets are
> >>>>>>> off,
> >>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> stores can allow threads to observe written records at
> >>> any
> >>>>>>> time,
> >>>>>>>>>>>>>>> which is
> >>>>>>>>>>>>>>>>> essentially "no transactions". That said, StateStores
> >>> are
> >>>>> free
> >>>>>>> to
> >>>>>>>>>>>>>>> implement
> >>>>>>>>>>>>>>>>> these guarantees however they can, which is a bit more
> >>>>> relaxed
> >>>>>>>>> than
> >>>>>>>>>>>>>>>>> dictating "you must use transactions". For example, with
> >>>>>>> RocksDB
> >>>>>>>>> we
> >>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
> >>>>> "transactions",
> >>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But
> >>> with
> >>>>>>> other
> >>>>>>>>>>>>>>> storage
> >>>>>>>>>>>>>>>>> engines, it might be preferable to *always* use
> >>> transactions,
> >>>>>>> even
> >>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines that don't
> >>>>> provide
> >>>>>>>>>>>>>>>>> transactions, but the isolation guarantees can be met
> >>> using a
> >>>>>>>>>>>>>>> different
> >>>>>>>>>>>>>>>>> technique.
> >>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore interface as
> >>>>> loosely
> >>>>>>>>>>> coupled
> >>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
> >>> implementers
> >>>>> more
> >>>>>>>>>>>>>>> freedom, and
> >>>>>>>>>>>>>>>>> reduce the amount of internal knowledge required.
> >>>>>>>>>>>>>>>>> That said, I understand that "IsolationLevel" might not
> >>> be
> >>>>> the
> >>>>>>>>> right
> >>>>>>>>>>>>>>>>> abstraction, and we can always make it much more
> >>> explicit if
> >>>>>>>>>>>>>>> required, e.g.
> >>>>>>>>>>>>>>>>> boolean transactional()
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 7-8.
> >>>>>>>>>>>>>>>>> I can make these changes either later today or tomorrow.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Small update:
> >>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of
> >>> issues
> >>>>>>> that
> >>>>>>>>>>>>>>> needed
> >>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass, which is
> >>>>> promising,
> >>>>>>> but
> >>>>>>>>>>> it
> >>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>> need to undergo some performance testing. I haven't
> >>> (yet)
> >>>>>>> worked
> >>>>>>>>> on
> >>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I would
> >>> expect
> >>>>> that,
> >>>>>>>>>>>>>>>>> behaviourally, it should make no difference. The branch
> >>> is
> >>>>>>>>> available
> >>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if
> >>>>> anyone
> >>>>>>> is
> >>>>>>>>>>>>>>>>> interested in taking an early look.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> >>>>>>> cadonna@apache.org>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually also my
> >>> point. I
> >>>>>>>>>>> understood
> >>>>>>>>>>>>>>>>>> that John was proposing the ingestion path as a way to
> >>> avoid
> >>>>>>> the
> >>>>>>>>>>>>>>> early
> >>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>> I agree with John here, that actually it is public
> >>> API. My
> >>>>>>>>> question
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> how this usage pattern affects normal processing.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>> My concern is that checking for the size of the
> >>> transaction
> >>>>>>>>> buffer
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> maybe triggering an early commit affects the whole
> >>>>> processing
> >>>>>>> of
> >>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>> Streams. The transactionality of a state store is not
> >>>>>>> confined to
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> state store itself, but spills over and changes the
> >>> behavior
> >>>>>>> of
> >>>>>>>>>>> other
> >>>>>>>>>>>>>>>>>> parts of the system. I agree with you that it is a
> >>> decent
> >>>>>>>>>>>>>>> compromise. I
> >>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and list the
> >>> options to
> >>>>>>>>>>> overcome
> >>>>>>>>>>>>>>>>>> them. I also agree with you that all options seem quite
> >>>>> heavy
> >>>>>>>>>>>>>>> compared
> >>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you mean with
> >>> "less
> >>>>>>>>>>>>>>> predictable
> >>>>>>>>>>>>>>>>>> for users", though.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I found the discussions about the alternatives really
> >>>>>>>>> interesting.
> >>>>>>>>>>>>>>> But I
> >>>>>>>>>>>>>>>>>> also think that your plan sounds good and we should
> >>> continue
> >>>>>>> with
> >>>>>>>>>>> it!
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind putting
> >>> isolation
> >>>>>>>>> level
> >>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>> the state store context. Thanks! Should that also be a
> >>> way
> >>>>> to
> >>>>>>>>> give
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> the state store the opportunity to decide whether to
> >>> turn on
> >>>>>>>>>>>>>>>>>> transactions or not?
> >>>>>>>>>>>>>>>>>> With my comment, I was more concerned about how do you
> >>> know
> >>>>>>> if a
> >>>>>>>>>>>>>>>>>> checkpoint file needs to be written under EOS, if you
> >>> do not
> >>>>>>>>> have a
> >>>>>>>>>>>>>>> way
> >>>>>>>>>>>>>>>>>> to know if the state store is transactional or not. If
> >>> a
> >>>>> state
> >>>>>>>>>>> store
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be written
> >>> during
> >>>>>>> normal
> >>>>>>>>>>>>>>>>>> processing under EOS. If the state store is not
> >>>>> transactional,
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>>> checkpoint file must not be written under EOS.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 7.
> >>>>>>>>>>>>>>>>>> My point was about not only considering the bytes in
> >>> memory
> >>>>> in
> >>>>>>>>>>> config
> >>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that
> >>> might
> >>>>> be
> >>>>>>>>>>>>>>> spilled
> >>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering whether you should
> >>>>> remove
> >>>>>>> the
> >>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory bytes to be used
> >>> to
> >>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My thinking
> >>> was
> >>>>> that
> >>>>>>>>> even
> >>>>>>>>>>>>>>> if a
> >>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to disk, limiting
> >>> the
> >>>>>>>>> overall
> >>>>>>>>>>>>>>> bytes
> >>>>>>>>>>>>>>>>>> might make sense. Thinking about it again and
> >>> considering
> >>>>> the
> >>>>>>>>>>> recent
> >>>>>>>>>>>>>>>>>> discussions, it does not make too much sense anymore.
> >>>>>>>>>>>>>>>>>> I like the name
> >>> statestore.transaction.buffer.max.bytes that
> >>>>>>> you
> >>>>>>>>>>>>>>> proposed.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 8.
> >>>>>>>>>>>>>>>>>> A high-level description (without implementation
> >>> details) of
> >>>>>>> how
> >>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
> >>> transactions,
> >>>>>>> state
> >>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>> transactions and checkpointing would be great. Would be
> >>>>> great
> >>>>>>> if
> >>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>> could also add some sentences about the behavior in
> >>> case of
> >>>>> a
> >>>>>>>>>>>>>>> failure.
> >>>>>>>>>>>>>>>>>> For instance how does a transactional state store
> >>> recover
> >>>>>>> after a
> >>>>>>>>>>>>>>>>>> failure or what happens with the transaction buffer,
> >>> etc.
> >>>>>>> (that
> >>>>>>>>> is
> >>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>> Bruno
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 1.
> >>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
> >>> WriteBatchWithIndex
> >>>>>>>>>>>>>>> transactions
> >>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
> >>> configured
> >>>>>>> memory
> >>>>>>>>>>>>>>> needs to
> >>>>>>>>>>>>>>>>>>> trigger an early commit?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 2.
> >>>>>>>>>>>>>>>>>>> This is one of my big concerns. Ultimately, any
> >>> approach
> >>>>>>> based
> >>>>>>>>> on
> >>>>>>>>>>>>>>>>>> cracking
> >>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in ways it's not
> >>> really
> >>>>>>>>>>> designed
> >>>>>>>>>>>>>>>>>> for is
> >>>>>>>>>>>>>>>>>>> likely to have some unforseen performance or
> >>> consistency
> >>>>>>> issues.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 3.
> >>>>>>>>>>>>>>>>>>> What's your motivation for removing these early
> >>> commits?
> >>>>>>> While
> >>>>>>>>> not
> >>>>>>>>>>>>>>>>>> ideal, I
> >>>>>>>>>>>>>>>>>>> think they're a decent compromise to ensure
> >>> consistency
> >>>>>>> whilst
> >>>>>>>>>>>>>>>>>> maintaining
> >>>>>>>>>>>>>>>>>>> good and predictable performance.
> >>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
> >>> complicated, and
> >>>>>>> might
> >>>>>>>>>>>>>>>>>> actually
> >>>>>>>>>>>>>>>>>>> make behaviour less predictable for users as a
> >>> consequence.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is
> >>> growing a
> >>>>>>> bit
> >>>>>>>>>>> out
> >>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas for future
> >>>>>>>>>>> improvements, I
> >>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>> it's important to narrow the scope down to a design
> >>> that
> >>>>>>>>> achieves
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> most
> >>>>>>>>>>>>>>>>>>> pressing objectives (constant sized restorations
> >>> during
> >>>>> dirty
> >>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that this KIP
> >>> produces
> >>>>>>> can
> >>>>>>>>>>>>>>>>>> ultimately
> >>>>>>>>>>>>>>>>>>> be changed in the future, especially if the bulk of
> >>> it is
> >>>>>>>>> internal
> >>>>>>>>>>>>>>>>>>> behaviour.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I'm going to spend some time next week trying to
> >>> re-work
> >>>>> the
> >>>>>>>>>>>>>>> original
> >>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
> >>> newTransaction()
> >>>>>>>>> method,
> >>>>>>>>>>>>>>> such
> >>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> it's just an implementation detail of RocksDBStore.
> >>> That
> >>>>>>> way, if
> >>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> want to
> >>>>>>>>>>>>>>>>>>> replace WBWI with something in the future, like the
> >>> SST
> >>>>> file
> >>>>>>>>>>>>>>> management
> >>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with little/no API
> >>>>>>> changes.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> >>
> >
>

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

Posted by Bruno Cadonna <ca...@apache.org>.
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,
Bruno

On 9/18/23 7:12 PM, Nick Telford wrote:
> 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 <ni...@gmail.com> wrote:
> 
>> Hi everyone,
>>
>> I agree that having IsolationLevel be determined at query-time is the
>> ideal design, but there are a few sticking points:
>>
>> 1.
>> There needs to be some way to communicate the IsolationLevel down to the
>> RocksDBStore itself, so that the query can respect it. Since stores are
>> "layered" in functionality (i.e. ChangeLoggingStore, MeteredStore, etc.),
>> we need some way to deliver that information to the bottom layer. For IQv2,
>> we can use the existing State#query() method, but IQv1 has no way to do
>> this.
>>
>> A simple approach, which would potentially open up other options, would be
>> to add something like: ReadOnlyKeyValueStore<K, V>
>> readOnlyView(IsolationLevel isolationLevel) to ReadOnlyKeyValueStore (and
>> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
>>
>> 2.
>> As mentioned above, RocksDB WriteBatches are not thread-safe, which causes
>> a problem if we want to provide READ_UNCOMMITTED Iterators. I also had a
>> look at RocksDB Transactions[1], but they solve a very different problem,
>> and have the same thread-safety issue.
>>
>> One possible approach that I mentioned is chaining WriteBatches: every
>> time a new Interactive Query is received (i.e. readOnlyView, see above,
>> is called) we "freeze" the existing WriteBatch, and start a new one for new
>> writes. The Interactive Query queries the "chain" of previous WriteBatches
>> + the underlying database; while the StreamThread starts writing to the
>> *new* WriteBatch. On-commit, the StreamThread would write *all*
>> WriteBatches in the chain to the database (that have not yet been written).
>>
>> WriteBatches would be closed/freed only when they have been both
>> committed, and all open Interactive Queries on them have been closed. This
>> would require some reference counting.
>>
>> Obviously a drawback of this approach is the potential for increased
>> memory usage: if an Interactive Query is long-lived, for example by doing a
>> full scan over a large database, or even just pausing in the middle of an
>> iteration, then the existing chain of WriteBatches could be kept around for
>> a long time, potentially forever.
>>
>> --
>>
>> A.
>> Going off on a tangent, it looks like in addition to supporting
>> READ_COMMITTED queries, we could go further and support REPEATABLE_READ
>> queries (i.e. where subsequent reads to the same key in the same
>> Interactive Query are guaranteed to yield the same value) by making use of
>> RocksDB Snapshots[2]. These are fairly lightweight, so the performance
>> impact is likely to be negligible, but they do require that the Interactive
>> Query session can be explicitly closed.
>>
>> This could be achieved if we made the above readOnlyView interface look
>> more like:
>>
>> interface ReadOnlyKeyValueView<K, V> implements ReadOnlyKeyValueStore<K,
>> V>, AutoCloseable {}
>>
>> interface ReadOnlyKeyValueStore<K, V> {
>>      ...
>>      ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel isolationLevel);
>> }
>>
>> But this would be a breaking change, as existing IQv1 queries are
>> guaranteed to never call store.close(), and therefore these would leak
>> memory under REPEATABLE_READ.
>>
>> B.
>> One thing that's notable: MyRocks states that they support READ_COMMITTED
>> and REPEATABLE_READ, but they make no mention of READ_UNCOMMITTED[3][4].
>> This could be because doing so is technically difficult/impossible using
>> the primitives available in RocksDB.
>>
>> --
>>
>> Lucas, to address your points:
>>
>> U1.
>> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
>> implementations of StateStore that do not support atomic writes. Obviously
>> in those cases, the guarantees Kafka Streams provides/expects would be
>> relaxed. Do you think we should require all implementations to support
>> atomic writes?
>>
>> U2.
>> Stores can support multiple IsolationLevels. As we've discussed above, the
>> ideal scenario would be to specify the IsolationLevel at query-time.
>> Failing that, I think the second-best approach is to define the
>> IsolationLevel for *all* queries based on the processing.mode, which is
>> what the default StateStoreContext#isolationLevel() achieves. Would you
>> prefer an alternative?
>>
>> While the existing implementation is equivalent to READ_UNCOMMITTED, this
>> can yield unexpected results/errors under EOS, if a transaction is rolled
>> back. While this would be a change in behaviour for users, it would look
>> more like a bug fix than a breaking change. That said, we *could* make it
>> configurable, and default to the existing behaviour (READ_UNCOMMITTED)
>> instead of inferring it from the processing.mode?
>>
>> N1, N2.
>> These were only primitives to avoid boxing costs, but since this is not a
>> performance sensitive area, it should be fine to change if that's desirable.
>>
>> N3.
>> It's because the store "manages its own offsets", which includes both
>> committing the offset, *and providing it* via getCommittedOffset().
>> Personally, I think "managesOffsets" conveys this best, but I don't mind
>> changing it if the nomenclature is unclear.
>>
>> Sorry for the massive emails/essays!
>> --
>> Nick
>>
>> 1: https://github.com/facebook/rocksdb/wiki/Transactions
>> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
>> 3: https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
>> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
>>
>> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
>> <lb...@confluent.io.invalid> wrote:
>>
>>> Hi Nick,
>>>
>>> since I last read it in April, the KIP has become much cleaner and
>>> easier to read. Great work!
>>>
>>> It feels to me the last big open point is whether we can implement
>>> isolation level as a query parameter. I understand that there are
>>> implementation concerns, but as Colt says, it would be a great
>>> addition, and would also simplify the migration path for this change.
>>> Is the implementation problem you mentioned caused by the WriteBatch
>>> not having a notion of a snapshot, as the underlying DB iterator does?
>>> In that case, I am not sure a chain of WriteBatches as you propose
>>> would fully solve the problem, but maybe I didn't dig enough into the
>>> details to fully understand it.
>>>
>>> If it's not possible to implement it now, would it be an option to
>>> make sure in this KIP that we do not fully close the door on per-query
>>> isolation levels in the interface, as it may be possible to implement
>>> the missing primitives in RocksDB or Speedb in the future.
>>>
>>> Understanding:
>>>
>>> * U1) Why is it only "SHOULD" for changelogOffsets to be persisted
>>> atomically with the records?
>>> * U2) Don't understand the default implementation of `isolationLevel`.
>>> The isolation level should be a property of the underlying store, and
>>> not be defined by the default config? Existing stores probably don't
>>> guarantee READ_COMMITTED, so the default should be to return
>>> READ_UNCOMMITTED.
>>>
>>> Nits:
>>> * N1) Could `getComittedOffset` use an `OptionalLong` return type, to
>>> avoid the `null`?
>>> * N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
>>> return type, to avoid the `-1`?
>>> * N3) I don't understand why `managesOffsets` uses the 'manage' verb,
>>> whereas all other methods use the "commits" verb. I'd suggest
>>> `commitsOffsets`.
>>>
>>> Either way, it feels this KIP is very close to the finish line, I'm
>>> looking forward to seeing this in production!
>>>
>>> Cheers,
>>> Lucas
>>>
>>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <co...@littlehorse.io> wrote:
>>>>
>>>>> Making IsolationLevel a query-time constraint, rather than linking it
>>> to
>>>> the processing.guarantee.
>>>>
>>>> As I understand it, would this allow even a user of EOS to control
>>> whether
>>>> reading committed or uncommitted records? If so, I am highly in favor of
>>>> this.
>>>>
>>>> I know that I was one of the early people to point out the current
>>>> shortcoming that IQ reads uncommitted records, but just this morning I
>>>> realized a pattern we use which means that (for certain queries) our
>>> system
>>>> needs to be able to read uncommitted records, which is the current
>>> behavior
>>>> of Kafka Streams in EOS.***
>>>>
>>>> If IsolationLevel being a query-time decision allows for this, then that
>>>> would be amazing. I would also vote that the default behavior should be
>>> for
>>>> reading uncommitted records, because it is totally possible for a valid
>>>> application to depend on that behavior, and breaking it in a minor
>>> release
>>>> might be a bit strong.
>>>>
>>>> *** (Note, for the curious reader....) Our use-case/query pattern is a
>>> bit
>>>> complex, but reading "uncommitted" records is actually safe in our case
>>>> 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 soon as the processor processes the result. If they
>>>> can't read uncommitted records, they may receive a "201 - Created"
>>>> response, immediately followed by a "404 - 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 <ni...@gmail.com>
>>> wrote:
>>>>
>>>>> Addendum:
>>>>>
>>>>> I think we would also face the same problem with the approach John
>>> outlined
>>>>> earlier (using the record cache as a transaction buffer and flushing
>>> it
>>>>> straight to SST files). This is because the record cache (the
>>> ThreadCache
>>>>> class) is not thread-safe, so every commit would invalidate open IQ
>>>>> Iterators in the same way that RocksDB WriteBatches do.
>>>>> --
>>>>> Nick
>>>>>
>>>>> On Wed, 13 Sept 2023 at 16:58, Nick Telford <ni...@gmail.com>
>>>>> 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.
>>>>>> 2. Making IsolationLevel a query-time constraint, rather than
>>> linking it
>>>>>> to the processing.guarantee.
>>>>>>
>>>>>> There's a wrinkle that makes this a challenge: Interactive Queries
>>> that
>>>>>> open an Iterator, when using transactions and READ_UNCOMMITTED.
>>>>>> The problem is that under READ_UNCOMMITTED, queries need to be able
>>> to
>>>>>> read records from the currently uncommitted transaction buffer
>>>>>> (WriteBatch). This includes for Iterators, which should iterate
>>> both the
>>>>>> transaction buffer and underlying database (using
>>>>>> WriteBatch#iteratorWithBase()).
>>>>>>
>>>>>> The issue is that when the StreamThread commits, it writes the
>>> current
>>>>>> WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing the
>>>>>> WriteBatch while an Interactive Query holds an open Iterator on it
>>> will
>>>>>> invalidate the Iterator. Worse, it turns out that Iterators over a
>>>>>> WriteBatch become invalidated not just when the WriteBatch is
>>> cleared,
>>>>> but
>>>>>> also when the Iterators' current key receives a new write.
>>>>>>
>>>>>> Now that I'm writing this, I remember that this is the major reason
>>> that
>>>>> I
>>>>>> switched the original design from having a query-time
>>> IsolationLevel to
>>>>>> having the IsolationLevel linked to the transactionality of the
>>> stores
>>>>>> themselves.
>>>>>>
>>>>>> It *might* be possible to resolve this, by having a "chain" of
>>>>>> WriteBatches, with the StreamThread switching to a new WriteBatch
>>>>> whenever
>>>>>> a new Interactive Query attempts to read from the database, but that
>>>>> could
>>>>>> cause some performance problems/memory pressure when subjected to a
>>> high
>>>>>> Interactive Query load. It would also reduce the efficiency of
>>>>> WriteBatches
>>>>>> on-commit, as we'd have to write N WriteBatches, where N is the
>>> number of
>>>>>> Interactive Queries since the last commit.
>>>>>>
>>>>>> I realise this is getting into the weeds of the implementation, and
>>> you'd
>>>>>> rather we focus on the API for now, but I think it's important to
>>>>> consider
>>>>>> how to implement the desired API, in case we come up with an API
>>> that
>>>>>> cannot be implemented efficiently, or even at all!
>>>>>>
>>>>>> Thoughts?
>>>>>> --
>>>>>> Nick
>>>>>>
>>>>>> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <ca...@apache.org>
>>> wrote:
>>>>>>
>>>>>>> Hi Nick,
>>>>>>>
>>>>>>> 6.
>>>>>>> Of course, you are right! My bad!
>>>>>>> Wiping out the state in the downgrading case is fine.
>>>>>>>
>>>>>>>
>>>>>>> 3a.
>>>>>>> Focus on the public facing changes for the KIP. We will manage 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.
>>>>>>>
>>>>>>>
>>>>>>> Best,
>>>>>>> Bruno
>>>>>>>
>>>>>>> On 9/13/23 11:57 AM, Nick Telford wrote:
>>>>>>>> Hi Bruno,
>>>>>>>>
>>>>>>>> Thanks for getting back to me!
>>>>>>>>
>>>>>>>> 2.
>>>>>>>> The fact that implementations can always track estimated memory
>>> usage
>>>>> in
>>>>>>>> the wrapper is a good point. I can remove -1 as an option, and
>>> I'll
>>>>>>> clarify
>>>>>>>> the JavaDoc that 0 is not just for non-transactional stores,
>>> which is
>>>>>>>> currently misleading.
>>>>>>>>
>>>>>>>> 6.
>>>>>>>> The problem with catching the exception in the downgrade process
>>> is
>>>>> that
>>>>>>>> would require new code in the Kafka version being downgraded to.
>>> Since
>>>>>>>> users could conceivably downgrade to almost *any* older version
>>> of
>>>>> Kafka
>>>>>>>> Streams, I'm not sure how we could add that code?
>>>>>>>> The only way I can think of doing it would be to provide a
>>> dedicated
>>>>>>>> downgrade tool, that goes through every local store and removes
>>> the
>>>>>>>> offsets column families. But that seems like an unnecessary
>>> amount of
>>>>>>> extra
>>>>>>>> code to maintain just to handle a somewhat niche situation, when
>>> the
>>>>>>>> alternative (automatically wipe and restore stores) should be
>>>>>>> acceptable.
>>>>>>>>
>>>>>>>> 1, 4, 5: Agreed. I'll make the changes you've requested.
>>>>>>>>
>>>>>>>> 3a.
>>>>>>>> I agree that IsolationLevel makes more sense at query-time, and I
>>>>>>> actually
>>>>>>>> initially attempted to place the IsolationLevel at query-time,
>>> but I
>>>>> ran
>>>>>>>> into some problems:
>>>>>>>> - The key issue is that, under ALOS we're not staging writes in
>>>>>>>> transactions, so can't perform writes at the READ_COMMITTED
>>> isolation
>>>>>>>> level. However, this may be addressed if we decide to *always*
>>> use
>>>>>>>> transactions as discussed under 3b.
>>>>>>>> - IQv1 and IQv2 have quite different implementations. I remember
>>>>> having
>>>>>>>> some difficulty understanding the IQv1 internals, which made it
>>>>>>> difficult
>>>>>>>> to determine what needed to be changed. However, I *think* this
>>> can be
>>>>>>>> addressed for both implementations by wrapping the RocksDBStore
>>> in an
>>>>>>>> IsolationLevel-dependent wrapper, that overrides read methods
>>> (get,
>>>>>>> etc.)
>>>>>>>> to either read directly from the database or from the ongoing
>>>>>>> transaction.
>>>>>>>> But IQv1 might still be difficult.
>>>>>>>> - If IsolationLevel becomes a query constraint, then all other
>>>>>>> StateStores
>>>>>>>> will need to respect it, including the in-memory stores. This
>>> would
>>>>>>> require
>>>>>>>> us to adapt in-memory stores to stage their writes so they can be
>>>>>>> isolated
>>>>>>>> from READ_COMMITTTED queries. It would also become an important
>>>>>>>> consideration for third-party stores on upgrade, as without
>>> changes,
>>>>>>> they
>>>>>>>> would not support READ_COMMITTED queries correctly.
>>>>>>>>
>>>>>>>> Ultimately, I may need some help making the necessary change to
>>> IQv1
>>>>> to
>>>>>>>> support this, but I don't think it's fundamentally impossible,
>>> if we
>>>>>>> want
>>>>>>>> to pursue this route.
>>>>>>>>
>>>>>>>> 3b.
>>>>>>>> The main reason I chose to keep ALOS un-transactional was to
>>> minimize
>>>>>>>> behavioural change for most users (I believe most Streams users
>>> use
>>>>> the
>>>>>>>> default configuration, which is ALOS). That said, it's clear
>>> that if
>>>>>>> ALOS
>>>>>>>> also used transactional stores, the only change in behaviour
>>> would be
>>>>>>> that
>>>>>>>> it would become *more correct*, which could be considered a "bug
>>> fix"
>>>>> by
>>>>>>>> users, rather than a change they need to handle.
>>>>>>>>
>>>>>>>> I believe that performance using transactions (aka. RocksDB
>>>>>>> WriteBatches)
>>>>>>>> should actually be *better* than the un-batched write-path that
>>> is
>>>>>>>> currently used[1]. The only "performance" consideration will be
>>> the
>>>>>>>> increased memory usage that transactions require. Given the
>>>>> mitigations
>>>>>>> for
>>>>>>>> this memory that we have in place, I would expect that this is
>>> not a
>>>>>>>> problem for most users.
>>>>>>>>
>>>>>>>> If we're happy to do so, we can make ALOS also use transactions.
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>> Nick
>>>>>>>>
>>>>>>>> Link 1:
>>>>>>>>
>>>>>
>>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>
>>>>>>>> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <cadonna@apache.org
>>>>
>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Nick,
>>>>>>>>>
>>>>>>>>> Thanks for the updates and sorry for the delay on my side!
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 1.
>>>>>>>>> Making the default implementation for flush() a no-op sounds
>>> good to
>>>>>>> me.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 2.
>>>>>>>>> I think what was bugging me here is that a third-party state
>>> store
>>>>>>> needs
>>>>>>>>> to implement the state store interface. That means they need to
>>>>>>>>> implement a wrapper around the actual state store as we do for
>>>>> RocksDB
>>>>>>>>> with RocksDBStore. So, a third-party state store can always
>>> estimate
>>>>>>> the
>>>>>>>>> uncommitted bytes, if it wants, because the wrapper can record
>>> the
>>>>>>> added
>>>>>>>>> bytes.
>>>>>>>>> One case I can think of where returning -1 makes sense is when
>>>>> Streams
>>>>>>>>> does not need to estimate the size of the write batch and
>>> trigger
>>>>>>>>> extraordinary commits, because the third-party state store
>>> takes care
>>>>>>> of
>>>>>>>>> memory. But in that case the method could also just return 0.
>>> Even
>>>>> that
>>>>>>>>> case would be better solved with a method that returns whether
>>> the
>>>>>>> state
>>>>>>>>> store manages itself the memory used for uncommitted bytes or
>>> not.
>>>>>>>>> Said that, I am fine with keeping the -1 return value, I was
>>> just
>>>>>>>>> wondering when and if it will be used.
>>>>>>>>>
>>>>>>>>> Regarding returning 0 for transactional state stores when the
>>> batch
>>>>> is
>>>>>>>>> empty, I was just wondering because you explicitly stated
>>>>>>>>>
>>>>>>>>> "or {@code 0} if this StateStore does not support transactions."
>>>>>>>>>
>>>>>>>>> So it seemed to me returning 0 could only happen for
>>>>> non-transactional
>>>>>>>>> state stores.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 3.
>>>>>>>>>
>>>>>>>>> a) What do you think if we move the isolation level to IQ (v1
>>> and
>>>>> v2)?
>>>>>>>>> In the end this is the only component that really needs to
>>> specify
>>>>> the
>>>>>>>>> isolation level. It is similar to the Kafka consumer that can
>>> choose
>>>>>>>>> with what isolation level to read the input topic.
>>>>>>>>> For IQv1 the isolation level should go into
>>> StoreQueryParameters. For
>>>>>>>>> IQv2, I would add it to the Query interface.
>>>>>>>>>
>>>>>>>>> b) Point a) raises the question what should happen during
>>>>> at-least-once
>>>>>>>>> processing when the state store does not use transactions? John
>>> in
>>>>> the
>>>>>>>>> past proposed to also use transactions on state stores for
>>>>>>>>> at-least-once. I like that idea, because it avoids aggregating
>>> the
>>>>> same
>>>>>>>>> records over and over again in the case of a failure. We had a
>>> case
>>>>> in
>>>>>>>>> the past where a Streams applications in at-least-once mode was
>>>>> failing
>>>>>>>>> continuously for some reasons I do not remember before
>>> committing the
>>>>>>>>> offsets. After each failover, the app aggregated again and
>>> again the
>>>>>>>>> same records. Of course the aggregate increased to very wrong
>>> values
>>>>>>>>> just because of the failover. With transactions on the state
>>> stores
>>>>> we
>>>>>>>>> could have avoided this. The app would have output the same
>>> aggregate
>>>>>>>>> multiple times (i.e., after each failover) but at least the
>>> value of
>>>>>>> the
>>>>>>>>> aggregate would not depend on the number of failovers.
>>> Outputting the
>>>>>>>>> same aggregate multiple times would be incorrect under
>>> exactly-once
>>>>> but
>>>>>>>>> it is OK for at-least-once.
>>>>>>>>> If it makes sense to add a config to turn on and off
>>> transactions on
>>>>>>>>> state stores under at-least-once or just use transactions in
>>> any case
>>>>>>> is
>>>>>>>>> a question we should also discuss in this KIP. It depends a bit
>>> on
>>>>> the
>>>>>>>>> performance trade-off. Maybe to be safe, I would add a config.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 4.
>>>>>>>>> Your points are all valid. I tend to say to keep the metrics
>>> around
>>>>>>>>> flush() until we remove flush() completely from the interface.
>>> Calls
>>>>> to
>>>>>>>>> flush() might still exist since existing processors might still
>>> call
>>>>>>>>> flush() explicitly as you mentioned in 1). For sure, we need to
>>>>>>> document
>>>>>>>>> how the metrics change due to the transactions in the upgrade
>>> notes.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 5.
>>>>>>>>> I see. Then you should describe how the .position files are
>>> handled
>>>>> in
>>>>>>>>> a dedicated section of the KIP or incorporate the description
>>> in the
>>>>>>>>> "Atomic Checkpointing" section instead of only mentioning it in
>>> the
>>>>>>>>> "Compatibility, 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:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>
>>> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
>>>>>>>>>
>>>>>>>>> It is a bit brittle, but it works.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Bruno
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 8/24/23 12:18 PM, Nick Telford wrote:
>>>>>>>>>> Hi Bruno,
>>>>>>>>>>
>>>>>>>>>> Thanks for taking the time to review the KIP. I'm back from
>>> leave
>>>>> now
>>>>>>> and
>>>>>>>>>> intend to move this forwards as quickly as I can.
>>>>>>>>>>
>>>>>>>>>> Addressing your points:
>>>>>>>>>>
>>>>>>>>>> 1.
>>>>>>>>>> Because flush() is part of the StateStore API, it's exposed to
>>>>> custom
>>>>>>>>>> Processors, which might be making calls to flush(). This was
>>>>> actually
>>>>>>> the
>>>>>>>>>> case in a few integration tests.
>>>>>>>>>> To maintain as much compatibility as possible, I'd prefer not
>>> to
>>>>> make
>>>>>>>>> this
>>>>>>>>>> an UnsupportedOperationException, as it will cause previously
>>>>> working
>>>>>>>>>> Processors to start throwing exceptions at runtime.
>>>>>>>>>> I agree that it doesn't make sense for it to proxy commit(),
>>> though,
>>>>>>> as
>>>>>>>>>> that would cause it to violate the "StateStores commit only
>>> when the
>>>>>>> Task
>>>>>>>>>> commits" rule.
>>>>>>>>>> Instead, I think we should make this a no-op. That way,
>>> existing
>>>>> user
>>>>>>>>>> Processors will continue to work as-before, without violation
>>> of
>>>>> store
>>>>>>>>>> consistency that would be caused by premature flush/commit of
>>>>>>> StateStore
>>>>>>>>>> data to disk.
>>>>>>>>>> What do you think?
>>>>>>>>>>
>>>>>>>>>> 2.
>>>>>>>>>> As stated in the JavaDoc, when a StateStore implementation is
>>>>>>>>>> transactional, but is unable to estimate the uncommitted memory
>>>>> usage,
>>>>>>>>> the
>>>>>>>>>> method will return -1.
>>>>>>>>>> The intention here is to permit third-party implementations
>>> that may
>>>>>>> not
>>>>>>>>> be
>>>>>>>>>> able to estimate memory usage.
>>>>>>>>>>
>>>>>>>>>> Yes, it will be 0 when nothing has been written to the store
>>> yet. I
>>>>>>>>> thought
>>>>>>>>>> that was implied by "This method will return an approximation
>>> of the
>>>>>>>>> memory
>>>>>>>>>> would be freed by the next call to {@link #commit(Map)}" and
>>>>> "@return
>>>>>>> The
>>>>>>>>>> approximate size of all records awaiting {@link #commit(Map)}",
>>>>>>> however,
>>>>>>>>> I
>>>>>>>>>> can add it explicitly to the JavaDoc if you think this is
>>> unclear?
>>>>>>>>>>
>>>>>>>>>> 3.
>>>>>>>>>> I realise this is probably the most contentious point in my
>>> design,
>>>>>>> and
>>>>>>>>> I'm
>>>>>>>>>> open to changing it if I'm unable to convince you of the
>>> benefits.
>>>>>>>>>> Nevertheless, here's my argument:
>>>>>>>>>> The Interactive Query (IQ) API(s) are directly provided
>>> StateStores
>>>>> to
>>>>>>>>>> query, and it may be important for users to programmatically
>>> know
>>>>>>> which
>>>>>>>>>> mode the StateStore is operating under. If we simply provide an
>>>>>>>>>> "eosEnabled" boolean (as used throughout the internal streams
>>>>>>> engine), or
>>>>>>>>>> similar, then users will need to understand the operation and
>>>>>>>>> consequences
>>>>>>>>>> of each available processing mode and how it pertains to their
>>>>>>>>> StateStore.
>>>>>>>>>>
>>>>>>>>>> Interactive Query users aren't the only people that care about
>>> the
>>>>>>>>>> processing.mode/IsolationLevel of a StateStore: implementers of
>>>>> custom
>>>>>>>>>> StateStores also need to understand the behaviour expected of
>>> their
>>>>>>>>>> implementation. KIP-892 introduces some assumptions into the
>>> Streams
>>>>>>>>> Engine
>>>>>>>>>> about how StateStores operate under each processing mode, and
>>> it's
>>>>>>>>>> important that custom implementations adhere to those
>>> assumptions in
>>>>>>>>> order
>>>>>>>>>> to maintain the consistency guarantees.
>>>>>>>>>>
>>>>>>>>>> IsolationLevels provide a high-level contract on the behaviour
>>> of
>>>>> the
>>>>>>>>>> StateStore: a user knows that under READ_COMMITTED, they will
>>> see
>>>>>>> writes
>>>>>>>>>> only after the Task has committed, and under READ_UNCOMMITTED
>>> they
>>>>>>> will
>>>>>>>>> see
>>>>>>>>>> writes immediately. No understanding of the details of each
>>>>>>>>> processing.mode
>>>>>>>>>> is required, either for IQ users or StateStore implementers.
>>>>>>>>>>
>>>>>>>>>> An argument can be made that these contractual guarantees can
>>> simply
>>>>>>> be
>>>>>>>>>> documented for the processing.mode (i.e. that exactly-once and
>>>>>>>>>> exactly-once-v2 behave like READ_COMMITTED and at-least-once
>>> behaves
>>>>>>> like
>>>>>>>>>> READ_UNCOMMITTED), but there are several small issues with
>>> this I'd
>>>>>>>>> prefer
>>>>>>>>>> to avoid:
>>>>>>>>>>
>>>>>>>>>>       - Where would we document these contracts, in a way that
>>> is
>>>>>>> difficult
>>>>>>>>>>       for users/implementers to miss/ignore?
>>>>>>>>>>       - It's not clear to users that the processing mode is
>>>>>>> communicating
>>>>>>>>>>       an expectation of read isolation, unless they read the
>>>>>>>>> documentation. Users
>>>>>>>>>>       rarely consult documentation unless they feel they need
>>> to, so
>>>>>>> it's
>>>>>>>>> likely
>>>>>>>>>>       this detail would get missed by many users.
>>>>>>>>>>       - It tightly couples processing modes to read isolation.
>>> Adding
>>>>>>> new
>>>>>>>>>>       processing modes, or changing the read isolation of
>>> existing
>>>>>>>>> processing
>>>>>>>>>>       modes would be difficult/impossible.
>>>>>>>>>>
>>>>>>>>>> Ultimately, the cost of introducing IsolationLevels is just a
>>> single
>>>>>>>>>> method, since we re-use the existing IsolationLevel enum from
>>> Kafka.
>>>>>>> This
>>>>>>>>>> gives us a clear place to document the contractual guarantees
>>>>> expected
>>>>>>>>>> of/provided by StateStores, that is accessible both by the
>>>>> StateStore
>>>>>>>>>> itself, and by IQ users.
>>>>>>>>>>
>>>>>>>>>> (Writing this I've just realised that the StateStore and IQ
>>> APIs
>>>>>>> actually
>>>>>>>>>> don't provide access to StateStoreContext that IQ users would
>>> have
>>>>>>> direct
>>>>>>>>>> access to... Perhaps StateStore should expose isolationLevel()
>>>>> itself
>>>>>>>>> too?)
>>>>>>>>>>
>>>>>>>>>> 4.
>>>>>>>>>> Yeah, I'm not comfortable renaming the metrics in-place
>>> either, as
>>>>>>> it's a
>>>>>>>>>> backwards incompatible change. My concern is that, if we leave
>>> the
>>>>>>>>> existing
>>>>>>>>>> "flush" metrics in place, they will be confusing to users.
>>> Right
>>>>> now,
>>>>>>>>>> "flush" metrics record explicit flushes to disk, but under
>>> KIP-892,
>>>>>>> even
>>>>>>>>> a
>>>>>>>>>> commit() will not explicitly flush data to disk - RocksDB will
>>>>> decide
>>>>>>> on
>>>>>>>>>> when to flush memtables to disk itself.
>>>>>>>>>>
>>>>>>>>>> If we keep the existing "flush" metrics, we'd have two options,
>>>>> which
>>>>>>>>> both
>>>>>>>>>> seem pretty bad to me:
>>>>>>>>>>
>>>>>>>>>>       1. Have them record calls to commit(), which would be
>>>>>>> misleading, as
>>>>>>>>>>       data is no longer explicitly "flushed" to disk by this
>>> call.
>>>>>>>>>>       2. Have them record nothing at all, which is equivalent to
>>>>>>> removing
>>>>>>>>> the
>>>>>>>>>>       metrics, except that users will see the metric still
>>> exists and
>>>>>>> so
>>>>>>>>> assume
>>>>>>>>>>       that the metric is correct, and that there's a problem
>>> with
>>>>> their
>>>>>>>>> system
>>>>>>>>>>       when there isn't.
>>>>>>>>>>
>>>>>>>>>> I agree that removing them is also a bad solution, and I'd
>>> like some
>>>>>>>>>> guidance on the best path forward here.
>>>>>>>>>>
>>>>>>>>>> 5.
>>>>>>>>>> Position files are updated on every write to a StateStore.
>>> Since our
>>>>>>>>> writes
>>>>>>>>>> are now buffered until commit(), we can't update the Position
>>> file
>>>>>>> until
>>>>>>>>>> commit() has been called, otherwise it would be inconsistent
>>> with
>>>>> the
>>>>>>>>> data
>>>>>>>>>> in the event of a rollback. Consequently, we need to manage
>>> these
>>>>>>> offsets
>>>>>>>>>> the same way we manage the checkpoint offsets, and ensure
>>> they're
>>>>> only
>>>>>>>>>> written on commit().
>>>>>>>>>>
>>>>>>>>>> 6.
>>>>>>>>>> Agreed, although I'm not exactly sure yet what tests to write.
>>> How
>>>>>>>>> explicit
>>>>>>>>>> do we need to be here in the KIP?
>>>>>>>>>>
>>>>>>>>>> As for upgrade/downgrade: upgrade is designed to be seamless,
>>> and we
>>>>>>>>> should
>>>>>>>>>> definitely add some tests around that. Downgrade, it
>>> transpires,
>>>>> isn't
>>>>>>>>>> currently possible, as the extra column family for offset
>>> storage is
>>>>>>>>>> incompatible with the pre-KIP-892 implementation: when you
>>> open a
>>>>>>> RocksDB
>>>>>>>>>> database, you must open all available column families or
>>> receive an
>>>>>>>>> error.
>>>>>>>>>> What currently happens on downgrade is that it attempts to
>>> open the
>>>>>>>>> store,
>>>>>>>>>> throws an error about the offsets column family not being
>>> opened,
>>>>>>> which
>>>>>>>>>> triggers a wipe and rebuild of the Task. Given that downgrades
>>>>> should
>>>>>>> be
>>>>>>>>>> uncommon, I think this is acceptable behaviour, as the
>>> end-state is
>>>>>>>>>> consistent, even if it results in an undesirable state restore.
>>>>>>>>>>
>>>>>>>>>> Should I document the upgrade/downgrade behaviour explicitly
>>> in the
>>>>>>> KIP?
>>>>>>>>>>
>>>>>>>>>> --
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>> Nick
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
>>> cadonna@apache.org>
>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Nick!
>>>>>>>>>>>
>>>>>>>>>>> Thanks for the updates!
>>>>>>>>>>>
>>>>>>>>>>> 1.
>>>>>>>>>>> Why does StateStore#flush() default to
>>>>>>>>>>> StateStore#commit(Collections.emptyMap())?
>>>>>>>>>>> Since calls to flush() will not exist anymore after this KIP
>>> is
>>>>>>>>>>> released, I would rather throw an unsupported operation
>>> exception
>>>>> by
>>>>>>>>>>> default.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 2.
>>>>>>>>>>> When would a state store return -1 from
>>>>>>>>>>> StateStore#approximateNumUncommittedBytes() while being
>>>>>>> transactional?
>>>>>>>>>>>
>>>>>>>>>>> Wouldn't StateStore#approximateNumUncommittedBytes() also
>>> return 0
>>>>> if
>>>>>>>>>>> the state store is transactional but nothing has been written
>>> to
>>>>> the
>>>>>>>>>>> state store yet?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 3.
>>>>>>>>>>> Sorry for bringing this up again. Does this KIP really need to
>>>>>>> introduce
>>>>>>>>>>> StateStoreContext#isolationLevel()? StateStoreContext has
>>> already
>>>>>>>>>>> appConfigs() which basically exposes the same information,
>>> i.e., if
>>>>>>> EOS
>>>>>>>>>>> is enabled or not.
>>>>>>>>>>> In one of your previous e-mails you wrote:
>>>>>>>>>>>
>>>>>>>>>>> "My idea was to try to keep the StateStore interface as
>>> loosely
>>>>>>> coupled
>>>>>>>>>>> from the Streams engine as possible, to give implementers more
>>>>>>> freedom,
>>>>>>>>>>> and reduce the amount of internal knowledge required."
>>>>>>>>>>>
>>>>>>>>>>> While I understand the intent, I doubt that it decreases the
>>>>>>> coupling of
>>>>>>>>>>> a StateStore interface and the Streams engine. READ_COMMITTED
>>> only
>>>>>>>>>>> applies to IQ but not to reads by processors. Thus,
>>> implementers
>>>>>>> need to
>>>>>>>>>>> understand how Streams accesses the state stores.
>>>>>>>>>>>
>>>>>>>>>>> I would like to hear what others think about this.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 4.
>>>>>>>>>>> Great exposing new metrics for transactional state stores!
>>>>> However, I
>>>>>>>>>>> would prefer to add new metrics and deprecate (in the docs)
>>> the 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 also need to adapt/add integration tests
>>> besides
>>>>> unit
>>>>>>>>>>> tests. Additionally, we probably need integration or system
>>> tests
>>>>> to
>>>>>>>>>>> verify that upgrades and downgrades between transactional and
>>>>>>>>>>> non-transactional state stores work as expected.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> Bruno
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
>>>>>>>>>>>> One more thing: I noted John's suggestion in the KIP, under
>>>>>>> "Rejected
>>>>>>>>>>>> Alternatives". I still think it's an idea worth pursuing,
>>> but I
>>>>>>> believe
>>>>>>>>>>>> that it's out of the scope of this KIP, because it solves a
>>>>>>> different
>>>>>>>>> set
>>>>>>>>>>>> of problems to this KIP, and the scope of this one has
>>> already
>>>>> grown
>>>>>>>>>>> quite
>>>>>>>>>>>> large!
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
>>>>> nick.telford@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>
>>>>>>>>>>>>> I've updated the KIP (
>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>> )
>>>>>>>>>>>>> with the latest changes; mostly bringing back "Atomic
>>>>>>> Checkpointing"
>>>>>>>>>>> (for
>>>>>>>>>>>>> what feels like the 10th time!). I think the one thing
>>> missing is
>>>>>>> some
>>>>>>>>>>>>> changes to metrics (notably the store "flush" metrics will
>>> need
>>>>> to
>>>>>>> be
>>>>>>>>>>>>> renamed to "commit").
>>>>>>>>>>>>>
>>>>>>>>>>>>> The reason I brought back Atomic Checkpointing was to
>>> decouple
>>>>>>> store
>>>>>>>>>>> flush
>>>>>>>>>>>>> from store commit. This is important, because with
>>> Transactional
>>>>>>>>>>>>> StateStores, we now need to call "flush" on *every* Task
>>> commit,
>>>>>>> and
>>>>>>>>> not
>>>>>>>>>>>>> just when the StateStore is closing, otherwise our
>>> transaction
>>>>>>> buffer
>>>>>>>>>>> will
>>>>>>>>>>>>> never be written and persisted, instead growing unbounded! I
>>>>>>>>>>> experimented
>>>>>>>>>>>>> with some simple solutions, like forcing a store flush
>>> whenever
>>>>> the
>>>>>>>>>>>>> transaction buffer was likely to exceed its configured
>>> size, but
>>>>>>> this
>>>>>>>>>>> was
>>>>>>>>>>>>> brittle: it prevented the transaction buffer from being
>>>>> configured
>>>>>>> to
>>>>>>>>> be
>>>>>>>>>>>>> unbounded, and it still would have required explicit
>>> flushes of
>>>>>>>>> RocksDB,
>>>>>>>>>>>>> yielding sub-optimal performance and memory utilization.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I deemed Atomic Checkpointing to be the "right" way to
>>> resolve
>>>>> this
>>>>>>>>>>>>> problem. By ensuring that the changelog offsets that
>>> correspond
>>>>> to
>>>>>>> the
>>>>>>>>>>> most
>>>>>>>>>>>>> recently written records are always atomically written to
>>> the
>>>>>>>>> StateStore
>>>>>>>>>>>>> (by writing them to the same transaction buffer), we can
>>> avoid
>>>>>>>>> forcibly
>>>>>>>>>>>>> flushing the RocksDB memtables to disk, letting RocksDB
>>> flush
>>>>> them
>>>>>>>>> only
>>>>>>>>>>>>> when necessary, without losing any of our consistency
>>> guarantees.
>>>>>>> See
>>>>>>>>>>> the
>>>>>>>>>>>>> updated KIP for more info.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I have fully implemented these changes, although I'm still
>>> not
>>>>>>>>> entirely
>>>>>>>>>>>>> happy with the implementation for segmented StateStores, so
>>> I
>>>>> plan
>>>>>>> to
>>>>>>>>>>>>> refactor that. Despite that, all tests pass. If you'd like
>>> to try
>>>>>>> out
>>>>>>>>> or
>>>>>>>>>>>>> review this highly experimental and incomplete branch, it's
>>>>>>> available
>>>>>>>>>>> here:
>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0.
>>> Note:
>>>>>>> it's
>>>>>>>>>>> built
>>>>>>>>>>>>> against Kafka 3.5.0 so that I had a stable base to build
>>> and test
>>>>>>> it
>>>>>>>>> on,
>>>>>>>>>>>>> and to enable easy apples-to-apples comparisons in a live
>>>>>>>>> environment. I
>>>>>>>>>>>>> plan to rebase it against trunk once it's nearer completion
>>> and
>>>>> has
>>>>>>>>> been
>>>>>>>>>>>>> proven on our main application.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I would really appreciate help in reviewing and testing:
>>>>>>>>>>>>> - Segmented (Versioned, Session and Window) stores
>>>>>>>>>>>>> - Global stores
>>>>>>>>>>>>>
>>>>>>>>>>>>> As I do not currently use either of these, so my primary
>>> test
>>>>>>>>>>> environment
>>>>>>>>>>>>> doesn't test these areas.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I'm going on Parental Leave starting next week for a few
>>> weeks,
>>>>> so
>>>>>>>>> will
>>>>>>>>>>>>> not have time to move this forward until late August. That
>>> said,
>>>>>>> your
>>>>>>>>>>>>> feedback is welcome and appreciated, I just won't be able to
>>>>>>> respond
>>>>>>>>> as
>>>>>>>>>>>>> quickly as usual.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
>>>>> nick.telford@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Bruno
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Yes, that's correct, although the impact on IQ is not
>>> something
>>>>> I
>>>>>>> had
>>>>>>>>>>>>>> considered.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> What about atomically updating the state store from the
>>>>>>> transaction
>>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
>>> (thus,
>>>>>>>>>>> flushing
>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
>>> number of
>>>>>>>>> commit
>>>>>>>>>>>>>>> intervals?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'm not quite sure I follow. Are you suggesting that we
>>> add an
>>>>>>>>>>> additional
>>>>>>>>>>>>>> config for the max number of commit intervals between
>>>>> checkpoints?
>>>>>>>>> That
>>>>>>>>>>>>>> way, we would checkpoint *either* when the transaction
>>> buffers
>>>>> are
>>>>>>>>>>> nearly
>>>>>>>>>>>>>> full, *OR* whenever a certain number of commit intervals
>>> have
>>>>>>>>> elapsed,
>>>>>>>>>>>>>> whichever comes first?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> That certainly seems reasonable, although this re-ignites
>>> an
>>>>>>> earlier
>>>>>>>>>>>>>> debate about whether a config should be measured in
>>> "number of
>>>>>>> commit
>>>>>>>>>>>>>> intervals", instead of just an absolute time.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> FWIW, I realised that this issue is the reason I was
>>> pursuing
>>>>> the
>>>>>>>>>>> Atomic
>>>>>>>>>>>>>> Checkpoints, as it de-couples memtable flush from
>>> checkpointing,
>>>>>>>>> which
>>>>>>>>>>>>>> enables us to just checkpoint on every commit without any
>>>>>>> performance
>>>>>>>>>>>>>> impact. Atomic Checkpointing is definitely the "best"
>>> solution,
>>>>>>> but
>>>>>>>>>>> I'm not
>>>>>>>>>>>>>> sure if this is enough to bring it back into this KIP.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'm currently working on moving all the transactional logic
>>>>>>> directly
>>>>>>>>>>> into
>>>>>>>>>>>>>> RocksDBStore itself, which does away with the
>>>>>>>>> StateStore#newTransaction
>>>>>>>>>>>>>> method, and reduces the number of new classes introduced,
>>>>>>>>> significantly
>>>>>>>>>>>>>> reducing the complexity. If it works, and the complexity is
>>>>>>>>> drastically
>>>>>>>>>>>>>> reduced, I may try bringing back Atomic Checkpoints into
>>> this
>>>>> KIP.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
>>> cadonna@apache.org>
>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks for the insights! Very interesting!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> As far as I understand, you want to atomically update the
>>> state
>>>>>>>>> store
>>>>>>>>>>>>>>> from the transaction buffer, flush the memtable of a state
>>>>> store
>>>>>>> and
>>>>>>>>>>>>>>> write the checkpoint not after the commit time elapsed but
>>>>> after
>>>>>>> the
>>>>>>>>>>>>>>> transaction buffer reached a size that would lead to
>>> exceeding
>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes before the next
>>> commit
>>>>>>>>>>> interval
>>>>>>>>>>>>>>> ends.
>>>>>>>>>>>>>>> That means, the Kafka transaction would commit every
>>> commit
>>>>>>> interval
>>>>>>>>>>> but
>>>>>>>>>>>>>>> the state store will only be atomically updated roughly
>>> every
>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ
>>> would
>>>>>>> then
>>>>>>>>>>> only
>>>>>>>>>>>>>>> see new data roughly every
>>>>>>> statestore.transaction.buffer.max.bytes.
>>>>>>>>>>>>>>> After a failure the state store needs to restore up to
>>>>>>>>>>>>>>> statestore.transaction.buffer.max.bytes.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Is this correct?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> What about atomically updating the state store from the
>>>>>>> transaction
>>>>>>>>>>>>>>> buffer every commit interval and writing the checkpoint
>>> (thus,
>>>>>>>>>>> flushing
>>>>>>>>>>>>>>> the memtable) every configured amount of data and/or
>>> number of
>>>>>>>>> commit
>>>>>>>>>>>>>>> intervals? In such a way, we would have the same delay for
>>>>>>> records
>>>>>>>>>>>>>>> appearing in output topics and IQ because both would
>>> appear
>>>>> when
>>>>>>> the
>>>>>>>>>>>>>>> Kafka transaction is committed. However, after a failure
>>> the
>>>>>>> state
>>>>>>>>>>> store
>>>>>>>>>>>>>>> still needs to restore up to
>>>>>>> statestore.transaction.buffer.max.bytes
>>>>>>>>>>> and
>>>>>>>>>>>>>>> it might restore data that is already in the state store
>>>>> because
>>>>>>> the
>>>>>>>>>>>>>>> checkpoint lags behind the last stable offset (i.e. the
>>> last
>>>>>>>>> committed
>>>>>>>>>>>>>>> offset) of the changelog topics. Restoring data that is
>>> already
>>>>>>> in
>>>>>>>>> the
>>>>>>>>>>>>>>> state store is idempotent, so eos should not violated.
>>>>>>>>>>>>>>> This solution needs at least one new config to specify
>>> when a
>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>> should be written.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> A small correction to your previous e-mail that does not
>>> change
>>>>>>>>>>> anything
>>>>>>>>>>>>>>> you said: Under alos the default commit interval is 30
>>> seconds,
>>>>>>> not
>>>>>>>>>>> five
>>>>>>>>>>>>>>> seconds.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I've begun performance testing my branch on our staging
>>>>>>>>> environment,
>>>>>>>>>>>>>>>> putting it through its paces in our non-trivial
>>> application.
>>>>> I'm
>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>> observing the same increased flush rate that we saw the
>>> last
>>>>>>> time
>>>>>>>>> we
>>>>>>>>>>>>>>>> attempted to use a version of this KIP, but this time, I
>>>>> think I
>>>>>>>>> know
>>>>>>>>>>>>>>> why.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at
>>> the end
>>>>>>> of
>>>>>>>>> the
>>>>>>>>>>>>>>> Task
>>>>>>>>>>>>>>>> commit process, has the following behaviour:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         - Under ALOS: checkpoint the state stores. This
>>>>> includes
>>>>>>>>>>>>>>>>         flushing memtables in RocksDB. This is acceptable
>>>>>>> because the
>>>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>         commit.interval.ms is 5 seconds, so forcibly
>>> flushing
>>>>>>>>> memtables
>>>>>>>>>>>>>>> every 5
>>>>>>>>>>>>>>>>         seconds is acceptable for most applications.
>>>>>>>>>>>>>>>>         - Under EOS: checkpointing is not done, *unless*
>>> it's
>>>>>>> being
>>>>>>>>>>>>>>> forced, due
>>>>>>>>>>>>>>>>         to e.g. the Task closing or being revoked. This
>>> means
>>>>>>> that
>>>>>>>>> under
>>>>>>>>>>>>>>> normal
>>>>>>>>>>>>>>>>         processing conditions, the state stores will not
>>> be
>>>>>>>>>>> checkpointed,
>>>>>>>>>>>>>>> and will
>>>>>>>>>>>>>>>>         not have memtables flushed at all , unless RocksDB
>>>>>>> decides to
>>>>>>>>>>>>>>> flush them on
>>>>>>>>>>>>>>>>         its own. Checkpointing stores and force-flushing
>>> their
>>>>>>>>> memtables
>>>>>>>>>>>>>>> is only
>>>>>>>>>>>>>>>>         done when a Task is being closed.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least
>>>>> *some*
>>>>>>>>>>> normal
>>>>>>>>>>>>>>>> Task commits, in order to write the RocksDB transaction
>>>>> buffers
>>>>>>> to
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> state stores, and to ensure the offsets are synced to
>>> disk to
>>>>>>>>> prevent
>>>>>>>>>>>>>>>> restores from getting out of hand. Consequently, my
>>> current
>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is
>>> far too
>>>>>>>>>>>>>>> frequent.
>>>>>>>>>>>>>>>> This causes checkpoints every 10,000 records, which is a
>>>>> change
>>>>>>> in
>>>>>>>>>>>>>>> flush
>>>>>>>>>>>>>>>> behaviour, potentially causing performance problems for
>>> some
>>>>>>>>>>>>>>> applications.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I'm looking into possible solutions, and I'm currently
>>> leaning
>>>>>>>>>>> towards
>>>>>>>>>>>>>>>> using the statestore.transaction.buffer.max.bytes
>>>>> configuration
>>>>>>> to
>>>>>>>>>>>>>>>> checkpoint Tasks once we are likely to exceed it. This
>>> would
>>>>>>>>>>>>>>> complement the
>>>>>>>>>>>>>>>> existing "early Task commit" functionality that this
>>>>>>> configuration
>>>>>>>>>>>>>>>> provides, in the following way:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         - Currently, we use
>>>>>>> statestore.transaction.buffer.max.bytes
>>>>>>>>> to
>>>>>>>>>>>>>>> force an
>>>>>>>>>>>>>>>>         early Task commit if processing more records would
>>>>> cause
>>>>>>> our
>>>>>>>>>>> state
>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>         transactions to exceed the memory assigned to
>>> them.
>>>>>>>>>>>>>>>>         - New functionality: when a Task *does* commit,
>>> we will
>>>>>>> not
>>>>>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>>>         the stores (and hence flush the transaction
>>> buffers)
>>>>>>> unless
>>>>>>>>> we
>>>>>>>>>>>>>>> expect to
>>>>>>>>>>>>>>>>         cross the statestore.transaction.buffer.max.bytes
>>>>>>> threshold
>>>>>>>>>>> before
>>>>>>>>>>>>>>> the next
>>>>>>>>>>>>>>>>         commit
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I'm also open to suggestions.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
>>>>>>> nick.telford@gmail.com
>>>>>>>>>>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Bruno!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>> By "less predictable for users", I meant in terms of
>>>>>>> understanding
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> performance profile under various circumstances. The
>>> more
>>>>>>> complex
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> solution, the more difficult it would be for users to
>>>>>>> understand
>>>>>>>>> the
>>>>>>>>>>>>>>>>> performance they see. For example, spilling records to
>>> disk
>>>>>>> when
>>>>>>>>> the
>>>>>>>>>>>>>>>>> transaction buffer reaches a threshold would, I expect,
>>>>> reduce
>>>>>>>>> write
>>>>>>>>>>>>>>>>> throughput. This reduction in write throughput could be
>>>>>>>>> unexpected,
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> potentially difficult to diagnose/understand for users.
>>>>>>>>>>>>>>>>> At the moment, I think the "early commit" concept is
>>>>> relatively
>>>>>>>>>>>>>>>>> straightforward; it's easy to document, and conceptually
>>>>> fairly
>>>>>>>>>>>>>>> obvious to
>>>>>>>>>>>>>>>>> users. We could probably add a metric to make it easier
>>> to
>>>>>>>>>>> understand
>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>> it happens though.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 3. (the second one)
>>>>>>>>>>>>>>>>> The IsolationLevel is *essentially* an indirect way of
>>>>> telling
>>>>>>>>>>>>>>> StateStores
>>>>>>>>>>>>>>>>> whether they should be transactional. READ_COMMITTED
>>>>>>> essentially
>>>>>>>>>>>>>>> requires
>>>>>>>>>>>>>>>>> transactions, because it dictates that two threads
>>> calling
>>>>>>>>>>>>>>>>> `newTransaction()` should not see writes from the other
>>>>>>>>> transaction
>>>>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all
>>> bets are
>>>>>>> off,
>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> stores can allow threads to observe written records at
>>> any
>>>>>>> time,
>>>>>>>>>>>>>>> which is
>>>>>>>>>>>>>>>>> essentially "no transactions". That said, StateStores
>>> are
>>>>> free
>>>>>>> to
>>>>>>>>>>>>>>> implement
>>>>>>>>>>>>>>>>> these guarantees however they can, which is a bit more
>>>>> relaxed
>>>>>>>>> than
>>>>>>>>>>>>>>>>> dictating "you must use transactions". For example, with
>>>>>>> RocksDB
>>>>>>>>> we
>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
>>>>> "transactions",
>>>>>>>>>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But
>>> with
>>>>>>> other
>>>>>>>>>>>>>>> storage
>>>>>>>>>>>>>>>>> engines, it might be preferable to *always* use
>>> transactions,
>>>>>>> even
>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>> unnecessary; or there may be storage engines that don't
>>>>> provide
>>>>>>>>>>>>>>>>> transactions, but the isolation guarantees can be met
>>> using a
>>>>>>>>>>>>>>> different
>>>>>>>>>>>>>>>>> technique.
>>>>>>>>>>>>>>>>> My idea was to try to keep the StateStore interface as
>>>>> loosely
>>>>>>>>>>> coupled
>>>>>>>>>>>>>>>>> from the Streams engine as possible, to give
>>> implementers
>>>>> more
>>>>>>>>>>>>>>> freedom, and
>>>>>>>>>>>>>>>>> reduce the amount of internal knowledge required.
>>>>>>>>>>>>>>>>> That said, I understand that "IsolationLevel" might not
>>> be
>>>>> the
>>>>>>>>> right
>>>>>>>>>>>>>>>>> abstraction, and we can always make it much more
>>> explicit if
>>>>>>>>>>>>>>> required, e.g.
>>>>>>>>>>>>>>>>> boolean transactional()
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 7-8.
>>>>>>>>>>>>>>>>> I can make these changes either later today or tomorrow.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Small update:
>>>>>>>>>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of
>>> issues
>>>>>>> that
>>>>>>>>>>>>>>> needed
>>>>>>>>>>>>>>>>> addressing. Currently, all the tests pass, which is
>>>>> promising,
>>>>>>> but
>>>>>>>>>>> it
>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>> need to undergo some performance testing. I haven't
>>> (yet)
>>>>>>> worked
>>>>>>>>> on
>>>>>>>>>>>>>>>>> removing the `newTransaction()` stuff, but I would
>>> expect
>>>>> that,
>>>>>>>>>>>>>>>>> behaviourally, it should make no difference. The branch
>>> is
>>>>>>>>> available
>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if
>>>>> anyone
>>>>>>> is
>>>>>>>>>>>>>>>>> interested in taking an early look.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
>>>>>>> cadonna@apache.org>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>> Yeah, I agree with you. That was actually also my
>>> point. I
>>>>>>>>>>> understood
>>>>>>>>>>>>>>>>>> that John was proposing the ingestion path as a way to
>>> avoid
>>>>>>> the
>>>>>>>>>>>>>>> early
>>>>>>>>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>> I agree with John here, that actually it is public
>>> API. My
>>>>>>>>> question
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> how this usage pattern affects normal processing.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>> My concern is that checking for the size of the
>>> transaction
>>>>>>>>> buffer
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> maybe triggering an early commit affects the whole
>>>>> processing
>>>>>>> of
>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>> Streams. The transactionality of a state store is not
>>>>>>> confined to
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> state store itself, but spills over and changes the
>>> behavior
>>>>>>> of
>>>>>>>>>>> other
>>>>>>>>>>>>>>>>>> parts of the system. I agree with you that it is a
>>> decent
>>>>>>>>>>>>>>> compromise. I
>>>>>>>>>>>>>>>>>> just wanted to analyse the downsides and list the
>>> options to
>>>>>>>>>>> overcome
>>>>>>>>>>>>>>>>>> them. I also agree with you that all options seem quite
>>>>> heavy
>>>>>>>>>>>>>>> compared
>>>>>>>>>>>>>>>>>> with your KIP. I do not understand what you mean with
>>> "less
>>>>>>>>>>>>>>> predictable
>>>>>>>>>>>>>>>>>> for users", though.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I found the discussions about the alternatives really
>>>>>>>>> interesting.
>>>>>>>>>>>>>>> But I
>>>>>>>>>>>>>>>>>> also think that your plan sounds good and we should
>>> continue
>>>>>>> with
>>>>>>>>>>> it!
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>> Ah, now, I understand the reasoning behind putting
>>> isolation
>>>>>>>>> level
>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> the state store context. Thanks! Should that also be a
>>> way
>>>>> to
>>>>>>>>> give
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> the state store the opportunity to decide whether to
>>> turn on
>>>>>>>>>>>>>>>>>> transactions or not?
>>>>>>>>>>>>>>>>>> With my comment, I was more concerned about how do you
>>> know
>>>>>>> if a
>>>>>>>>>>>>>>>>>> checkpoint file needs to be written under EOS, if you
>>> do not
>>>>>>>>> have a
>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>> to know if the state store is transactional or not. If
>>> a
>>>>> state
>>>>>>>>>>> store
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> transactional, the checkpoint file can be written
>>> during
>>>>>>> normal
>>>>>>>>>>>>>>>>>> processing under EOS. If the state store is not
>>>>> transactional,
>>>>>>>>> the
>>>>>>>>>>>>>>>>>> checkpoint file must not be written under EOS.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 7.
>>>>>>>>>>>>>>>>>> My point was about not only considering the bytes in
>>> memory
>>>>> in
>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that
>>> might
>>>>> be
>>>>>>>>>>>>>>> spilled
>>>>>>>>>>>>>>>>>> on disk. Basically, I was wondering whether you should
>>>>> remove
>>>>>>> the
>>>>>>>>>>>>>>>>>> "memory" in "Maximum number of memory bytes to be used
>>> to
>>>>>>>>>>>>>>>>>> buffer uncommitted state-store records." My thinking
>>> was
>>>>> that
>>>>>>>>> even
>>>>>>>>>>>>>>> if a
>>>>>>>>>>>>>>>>>> state store spills uncommitted bytes to disk, limiting
>>> the
>>>>>>>>> overall
>>>>>>>>>>>>>>> bytes
>>>>>>>>>>>>>>>>>> might make sense. Thinking about it again and
>>> considering
>>>>> the
>>>>>>>>>>> recent
>>>>>>>>>>>>>>>>>> discussions, it does not make too much sense anymore.
>>>>>>>>>>>>>>>>>> I like the name
>>> statestore.transaction.buffer.max.bytes that
>>>>>>> you
>>>>>>>>>>>>>>> proposed.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 8.
>>>>>>>>>>>>>>>>>> A high-level description (without implementation
>>> details) of
>>>>>>> how
>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>> Streams will manage the commit of changelog
>>> transactions,
>>>>>>> state
>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>> transactions and checkpointing would be great. Would be
>>>>> great
>>>>>>> if
>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>> could also add some sentences about the behavior in
>>> case of
>>>>> a
>>>>>>>>>>>>>>> failure.
>>>>>>>>>>>>>>>>>> For instance how does a transactional state store
>>> recover
>>>>>>> after a
>>>>>>>>>>>>>>>>>> failure or what happens with the transaction buffer,
>>> etc.
>>>>>>> (that
>>>>>>>>> is
>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>> I meant by "fail-over" in point 9.)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
>>>>>>>>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>>>>>>>>>>> Isn't this exactly the same issue that
>>> WriteBatchWithIndex
>>>>>>>>>>>>>>> transactions
>>>>>>>>>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
>>> configured
>>>>>>> memory
>>>>>>>>>>>>>>> needs to
>>>>>>>>>>>>>>>>>>> trigger an early commit?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 2.
>>>>>>>>>>>>>>>>>>> This is one of my big concerns. Ultimately, any
>>> approach
>>>>>>> based
>>>>>>>>> on
>>>>>>>>>>>>>>>>>> cracking
>>>>>>>>>>>>>>>>>>> open RocksDB internals and using it in ways it's not
>>> really
>>>>>>>>>>> designed
>>>>>>>>>>>>>>>>>> for is
>>>>>>>>>>>>>>>>>>> likely to have some unforseen performance or
>>> consistency
>>>>>>> issues.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 3.
>>>>>>>>>>>>>>>>>>> What's your motivation for removing these early
>>> commits?
>>>>>>> While
>>>>>>>>> not
>>>>>>>>>>>>>>>>>> ideal, I
>>>>>>>>>>>>>>>>>>> think they're a decent compromise to ensure
>>> consistency
>>>>>>> whilst
>>>>>>>>>>>>>>>>>> maintaining
>>>>>>>>>>>>>>>>>>> good and predictable performance.
>>>>>>>>>>>>>>>>>>> All 3 of your suggested ideas seem *very*
>>> complicated, and
>>>>>>> might
>>>>>>>>>>>>>>>>>> actually
>>>>>>>>>>>>>>>>>>> make behaviour less predictable for users as a
>>> consequence.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is
>>> growing a
>>>>>>> bit
>>>>>>>>>>> out
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>> control. While it's good to discuss ideas for future
>>>>>>>>>>> improvements, I
>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>> it's important to narrow the scope down to a design
>>> that
>>>>>>>>> achieves
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> most
>>>>>>>>>>>>>>>>>>> pressing objectives (constant sized restorations
>>> during
>>>>> dirty
>>>>>>>>>>>>>>>>>>> close/unexpected errors). Any design that this KIP
>>> produces
>>>>>>> can
>>>>>>>>>>>>>>>>>> ultimately
>>>>>>>>>>>>>>>>>>> be changed in the future, especially if the bulk of
>>> it is
>>>>>>>>> internal
>>>>>>>>>>>>>>>>>>> behaviour.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I'm going to spend some time next week trying to
>>> re-work
>>>>> the
>>>>>>>>>>>>>>> original
>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex design to remove the
>>> newTransaction()
>>>>>>>>> method,
>>>>>>>>>>>>>>> such
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> it's just an implementation detail of RocksDBStore.
>>> That
>>>>>>> way, if
>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> want to
>>>>>>>>>>>>>>>>>>> replace WBWI with something in the future, like the
>>> SST
>>>>> file
>>>>>>>>>>>>>>> management
>>>>>>>>>>>>>>>>>>> outlined by John, then we can do so with little/no API
>>>>>>> changes.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>
>>
> 

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

Posted by Nick Telford <ni...@gmail.com>.
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 <ni...@gmail.com> wrote:

> Hi everyone,
>
> I agree that having IsolationLevel be determined at query-time is the
> ideal design, but there are a few sticking points:
>
> 1.
> There needs to be some way to communicate the IsolationLevel down to the
> RocksDBStore itself, so that the query can respect it. Since stores are
> "layered" in functionality (i.e. ChangeLoggingStore, MeteredStore, etc.),
> we need some way to deliver that information to the bottom layer. For IQv2,
> we can use the existing State#query() method, but IQv1 has no way to do
> this.
>
> A simple approach, which would potentially open up other options, would be
> to add something like: ReadOnlyKeyValueStore<K, V>
> readOnlyView(IsolationLevel isolationLevel) to ReadOnlyKeyValueStore (and
> similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).
>
> 2.
> As mentioned above, RocksDB WriteBatches are not thread-safe, which causes
> a problem if we want to provide READ_UNCOMMITTED Iterators. I also had a
> look at RocksDB Transactions[1], but they solve a very different problem,
> and have the same thread-safety issue.
>
> One possible approach that I mentioned is chaining WriteBatches: every
> time a new Interactive Query is received (i.e. readOnlyView, see above,
> is called) we "freeze" the existing WriteBatch, and start a new one for new
> writes. The Interactive Query queries the "chain" of previous WriteBatches
> + the underlying database; while the StreamThread starts writing to the
> *new* WriteBatch. On-commit, the StreamThread would write *all*
> WriteBatches in the chain to the database (that have not yet been written).
>
> WriteBatches would be closed/freed only when they have been both
> committed, and all open Interactive Queries on them have been closed. This
> would require some reference counting.
>
> Obviously a drawback of this approach is the potential for increased
> memory usage: if an Interactive Query is long-lived, for example by doing a
> full scan over a large database, or even just pausing in the middle of an
> iteration, then the existing chain of WriteBatches could be kept around for
> a long time, potentially forever.
>
> --
>
> A.
> Going off on a tangent, it looks like in addition to supporting
> READ_COMMITTED queries, we could go further and support REPEATABLE_READ
> queries (i.e. where subsequent reads to the same key in the same
> Interactive Query are guaranteed to yield the same value) by making use of
> RocksDB Snapshots[2]. These are fairly lightweight, so the performance
> impact is likely to be negligible, but they do require that the Interactive
> Query session can be explicitly closed.
>
> This could be achieved if we made the above readOnlyView interface look
> more like:
>
> interface ReadOnlyKeyValueView<K, V> implements ReadOnlyKeyValueStore<K,
> V>, AutoCloseable {}
>
> interface ReadOnlyKeyValueStore<K, V> {
>     ...
>     ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel isolationLevel);
> }
>
> But this would be a breaking change, as existing IQv1 queries are
> guaranteed to never call store.close(), and therefore these would leak
> memory under REPEATABLE_READ.
>
> B.
> One thing that's notable: MyRocks states that they support READ_COMMITTED
> and REPEATABLE_READ, but they make no mention of READ_UNCOMMITTED[3][4].
> This could be because doing so is technically difficult/impossible using
> the primitives available in RocksDB.
>
> --
>
> Lucas, to address your points:
>
> U1.
> It's only "SHOULD" to permit alternative (i.e. non-RocksDB)
> implementations of StateStore that do not support atomic writes. Obviously
> in those cases, the guarantees Kafka Streams provides/expects would be
> relaxed. Do you think we should require all implementations to support
> atomic writes?
>
> U2.
> Stores can support multiple IsolationLevels. As we've discussed above, the
> ideal scenario would be to specify the IsolationLevel at query-time.
> Failing that, I think the second-best approach is to define the
> IsolationLevel for *all* queries based on the processing.mode, which is
> what the default StateStoreContext#isolationLevel() achieves. Would you
> prefer an alternative?
>
> While the existing implementation is equivalent to READ_UNCOMMITTED, this
> can yield unexpected results/errors under EOS, if a transaction is rolled
> back. While this would be a change in behaviour for users, it would look
> more like a bug fix than a breaking change. That said, we *could* make it
> configurable, and default to the existing behaviour (READ_UNCOMMITTED)
> instead of inferring it from the processing.mode?
>
> N1, N2.
> These were only primitives to avoid boxing costs, but since this is not a
> performance sensitive area, it should be fine to change if that's desirable.
>
> N3.
> It's because the store "manages its own offsets", which includes both
> committing the offset, *and providing it* via getCommittedOffset().
> Personally, I think "managesOffsets" conveys this best, but I don't mind
> changing it if the nomenclature is unclear.
>
> Sorry for the massive emails/essays!
> --
> Nick
>
> 1: https://github.com/facebook/rocksdb/wiki/Transactions
> 2: https://github.com/facebook/rocksdb/wiki/Snapshot
> 3: https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
> 4: https://mariadb.com/kb/en/myrocks-transactional-isolation/
>
> On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
> <lb...@confluent.io.invalid> wrote:
>
>> Hi Nick,
>>
>> since I last read it in April, the KIP has become much cleaner and
>> easier to read. Great work!
>>
>> It feels to me the last big open point is whether we can implement
>> isolation level as a query parameter. I understand that there are
>> implementation concerns, but as Colt says, it would be a great
>> addition, and would also simplify the migration path for this change.
>> Is the implementation problem you mentioned caused by the WriteBatch
>> not having a notion of a snapshot, as the underlying DB iterator does?
>> In that case, I am not sure a chain of WriteBatches as you propose
>> would fully solve the problem, but maybe I didn't dig enough into the
>> details to fully understand it.
>>
>> If it's not possible to implement it now, would it be an option to
>> make sure in this KIP that we do not fully close the door on per-query
>> isolation levels in the interface, as it may be possible to implement
>> the missing primitives in RocksDB or Speedb in the future.
>>
>> Understanding:
>>
>> * U1) Why is it only "SHOULD" for changelogOffsets to be persisted
>> atomically with the records?
>> * U2) Don't understand the default implementation of `isolationLevel`.
>> The isolation level should be a property of the underlying store, and
>> not be defined by the default config? Existing stores probably don't
>> guarantee READ_COMMITTED, so the default should be to return
>> READ_UNCOMMITTED.
>>
>> Nits:
>> * N1) Could `getComittedOffset` use an `OptionalLong` return type, to
>> avoid the `null`?
>> * N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
>> return type, to avoid the `-1`?
>> * N3) I don't understand why `managesOffsets` uses the 'manage' verb,
>> whereas all other methods use the "commits" verb. I'd suggest
>> `commitsOffsets`.
>>
>> Either way, it feels this KIP is very close to the finish line, I'm
>> looking forward to seeing this in production!
>>
>> Cheers,
>> Lucas
>>
>> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <co...@littlehorse.io> wrote:
>> >
>> > > Making IsolationLevel a query-time constraint, rather than linking it
>> to
>> > the processing.guarantee.
>> >
>> > As I understand it, would this allow even a user of EOS to control
>> whether
>> > reading committed or uncommitted records? If so, I am highly in favor of
>> > this.
>> >
>> > I know that I was one of the early people to point out the current
>> > shortcoming that IQ reads uncommitted records, but just this morning I
>> > realized a pattern we use which means that (for certain queries) our
>> system
>> > needs to be able to read uncommitted records, which is the current
>> behavior
>> > of Kafka Streams in EOS.***
>> >
>> > If IsolationLevel being a query-time decision allows for this, then that
>> > would be amazing. I would also vote that the default behavior should be
>> for
>> > reading uncommitted records, because it is totally possible for a valid
>> > application to depend on that behavior, and breaking it in a minor
>> release
>> > might be a bit strong.
>> >
>> > *** (Note, for the curious reader....) Our use-case/query pattern is a
>> bit
>> > complex, but reading "uncommitted" records is actually safe in our case
>> > 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 soon as the processor processes the result. If they
>> > can't read uncommitted records, they may receive a "201 - Created"
>> > response, immediately followed by a "404 - 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 <ni...@gmail.com>
>> wrote:
>> >
>> > > Addendum:
>> > >
>> > > I think we would also face the same problem with the approach John
>> outlined
>> > > earlier (using the record cache as a transaction buffer and flushing
>> it
>> > > straight to SST files). This is because the record cache (the
>> ThreadCache
>> > > class) is not thread-safe, so every commit would invalidate open IQ
>> > > Iterators in the same way that RocksDB WriteBatches do.
>> > > --
>> > > Nick
>> > >
>> > > On Wed, 13 Sept 2023 at 16:58, Nick Telford <ni...@gmail.com>
>> > > 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.
>> > > > 2. Making IsolationLevel a query-time constraint, rather than
>> linking it
>> > > > to the processing.guarantee.
>> > > >
>> > > > There's a wrinkle that makes this a challenge: Interactive Queries
>> that
>> > > > open an Iterator, when using transactions and READ_UNCOMMITTED.
>> > > > The problem is that under READ_UNCOMMITTED, queries need to be able
>> to
>> > > > read records from the currently uncommitted transaction buffer
>> > > > (WriteBatch). This includes for Iterators, which should iterate
>> both the
>> > > > transaction buffer and underlying database (using
>> > > > WriteBatch#iteratorWithBase()).
>> > > >
>> > > > The issue is that when the StreamThread commits, it writes the
>> current
>> > > > WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing the
>> > > > WriteBatch while an Interactive Query holds an open Iterator on it
>> will
>> > > > invalidate the Iterator. Worse, it turns out that Iterators over a
>> > > > WriteBatch become invalidated not just when the WriteBatch is
>> cleared,
>> > > but
>> > > > also when the Iterators' current key receives a new write.
>> > > >
>> > > > Now that I'm writing this, I remember that this is the major reason
>> that
>> > > I
>> > > > switched the original design from having a query-time
>> IsolationLevel to
>> > > > having the IsolationLevel linked to the transactionality of the
>> stores
>> > > > themselves.
>> > > >
>> > > > It *might* be possible to resolve this, by having a "chain" of
>> > > > WriteBatches, with the StreamThread switching to a new WriteBatch
>> > > whenever
>> > > > a new Interactive Query attempts to read from the database, but that
>> > > could
>> > > > cause some performance problems/memory pressure when subjected to a
>> high
>> > > > Interactive Query load. It would also reduce the efficiency of
>> > > WriteBatches
>> > > > on-commit, as we'd have to write N WriteBatches, where N is the
>> number of
>> > > > Interactive Queries since the last commit.
>> > > >
>> > > > I realise this is getting into the weeds of the implementation, and
>> you'd
>> > > > rather we focus on the API for now, but I think it's important to
>> > > consider
>> > > > how to implement the desired API, in case we come up with an API
>> that
>> > > > cannot be implemented efficiently, or even at all!
>> > > >
>> > > > Thoughts?
>> > > > --
>> > > > Nick
>> > > >
>> > > > On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <ca...@apache.org>
>> wrote:
>> > > >
>> > > >> Hi Nick,
>> > > >>
>> > > >> 6.
>> > > >> Of course, you are right! My bad!
>> > > >> Wiping out the state in the downgrading case is fine.
>> > > >>
>> > > >>
>> > > >> 3a.
>> > > >> Focus on the public facing changes for the KIP. We will manage 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.
>> > > >>
>> > > >>
>> > > >> Best,
>> > > >> Bruno
>> > > >>
>> > > >> On 9/13/23 11:57 AM, Nick Telford wrote:
>> > > >> > Hi Bruno,
>> > > >> >
>> > > >> > Thanks for getting back to me!
>> > > >> >
>> > > >> > 2.
>> > > >> > The fact that implementations can always track estimated memory
>> usage
>> > > in
>> > > >> > the wrapper is a good point. I can remove -1 as an option, and
>> I'll
>> > > >> clarify
>> > > >> > the JavaDoc that 0 is not just for non-transactional stores,
>> which is
>> > > >> > currently misleading.
>> > > >> >
>> > > >> > 6.
>> > > >> > The problem with catching the exception in the downgrade process
>> is
>> > > that
>> > > >> > would require new code in the Kafka version being downgraded to.
>> Since
>> > > >> > users could conceivably downgrade to almost *any* older version
>> of
>> > > Kafka
>> > > >> > Streams, I'm not sure how we could add that code?
>> > > >> > The only way I can think of doing it would be to provide a
>> dedicated
>> > > >> > downgrade tool, that goes through every local store and removes
>> the
>> > > >> > offsets column families. But that seems like an unnecessary
>> amount of
>> > > >> extra
>> > > >> > code to maintain just to handle a somewhat niche situation, when
>> the
>> > > >> > alternative (automatically wipe and restore stores) should be
>> > > >> acceptable.
>> > > >> >
>> > > >> > 1, 4, 5: Agreed. I'll make the changes you've requested.
>> > > >> >
>> > > >> > 3a.
>> > > >> > I agree that IsolationLevel makes more sense at query-time, and I
>> > > >> actually
>> > > >> > initially attempted to place the IsolationLevel at query-time,
>> but I
>> > > ran
>> > > >> > into some problems:
>> > > >> > - The key issue is that, under ALOS we're not staging writes in
>> > > >> > transactions, so can't perform writes at the READ_COMMITTED
>> isolation
>> > > >> > level. However, this may be addressed if we decide to *always*
>> use
>> > > >> > transactions as discussed under 3b.
>> > > >> > - IQv1 and IQv2 have quite different implementations. I remember
>> > > having
>> > > >> > some difficulty understanding the IQv1 internals, which made it
>> > > >> difficult
>> > > >> > to determine what needed to be changed. However, I *think* this
>> can be
>> > > >> > addressed for both implementations by wrapping the RocksDBStore
>> in an
>> > > >> > IsolationLevel-dependent wrapper, that overrides read methods
>> (get,
>> > > >> etc.)
>> > > >> > to either read directly from the database or from the ongoing
>> > > >> transaction.
>> > > >> > But IQv1 might still be difficult.
>> > > >> > - If IsolationLevel becomes a query constraint, then all other
>> > > >> StateStores
>> > > >> > will need to respect it, including the in-memory stores. This
>> would
>> > > >> require
>> > > >> > us to adapt in-memory stores to stage their writes so they can be
>> > > >> isolated
>> > > >> > from READ_COMMITTTED queries. It would also become an important
>> > > >> > consideration for third-party stores on upgrade, as without
>> changes,
>> > > >> they
>> > > >> > would not support READ_COMMITTED queries correctly.
>> > > >> >
>> > > >> > Ultimately, I may need some help making the necessary change to
>> IQv1
>> > > to
>> > > >> > support this, but I don't think it's fundamentally impossible,
>> if we
>> > > >> want
>> > > >> > to pursue this route.
>> > > >> >
>> > > >> > 3b.
>> > > >> > The main reason I chose to keep ALOS un-transactional was to
>> minimize
>> > > >> > behavioural change for most users (I believe most Streams users
>> use
>> > > the
>> > > >> > default configuration, which is ALOS). That said, it's clear
>> that if
>> > > >> ALOS
>> > > >> > also used transactional stores, the only change in behaviour
>> would be
>> > > >> that
>> > > >> > it would become *more correct*, which could be considered a "bug
>> fix"
>> > > by
>> > > >> > users, rather than a change they need to handle.
>> > > >> >
>> > > >> > I believe that performance using transactions (aka. RocksDB
>> > > >> WriteBatches)
>> > > >> > should actually be *better* than the un-batched write-path that
>> is
>> > > >> > currently used[1]. The only "performance" consideration will be
>> the
>> > > >> > increased memory usage that transactions require. Given the
>> > > mitigations
>> > > >> for
>> > > >> > this memory that we have in place, I would expect that this is
>> not a
>> > > >> > problem for most users.
>> > > >> >
>> > > >> > If we're happy to do so, we can make ALOS also use transactions.
>> > > >> >
>> > > >> > Regards,
>> > > >> > Nick
>> > > >> >
>> > > >> > Link 1:
>> > > >> >
>> > >
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>> > > >> >
>> > > >> > On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <cadonna@apache.org
>> >
>> > > >> wrote:
>> > > >> >
>> > > >> >> Hi Nick,
>> > > >> >>
>> > > >> >> Thanks for the updates and sorry for the delay on my side!
>> > > >> >>
>> > > >> >>
>> > > >> >> 1.
>> > > >> >> Making the default implementation for flush() a no-op sounds
>> good to
>> > > >> me.
>> > > >> >>
>> > > >> >>
>> > > >> >> 2.
>> > > >> >> I think what was bugging me here is that a third-party state
>> store
>> > > >> needs
>> > > >> >> to implement the state store interface. That means they need to
>> > > >> >> implement a wrapper around the actual state store as we do for
>> > > RocksDB
>> > > >> >> with RocksDBStore. So, a third-party state store can always
>> estimate
>> > > >> the
>> > > >> >> uncommitted bytes, if it wants, because the wrapper can record
>> the
>> > > >> added
>> > > >> >> bytes.
>> > > >> >> One case I can think of where returning -1 makes sense is when
>> > > Streams
>> > > >> >> does not need to estimate the size of the write batch and
>> trigger
>> > > >> >> extraordinary commits, because the third-party state store
>> takes care
>> > > >> of
>> > > >> >> memory. But in that case the method could also just return 0.
>> Even
>> > > that
>> > > >> >> case would be better solved with a method that returns whether
>> the
>> > > >> state
>> > > >> >> store manages itself the memory used for uncommitted bytes or
>> not.
>> > > >> >> Said that, I am fine with keeping the -1 return value, I was
>> just
>> > > >> >> wondering when and if it will be used.
>> > > >> >>
>> > > >> >> Regarding returning 0 for transactional state stores when the
>> batch
>> > > is
>> > > >> >> empty, I was just wondering because you explicitly stated
>> > > >> >>
>> > > >> >> "or {@code 0} if this StateStore does not support transactions."
>> > > >> >>
>> > > >> >> So it seemed to me returning 0 could only happen for
>> > > non-transactional
>> > > >> >> state stores.
>> > > >> >>
>> > > >> >>
>> > > >> >> 3.
>> > > >> >>
>> > > >> >> a) What do you think if we move the isolation level to IQ (v1
>> and
>> > > v2)?
>> > > >> >> In the end this is the only component that really needs to
>> specify
>> > > the
>> > > >> >> isolation level. It is similar to the Kafka consumer that can
>> choose
>> > > >> >> with what isolation level to read the input topic.
>> > > >> >> For IQv1 the isolation level should go into
>> StoreQueryParameters. For
>> > > >> >> IQv2, I would add it to the Query interface.
>> > > >> >>
>> > > >> >> b) Point a) raises the question what should happen during
>> > > at-least-once
>> > > >> >> processing when the state store does not use transactions? John
>> in
>> > > the
>> > > >> >> past proposed to also use transactions on state stores for
>> > > >> >> at-least-once. I like that idea, because it avoids aggregating
>> the
>> > > same
>> > > >> >> records over and over again in the case of a failure. We had a
>> case
>> > > in
>> > > >> >> the past where a Streams applications in at-least-once mode was
>> > > failing
>> > > >> >> continuously for some reasons I do not remember before
>> committing the
>> > > >> >> offsets. After each failover, the app aggregated again and
>> again the
>> > > >> >> same records. Of course the aggregate increased to very wrong
>> values
>> > > >> >> just because of the failover. With transactions on the state
>> stores
>> > > we
>> > > >> >> could have avoided this. The app would have output the same
>> aggregate
>> > > >> >> multiple times (i.e., after each failover) but at least the
>> value of
>> > > >> the
>> > > >> >> aggregate would not depend on the number of failovers.
>> Outputting the
>> > > >> >> same aggregate multiple times would be incorrect under
>> exactly-once
>> > > but
>> > > >> >> it is OK for at-least-once.
>> > > >> >> If it makes sense to add a config to turn on and off
>> transactions on
>> > > >> >> state stores under at-least-once or just use transactions in
>> any case
>> > > >> is
>> > > >> >> a question we should also discuss in this KIP. It depends a bit
>> on
>> > > the
>> > > >> >> performance trade-off. Maybe to be safe, I would add a config.
>> > > >> >>
>> > > >> >>
>> > > >> >> 4.
>> > > >> >> Your points are all valid. I tend to say to keep the metrics
>> around
>> > > >> >> flush() until we remove flush() completely from the interface.
>> Calls
>> > > to
>> > > >> >> flush() might still exist since existing processors might still
>> call
>> > > >> >> flush() explicitly as you mentioned in 1). For sure, we need to
>> > > >> document
>> > > >> >> how the metrics change due to the transactions in the upgrade
>> notes.
>> > > >> >>
>> > > >> >>
>> > > >> >> 5.
>> > > >> >> I see. Then you should describe how the .position files are
>> handled
>> > > in
>> > > >> >> a dedicated section of the KIP or incorporate the description
>> in the
>> > > >> >> "Atomic Checkpointing" section instead of only mentioning it in
>> the
>> > > >> >> "Compatibility, 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:
>> > > >> >>
>> > > >> >>
>> > > >> >>
>> > > >>
>> > >
>> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
>> > > >> >>
>> > > >> >> It is a bit brittle, but it works.
>> > > >> >>
>> > > >> >>
>> > > >> >> Best,
>> > > >> >> Bruno
>> > > >> >>
>> > > >> >>
>> > > >> >> On 8/24/23 12:18 PM, Nick Telford wrote:
>> > > >> >>> Hi Bruno,
>> > > >> >>>
>> > > >> >>> Thanks for taking the time to review the KIP. I'm back from
>> leave
>> > > now
>> > > >> and
>> > > >> >>> intend to move this forwards as quickly as I can.
>> > > >> >>>
>> > > >> >>> Addressing your points:
>> > > >> >>>
>> > > >> >>> 1.
>> > > >> >>> Because flush() is part of the StateStore API, it's exposed to
>> > > custom
>> > > >> >>> Processors, which might be making calls to flush(). This was
>> > > actually
>> > > >> the
>> > > >> >>> case in a few integration tests.
>> > > >> >>> To maintain as much compatibility as possible, I'd prefer not
>> to
>> > > make
>> > > >> >> this
>> > > >> >>> an UnsupportedOperationException, as it will cause previously
>> > > working
>> > > >> >>> Processors to start throwing exceptions at runtime.
>> > > >> >>> I agree that it doesn't make sense for it to proxy commit(),
>> though,
>> > > >> as
>> > > >> >>> that would cause it to violate the "StateStores commit only
>> when the
>> > > >> Task
>> > > >> >>> commits" rule.
>> > > >> >>> Instead, I think we should make this a no-op. That way,
>> existing
>> > > user
>> > > >> >>> Processors will continue to work as-before, without violation
>> of
>> > > store
>> > > >> >>> consistency that would be caused by premature flush/commit of
>> > > >> StateStore
>> > > >> >>> data to disk.
>> > > >> >>> What do you think?
>> > > >> >>>
>> > > >> >>> 2.
>> > > >> >>> As stated in the JavaDoc, when a StateStore implementation is
>> > > >> >>> transactional, but is unable to estimate the uncommitted memory
>> > > usage,
>> > > >> >> the
>> > > >> >>> method will return -1.
>> > > >> >>> The intention here is to permit third-party implementations
>> that may
>> > > >> not
>> > > >> >> be
>> > > >> >>> able to estimate memory usage.
>> > > >> >>>
>> > > >> >>> Yes, it will be 0 when nothing has been written to the store
>> yet. I
>> > > >> >> thought
>> > > >> >>> that was implied by "This method will return an approximation
>> of the
>> > > >> >> memory
>> > > >> >>> would be freed by the next call to {@link #commit(Map)}" and
>> > > "@return
>> > > >> The
>> > > >> >>> approximate size of all records awaiting {@link #commit(Map)}",
>> > > >> however,
>> > > >> >> I
>> > > >> >>> can add it explicitly to the JavaDoc if you think this is
>> unclear?
>> > > >> >>>
>> > > >> >>> 3.
>> > > >> >>> I realise this is probably the most contentious point in my
>> design,
>> > > >> and
>> > > >> >> I'm
>> > > >> >>> open to changing it if I'm unable to convince you of the
>> benefits.
>> > > >> >>> Nevertheless, here's my argument:
>> > > >> >>> The Interactive Query (IQ) API(s) are directly provided
>> StateStores
>> > > to
>> > > >> >>> query, and it may be important for users to programmatically
>> know
>> > > >> which
>> > > >> >>> mode the StateStore is operating under. If we simply provide an
>> > > >> >>> "eosEnabled" boolean (as used throughout the internal streams
>> > > >> engine), or
>> > > >> >>> similar, then users will need to understand the operation and
>> > > >> >> consequences
>> > > >> >>> of each available processing mode and how it pertains to their
>> > > >> >> StateStore.
>> > > >> >>>
>> > > >> >>> Interactive Query users aren't the only people that care about
>> the
>> > > >> >>> processing.mode/IsolationLevel of a StateStore: implementers of
>> > > custom
>> > > >> >>> StateStores also need to understand the behaviour expected of
>> their
>> > > >> >>> implementation. KIP-892 introduces some assumptions into the
>> Streams
>> > > >> >> Engine
>> > > >> >>> about how StateStores operate under each processing mode, and
>> it's
>> > > >> >>> important that custom implementations adhere to those
>> assumptions in
>> > > >> >> order
>> > > >> >>> to maintain the consistency guarantees.
>> > > >> >>>
>> > > >> >>> IsolationLevels provide a high-level contract on the behaviour
>> of
>> > > the
>> > > >> >>> StateStore: a user knows that under READ_COMMITTED, they will
>> see
>> > > >> writes
>> > > >> >>> only after the Task has committed, and under READ_UNCOMMITTED
>> they
>> > > >> will
>> > > >> >> see
>> > > >> >>> writes immediately. No understanding of the details of each
>> > > >> >> processing.mode
>> > > >> >>> is required, either for IQ users or StateStore implementers.
>> > > >> >>>
>> > > >> >>> An argument can be made that these contractual guarantees can
>> simply
>> > > >> be
>> > > >> >>> documented for the processing.mode (i.e. that exactly-once and
>> > > >> >>> exactly-once-v2 behave like READ_COMMITTED and at-least-once
>> behaves
>> > > >> like
>> > > >> >>> READ_UNCOMMITTED), but there are several small issues with
>> this I'd
>> > > >> >> prefer
>> > > >> >>> to avoid:
>> > > >> >>>
>> > > >> >>>      - Where would we document these contracts, in a way that
>> is
>> > > >> difficult
>> > > >> >>>      for users/implementers to miss/ignore?
>> > > >> >>>      - It's not clear to users that the processing mode is
>> > > >> communicating
>> > > >> >>>      an expectation of read isolation, unless they read the
>> > > >> >> documentation. Users
>> > > >> >>>      rarely consult documentation unless they feel they need
>> to, so
>> > > >> it's
>> > > >> >> likely
>> > > >> >>>      this detail would get missed by many users.
>> > > >> >>>      - It tightly couples processing modes to read isolation.
>> Adding
>> > > >> new
>> > > >> >>>      processing modes, or changing the read isolation of
>> existing
>> > > >> >> processing
>> > > >> >>>      modes would be difficult/impossible.
>> > > >> >>>
>> > > >> >>> Ultimately, the cost of introducing IsolationLevels is just a
>> single
>> > > >> >>> method, since we re-use the existing IsolationLevel enum from
>> Kafka.
>> > > >> This
>> > > >> >>> gives us a clear place to document the contractual guarantees
>> > > expected
>> > > >> >>> of/provided by StateStores, that is accessible both by the
>> > > StateStore
>> > > >> >>> itself, and by IQ users.
>> > > >> >>>
>> > > >> >>> (Writing this I've just realised that the StateStore and IQ
>> APIs
>> > > >> actually
>> > > >> >>> don't provide access to StateStoreContext that IQ users would
>> have
>> > > >> direct
>> > > >> >>> access to... Perhaps StateStore should expose isolationLevel()
>> > > itself
>> > > >> >> too?)
>> > > >> >>>
>> > > >> >>> 4.
>> > > >> >>> Yeah, I'm not comfortable renaming the metrics in-place
>> either, as
>> > > >> it's a
>> > > >> >>> backwards incompatible change. My concern is that, if we leave
>> the
>> > > >> >> existing
>> > > >> >>> "flush" metrics in place, they will be confusing to users.
>> Right
>> > > now,
>> > > >> >>> "flush" metrics record explicit flushes to disk, but under
>> KIP-892,
>> > > >> even
>> > > >> >> a
>> > > >> >>> commit() will not explicitly flush data to disk - RocksDB will
>> > > decide
>> > > >> on
>> > > >> >>> when to flush memtables to disk itself.
>> > > >> >>>
>> > > >> >>> If we keep the existing "flush" metrics, we'd have two options,
>> > > which
>> > > >> >> both
>> > > >> >>> seem pretty bad to me:
>> > > >> >>>
>> > > >> >>>      1. Have them record calls to commit(), which would be
>> > > >> misleading, as
>> > > >> >>>      data is no longer explicitly "flushed" to disk by this
>> call.
>> > > >> >>>      2. Have them record nothing at all, which is equivalent to
>> > > >> removing
>> > > >> >> the
>> > > >> >>>      metrics, except that users will see the metric still
>> exists and
>> > > >> so
>> > > >> >> assume
>> > > >> >>>      that the metric is correct, and that there's a problem
>> with
>> > > their
>> > > >> >> system
>> > > >> >>>      when there isn't.
>> > > >> >>>
>> > > >> >>> I agree that removing them is also a bad solution, and I'd
>> like some
>> > > >> >>> guidance on the best path forward here.
>> > > >> >>>
>> > > >> >>> 5.
>> > > >> >>> Position files are updated on every write to a StateStore.
>> Since our
>> > > >> >> writes
>> > > >> >>> are now buffered until commit(), we can't update the Position
>> file
>> > > >> until
>> > > >> >>> commit() has been called, otherwise it would be inconsistent
>> with
>> > > the
>> > > >> >> data
>> > > >> >>> in the event of a rollback. Consequently, we need to manage
>> these
>> > > >> offsets
>> > > >> >>> the same way we manage the checkpoint offsets, and ensure
>> they're
>> > > only
>> > > >> >>> written on commit().
>> > > >> >>>
>> > > >> >>> 6.
>> > > >> >>> Agreed, although I'm not exactly sure yet what tests to write.
>> How
>> > > >> >> explicit
>> > > >> >>> do we need to be here in the KIP?
>> > > >> >>>
>> > > >> >>> As for upgrade/downgrade: upgrade is designed to be seamless,
>> and we
>> > > >> >> should
>> > > >> >>> definitely add some tests around that. Downgrade, it
>> transpires,
>> > > isn't
>> > > >> >>> currently possible, as the extra column family for offset
>> storage is
>> > > >> >>> incompatible with the pre-KIP-892 implementation: when you
>> open a
>> > > >> RocksDB
>> > > >> >>> database, you must open all available column families or
>> receive an
>> > > >> >> error.
>> > > >> >>> What currently happens on downgrade is that it attempts to
>> open the
>> > > >> >> store,
>> > > >> >>> throws an error about the offsets column family not being
>> opened,
>> > > >> which
>> > > >> >>> triggers a wipe and rebuild of the Task. Given that downgrades
>> > > should
>> > > >> be
>> > > >> >>> uncommon, I think this is acceptable behaviour, as the
>> end-state is
>> > > >> >>> consistent, even if it results in an undesirable state restore.
>> > > >> >>>
>> > > >> >>> Should I document the upgrade/downgrade behaviour explicitly
>> in the
>> > > >> KIP?
>> > > >> >>>
>> > > >> >>> --
>> > > >> >>>
>> > > >> >>> Regards,
>> > > >> >>> Nick
>> > > >> >>>
>> > > >> >>>
>> > > >> >>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <
>> cadonna@apache.org>
>> > > >> wrote:
>> > > >> >>>
>> > > >> >>>> Hi Nick!
>> > > >> >>>>
>> > > >> >>>> Thanks for the updates!
>> > > >> >>>>
>> > > >> >>>> 1.
>> > > >> >>>> Why does StateStore#flush() default to
>> > > >> >>>> StateStore#commit(Collections.emptyMap())?
>> > > >> >>>> Since calls to flush() will not exist anymore after this KIP
>> is
>> > > >> >>>> released, I would rather throw an unsupported operation
>> exception
>> > > by
>> > > >> >>>> default.
>> > > >> >>>>
>> > > >> >>>>
>> > > >> >>>> 2.
>> > > >> >>>> When would a state store return -1 from
>> > > >> >>>> StateStore#approximateNumUncommittedBytes() while being
>> > > >> transactional?
>> > > >> >>>>
>> > > >> >>>> Wouldn't StateStore#approximateNumUncommittedBytes() also
>> return 0
>> > > if
>> > > >> >>>> the state store is transactional but nothing has been written
>> to
>> > > the
>> > > >> >>>> state store yet?
>> > > >> >>>>
>> > > >> >>>>
>> > > >> >>>> 3.
>> > > >> >>>> Sorry for bringing this up again. Does this KIP really need to
>> > > >> introduce
>> > > >> >>>> StateStoreContext#isolationLevel()? StateStoreContext has
>> already
>> > > >> >>>> appConfigs() which basically exposes the same information,
>> i.e., if
>> > > >> EOS
>> > > >> >>>> is enabled or not.
>> > > >> >>>> In one of your previous e-mails you wrote:
>> > > >> >>>>
>> > > >> >>>> "My idea was to try to keep the StateStore interface as
>> loosely
>> > > >> coupled
>> > > >> >>>> from the Streams engine as possible, to give implementers more
>> > > >> freedom,
>> > > >> >>>> and reduce the amount of internal knowledge required."
>> > > >> >>>>
>> > > >> >>>> While I understand the intent, I doubt that it decreases the
>> > > >> coupling of
>> > > >> >>>> a StateStore interface and the Streams engine. READ_COMMITTED
>> only
>> > > >> >>>> applies to IQ but not to reads by processors. Thus,
>> implementers
>> > > >> need to
>> > > >> >>>> understand how Streams accesses the state stores.
>> > > >> >>>>
>> > > >> >>>> I would like to hear what others think about this.
>> > > >> >>>>
>> > > >> >>>>
>> > > >> >>>> 4.
>> > > >> >>>> Great exposing new metrics for transactional state stores!
>> > > However, I
>> > > >> >>>> would prefer to add new metrics and deprecate (in the docs)
>> the 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 also need to adapt/add integration tests
>> besides
>> > > unit
>> > > >> >>>> tests. Additionally, we probably need integration or system
>> tests
>> > > to
>> > > >> >>>> verify that upgrades and downgrades between transactional and
>> > > >> >>>> non-transactional state stores work as expected.
>> > > >> >>>>
>> > > >> >>>>
>> > > >> >>>> Best,
>> > > >> >>>> Bruno
>> > > >> >>>>
>> > > >> >>>>
>> > > >> >>>>
>> > > >> >>>>
>> > > >> >>>>
>> > > >> >>>> On 7/21/23 10:34 PM, Nick Telford wrote:
>> > > >> >>>>> One more thing: I noted John's suggestion in the KIP, under
>> > > >> "Rejected
>> > > >> >>>>> Alternatives". I still think it's an idea worth pursuing,
>> but I
>> > > >> believe
>> > > >> >>>>> that it's out of the scope of this KIP, because it solves a
>> > > >> different
>> > > >> >> set
>> > > >> >>>>> of problems to this KIP, and the scope of this one has
>> already
>> > > grown
>> > > >> >>>> quite
>> > > >> >>>>> large!
>> > > >> >>>>>
>> > > >> >>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
>> > > nick.telford@gmail.com>
>> > > >> >>>> wrote:
>> > > >> >>>>>
>> > > >> >>>>>> Hi everyone,
>> > > >> >>>>>>
>> > > >> >>>>>> I've updated the KIP (
>> > > >> >>>>>>
>> > > >> >>>>
>> > > >> >>
>> > > >>
>> > >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> > > >> >>>> )
>> > > >> >>>>>> with the latest changes; mostly bringing back "Atomic
>> > > >> Checkpointing"
>> > > >> >>>> (for
>> > > >> >>>>>> what feels like the 10th time!). I think the one thing
>> missing is
>> > > >> some
>> > > >> >>>>>> changes to metrics (notably the store "flush" metrics will
>> need
>> > > to
>> > > >> be
>> > > >> >>>>>> renamed to "commit").
>> > > >> >>>>>>
>> > > >> >>>>>> The reason I brought back Atomic Checkpointing was to
>> decouple
>> > > >> store
>> > > >> >>>> flush
>> > > >> >>>>>> from store commit. This is important, because with
>> Transactional
>> > > >> >>>>>> StateStores, we now need to call "flush" on *every* Task
>> commit,
>> > > >> and
>> > > >> >> not
>> > > >> >>>>>> just when the StateStore is closing, otherwise our
>> transaction
>> > > >> buffer
>> > > >> >>>> will
>> > > >> >>>>>> never be written and persisted, instead growing unbounded! I
>> > > >> >>>> experimented
>> > > >> >>>>>> with some simple solutions, like forcing a store flush
>> whenever
>> > > the
>> > > >> >>>>>> transaction buffer was likely to exceed its configured
>> size, but
>> > > >> this
>> > > >> >>>> was
>> > > >> >>>>>> brittle: it prevented the transaction buffer from being
>> > > configured
>> > > >> to
>> > > >> >> be
>> > > >> >>>>>> unbounded, and it still would have required explicit
>> flushes of
>> > > >> >> RocksDB,
>> > > >> >>>>>> yielding sub-optimal performance and memory utilization.
>> > > >> >>>>>>
>> > > >> >>>>>> I deemed Atomic Checkpointing to be the "right" way to
>> resolve
>> > > this
>> > > >> >>>>>> problem. By ensuring that the changelog offsets that
>> correspond
>> > > to
>> > > >> the
>> > > >> >>>> most
>> > > >> >>>>>> recently written records are always atomically written to
>> the
>> > > >> >> StateStore
>> > > >> >>>>>> (by writing them to the same transaction buffer), we can
>> avoid
>> > > >> >> forcibly
>> > > >> >>>>>> flushing the RocksDB memtables to disk, letting RocksDB
>> flush
>> > > them
>> > > >> >> only
>> > > >> >>>>>> when necessary, without losing any of our consistency
>> guarantees.
>> > > >> See
>> > > >> >>>> the
>> > > >> >>>>>> updated KIP for more info.
>> > > >> >>>>>>
>> > > >> >>>>>> I have fully implemented these changes, although I'm still
>> not
>> > > >> >> entirely
>> > > >> >>>>>> happy with the implementation for segmented StateStores, so
>> I
>> > > plan
>> > > >> to
>> > > >> >>>>>> refactor that. Despite that, all tests pass. If you'd like
>> to try
>> > > >> out
>> > > >> >> or
>> > > >> >>>>>> review this highly experimental and incomplete branch, it's
>> > > >> available
>> > > >> >>>> here:
>> > > >> >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0.
>> Note:
>> > > >> it's
>> > > >> >>>> built
>> > > >> >>>>>> against Kafka 3.5.0 so that I had a stable base to build
>> and test
>> > > >> it
>> > > >> >> on,
>> > > >> >>>>>> and to enable easy apples-to-apples comparisons in a live
>> > > >> >> environment. I
>> > > >> >>>>>> plan to rebase it against trunk once it's nearer completion
>> and
>> > > has
>> > > >> >> been
>> > > >> >>>>>> proven on our main application.
>> > > >> >>>>>>
>> > > >> >>>>>> I would really appreciate help in reviewing and testing:
>> > > >> >>>>>> - Segmented (Versioned, Session and Window) stores
>> > > >> >>>>>> - Global stores
>> > > >> >>>>>>
>> > > >> >>>>>> As I do not currently use either of these, so my primary
>> test
>> > > >> >>>> environment
>> > > >> >>>>>> doesn't test these areas.
>> > > >> >>>>>>
>> > > >> >>>>>> I'm going on Parental Leave starting next week for a few
>> weeks,
>> > > so
>> > > >> >> will
>> > > >> >>>>>> not have time to move this forward until late August. That
>> said,
>> > > >> your
>> > > >> >>>>>> feedback is welcome and appreciated, I just won't be able to
>> > > >> respond
>> > > >> >> as
>> > > >> >>>>>> quickly as usual.
>> > > >> >>>>>>
>> > > >> >>>>>> Regards,
>> > > >> >>>>>> Nick
>> > > >> >>>>>>
>> > > >> >>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
>> > > nick.telford@gmail.com>
>> > > >> >>>> wrote:
>> > > >> >>>>>>
>> > > >> >>>>>>> Hi Bruno
>> > > >> >>>>>>>
>> > > >> >>>>>>> Yes, that's correct, although the impact on IQ is not
>> something
>> > > I
>> > > >> had
>> > > >> >>>>>>> considered.
>> > > >> >>>>>>>
>> > > >> >>>>>>> What about atomically updating the state store from the
>> > > >> transaction
>> > > >> >>>>>>>> buffer every commit interval and writing the checkpoint
>> (thus,
>> > > >> >>>> flushing
>> > > >> >>>>>>>> the memtable) every configured amount of data and/or
>> number of
>> > > >> >> commit
>> > > >> >>>>>>>> intervals?
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>>> I'm not quite sure I follow. Are you suggesting that we
>> add an
>> > > >> >>>> additional
>> > > >> >>>>>>> config for the max number of commit intervals between
>> > > checkpoints?
>> > > >> >> That
>> > > >> >>>>>>> way, we would checkpoint *either* when the transaction
>> buffers
>> > > are
>> > > >> >>>> nearly
>> > > >> >>>>>>> full, *OR* whenever a certain number of commit intervals
>> have
>> > > >> >> elapsed,
>> > > >> >>>>>>> whichever comes first?
>> > > >> >>>>>>>
>> > > >> >>>>>>> That certainly seems reasonable, although this re-ignites
>> an
>> > > >> earlier
>> > > >> >>>>>>> debate about whether a config should be measured in
>> "number of
>> > > >> commit
>> > > >> >>>>>>> intervals", instead of just an absolute time.
>> > > >> >>>>>>>
>> > > >> >>>>>>> FWIW, I realised that this issue is the reason I was
>> pursuing
>> > > the
>> > > >> >>>> Atomic
>> > > >> >>>>>>> Checkpoints, as it de-couples memtable flush from
>> checkpointing,
>> > > >> >> which
>> > > >> >>>>>>> enables us to just checkpoint on every commit without any
>> > > >> performance
>> > > >> >>>>>>> impact. Atomic Checkpointing is definitely the "best"
>> solution,
>> > > >> but
>> > > >> >>>> I'm not
>> > > >> >>>>>>> sure if this is enough to bring it back into this KIP.
>> > > >> >>>>>>>
>> > > >> >>>>>>> I'm currently working on moving all the transactional logic
>> > > >> directly
>> > > >> >>>> into
>> > > >> >>>>>>> RocksDBStore itself, which does away with the
>> > > >> >> StateStore#newTransaction
>> > > >> >>>>>>> method, and reduces the number of new classes introduced,
>> > > >> >> significantly
>> > > >> >>>>>>> reducing the complexity. If it works, and the complexity is
>> > > >> >> drastically
>> > > >> >>>>>>> reduced, I may try bringing back Atomic Checkpoints into
>> this
>> > > KIP.
>> > > >> >>>>>>>
>> > > >> >>>>>>> Regards,
>> > > >> >>>>>>> Nick
>> > > >> >>>>>>>
>> > > >> >>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
>> cadonna@apache.org>
>> > > >> >> wrote:
>> > > >> >>>>>>>
>> > > >> >>>>>>>> Hi Nick,
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> Thanks for the insights! Very interesting!
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> As far as I understand, you want to atomically update the
>> state
>> > > >> >> store
>> > > >> >>>>>>>> from the transaction buffer, flush the memtable of a state
>> > > store
>> > > >> and
>> > > >> >>>>>>>> write the checkpoint not after the commit time elapsed but
>> > > after
>> > > >> the
>> > > >> >>>>>>>> transaction buffer reached a size that would lead to
>> exceeding
>> > > >> >>>>>>>> statestore.transaction.buffer.max.bytes before the next
>> commit
>> > > >> >>>> interval
>> > > >> >>>>>>>> ends.
>> > > >> >>>>>>>> That means, the Kafka transaction would commit every
>> commit
>> > > >> interval
>> > > >> >>>> but
>> > > >> >>>>>>>> the state store will only be atomically updated roughly
>> every
>> > > >> >>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ
>> would
>> > > >> then
>> > > >> >>>> only
>> > > >> >>>>>>>> see new data roughly every
>> > > >> statestore.transaction.buffer.max.bytes.
>> > > >> >>>>>>>> After a failure the state store needs to restore up to
>> > > >> >>>>>>>> statestore.transaction.buffer.max.bytes.
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> Is this correct?
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> What about atomically updating the state store from the
>> > > >> transaction
>> > > >> >>>>>>>> buffer every commit interval and writing the checkpoint
>> (thus,
>> > > >> >>>> flushing
>> > > >> >>>>>>>> the memtable) every configured amount of data and/or
>> number of
>> > > >> >> commit
>> > > >> >>>>>>>> intervals? In such a way, we would have the same delay for
>> > > >> records
>> > > >> >>>>>>>> appearing in output topics and IQ because both would
>> appear
>> > > when
>> > > >> the
>> > > >> >>>>>>>> Kafka transaction is committed. However, after a failure
>> the
>> > > >> state
>> > > >> >>>> store
>> > > >> >>>>>>>> still needs to restore up to
>> > > >> statestore.transaction.buffer.max.bytes
>> > > >> >>>> and
>> > > >> >>>>>>>> it might restore data that is already in the state store
>> > > because
>> > > >> the
>> > > >> >>>>>>>> checkpoint lags behind the last stable offset (i.e. the
>> last
>> > > >> >> committed
>> > > >> >>>>>>>> offset) of the changelog topics. Restoring data that is
>> already
>> > > >> in
>> > > >> >> the
>> > > >> >>>>>>>> state store is idempotent, so eos should not violated.
>> > > >> >>>>>>>> This solution needs at least one new config to specify
>> when a
>> > > >> >>>> checkpoint
>> > > >> >>>>>>>> should be written.
>> > > >> >>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> A small correction to your previous e-mail that does not
>> change
>> > > >> >>>> anything
>> > > >> >>>>>>>> you said: Under alos the default commit interval is 30
>> seconds,
>> > > >> not
>> > > >> >>>> five
>> > > >> >>>>>>>> seconds.
>> > > >> >>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> Best,
>> > > >> >>>>>>>> Bruno
>> > > >> >>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
>> > > >> >>>>>>>>> Hi everyone,
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>> I've begun performance testing my branch on our staging
>> > > >> >> environment,
>> > > >> >>>>>>>>> putting it through its paces in our non-trivial
>> application.
>> > > I'm
>> > > >> >>>>>>>> already
>> > > >> >>>>>>>>> observing the same increased flush rate that we saw the
>> last
>> > > >> time
>> > > >> >> we
>> > > >> >>>>>>>>> attempted to use a version of this KIP, but this time, I
>> > > think I
>> > > >> >> know
>> > > >> >>>>>>>> why.
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at
>> the end
>> > > >> of
>> > > >> >> the
>> > > >> >>>>>>>> Task
>> > > >> >>>>>>>>> commit process, has the following behaviour:
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>>        - Under ALOS: checkpoint the state stores. This
>> > > includes
>> > > >> >>>>>>>>>        flushing memtables in RocksDB. This is acceptable
>> > > >> because the
>> > > >> >>>>>>>> default
>> > > >> >>>>>>>>>        commit.interval.ms is 5 seconds, so forcibly
>> flushing
>> > > >> >> memtables
>> > > >> >>>>>>>> every 5
>> > > >> >>>>>>>>>        seconds is acceptable for most applications.
>> > > >> >>>>>>>>>        - Under EOS: checkpointing is not done, *unless*
>> it's
>> > > >> being
>> > > >> >>>>>>>> forced, due
>> > > >> >>>>>>>>>        to e.g. the Task closing or being revoked. This
>> means
>> > > >> that
>> > > >> >> under
>> > > >> >>>>>>>> normal
>> > > >> >>>>>>>>>        processing conditions, the state stores will not
>> be
>> > > >> >>>> checkpointed,
>> > > >> >>>>>>>> and will
>> > > >> >>>>>>>>>        not have memtables flushed at all , unless RocksDB
>> > > >> decides to
>> > > >> >>>>>>>> flush them on
>> > > >> >>>>>>>>>        its own. Checkpointing stores and force-flushing
>> their
>> > > >> >> memtables
>> > > >> >>>>>>>> is only
>> > > >> >>>>>>>>>        done when a Task is being closed.
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least
>> > > *some*
>> > > >> >>>> normal
>> > > >> >>>>>>>>> Task commits, in order to write the RocksDB transaction
>> > > buffers
>> > > >> to
>> > > >> >>>> the
>> > > >> >>>>>>>>> state stores, and to ensure the offsets are synced to
>> disk to
>> > > >> >> prevent
>> > > >> >>>>>>>>> restores from getting out of hand. Consequently, my
>> current
>> > > >> >>>>>>>> implementation
>> > > >> >>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is
>> far too
>> > > >> >>>>>>>> frequent.
>> > > >> >>>>>>>>> This causes checkpoints every 10,000 records, which is a
>> > > change
>> > > >> in
>> > > >> >>>>>>>> flush
>> > > >> >>>>>>>>> behaviour, potentially causing performance problems for
>> some
>> > > >> >>>>>>>> applications.
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>> I'm looking into possible solutions, and I'm currently
>> leaning
>> > > >> >>>> towards
>> > > >> >>>>>>>>> using the statestore.transaction.buffer.max.bytes
>> > > configuration
>> > > >> to
>> > > >> >>>>>>>>> checkpoint Tasks once we are likely to exceed it. This
>> would
>> > > >> >>>>>>>> complement the
>> > > >> >>>>>>>>> existing "early Task commit" functionality that this
>> > > >> configuration
>> > > >> >>>>>>>>> provides, in the following way:
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>>        - Currently, we use
>> > > >> statestore.transaction.buffer.max.bytes
>> > > >> >> to
>> > > >> >>>>>>>> force an
>> > > >> >>>>>>>>>        early Task commit if processing more records would
>> > > cause
>> > > >> our
>> > > >> >>>> state
>> > > >> >>>>>>>> store
>> > > >> >>>>>>>>>        transactions to exceed the memory assigned to
>> them.
>> > > >> >>>>>>>>>        - New functionality: when a Task *does* commit,
>> we will
>> > > >> not
>> > > >> >>>>>>>> checkpoint
>> > > >> >>>>>>>>>        the stores (and hence flush the transaction
>> buffers)
>> > > >> unless
>> > > >> >> we
>> > > >> >>>>>>>> expect to
>> > > >> >>>>>>>>>        cross the statestore.transaction.buffer.max.bytes
>> > > >> threshold
>> > > >> >>>> before
>> > > >> >>>>>>>> the next
>> > > >> >>>>>>>>>        commit
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>> I'm also open to suggestions.
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>> Regards,
>> > > >> >>>>>>>>> Nick
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
>> > > >> nick.telford@gmail.com
>> > > >> >>>
>> > > >> >>>>>>>> wrote:
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>>> Hi Bruno!
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 3.
>> > > >> >>>>>>>>>> By "less predictable for users", I meant in terms of
>> > > >> understanding
>> > > >> >>>> the
>> > > >> >>>>>>>>>> performance profile under various circumstances. The
>> more
>> > > >> complex
>> > > >> >>>> the
>> > > >> >>>>>>>>>> solution, the more difficult it would be for users to
>> > > >> understand
>> > > >> >> the
>> > > >> >>>>>>>>>> performance they see. For example, spilling records to
>> disk
>> > > >> when
>> > > >> >> the
>> > > >> >>>>>>>>>> transaction buffer reaches a threshold would, I expect,
>> > > reduce
>> > > >> >> write
>> > > >> >>>>>>>>>> throughput. This reduction in write throughput could be
>> > > >> >> unexpected,
>> > > >> >>>>>>>> and
>> > > >> >>>>>>>>>> potentially difficult to diagnose/understand for users.
>> > > >> >>>>>>>>>> At the moment, I think the "early commit" concept is
>> > > relatively
>> > > >> >>>>>>>>>> straightforward; it's easy to document, and conceptually
>> > > fairly
>> > > >> >>>>>>>> obvious to
>> > > >> >>>>>>>>>> users. We could probably add a metric to make it easier
>> to
>> > > >> >>>> understand
>> > > >> >>>>>>>> when
>> > > >> >>>>>>>>>> it happens though.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 3. (the second one)
>> > > >> >>>>>>>>>> The IsolationLevel is *essentially* an indirect way of
>> > > telling
>> > > >> >>>>>>>> StateStores
>> > > >> >>>>>>>>>> whether they should be transactional. READ_COMMITTED
>> > > >> essentially
>> > > >> >>>>>>>> requires
>> > > >> >>>>>>>>>> transactions, because it dictates that two threads
>> calling
>> > > >> >>>>>>>>>> `newTransaction()` should not see writes from the other
>> > > >> >> transaction
>> > > >> >>>>>>>> until
>> > > >> >>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all
>> bets are
>> > > >> off,
>> > > >> >>>> and
>> > > >> >>>>>>>>>> stores can allow threads to observe written records at
>> any
>> > > >> time,
>> > > >> >>>>>>>> which is
>> > > >> >>>>>>>>>> essentially "no transactions". That said, StateStores
>> are
>> > > free
>> > > >> to
>> > > >> >>>>>>>> implement
>> > > >> >>>>>>>>>> these guarantees however they can, which is a bit more
>> > > relaxed
>> > > >> >> than
>> > > >> >>>>>>>>>> dictating "you must use transactions". For example, with
>> > > >> RocksDB
>> > > >> >> we
>> > > >> >>>>>>>> would
>> > > >> >>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
>> > > "transactions",
>> > > >> >>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But
>> with
>> > > >> other
>> > > >> >>>>>>>> storage
>> > > >> >>>>>>>>>> engines, it might be preferable to *always* use
>> transactions,
>> > > >> even
>> > > >> >>>>>>>> when
>> > > >> >>>>>>>>>> unnecessary; or there may be storage engines that don't
>> > > provide
>> > > >> >>>>>>>>>> transactions, but the isolation guarantees can be met
>> using a
>> > > >> >>>>>>>> different
>> > > >> >>>>>>>>>> technique.
>> > > >> >>>>>>>>>> My idea was to try to keep the StateStore interface as
>> > > loosely
>> > > >> >>>> coupled
>> > > >> >>>>>>>>>> from the Streams engine as possible, to give
>> implementers
>> > > more
>> > > >> >>>>>>>> freedom, and
>> > > >> >>>>>>>>>> reduce the amount of internal knowledge required.
>> > > >> >>>>>>>>>> That said, I understand that "IsolationLevel" might not
>> be
>> > > the
>> > > >> >> right
>> > > >> >>>>>>>>>> abstraction, and we can always make it much more
>> explicit if
>> > > >> >>>>>>>> required, e.g.
>> > > >> >>>>>>>>>> boolean transactional()
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> 7-8.
>> > > >> >>>>>>>>>> I can make these changes either later today or tomorrow.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Small update:
>> > > >> >>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of
>> issues
>> > > >> that
>> > > >> >>>>>>>> needed
>> > > >> >>>>>>>>>> addressing. Currently, all the tests pass, which is
>> > > promising,
>> > > >> but
>> > > >> >>>> it
>> > > >> >>>>>>>> will
>> > > >> >>>>>>>>>> need to undergo some performance testing. I haven't
>> (yet)
>> > > >> worked
>> > > >> >> on
>> > > >> >>>>>>>>>> removing the `newTransaction()` stuff, but I would
>> expect
>> > > that,
>> > > >> >>>>>>>>>> behaviourally, it should make no difference. The branch
>> is
>> > > >> >> available
>> > > >> >>>>>>>> at
>> > > >> >>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if
>> > > anyone
>> > > >> is
>> > > >> >>>>>>>>>> interested in taking an early look.
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> Regards,
>> > > >> >>>>>>>>>> Nick
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
>> > > >> cadonna@apache.org>
>> > > >> >>>>>>>> wrote:
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>>> Hi Nick,
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>> 1.
>> > > >> >>>>>>>>>>> Yeah, I agree with you. That was actually also my
>> point. I
>> > > >> >>>> understood
>> > > >> >>>>>>>>>>> that John was proposing the ingestion path as a way to
>> avoid
>> > > >> the
>> > > >> >>>>>>>> early
>> > > >> >>>>>>>>>>> commits. Probably, I misinterpreted the intent.
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>> 2.
>> > > >> >>>>>>>>>>> I agree with John here, that actually it is public
>> API. My
>> > > >> >> question
>> > > >> >>>>>>>> is
>> > > >> >>>>>>>>>>> how this usage pattern affects normal processing.
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>> 3.
>> > > >> >>>>>>>>>>> My concern is that checking for the size of the
>> transaction
>> > > >> >> buffer
>> > > >> >>>>>>>> and
>> > > >> >>>>>>>>>>> maybe triggering an early commit affects the whole
>> > > processing
>> > > >> of
>> > > >> >>>>>>>> Kafka
>> > > >> >>>>>>>>>>> Streams. The transactionality of a state store is not
>> > > >> confined to
>> > > >> >>>> the
>> > > >> >>>>>>>>>>> state store itself, but spills over and changes the
>> behavior
>> > > >> of
>> > > >> >>>> other
>> > > >> >>>>>>>>>>> parts of the system. I agree with you that it is a
>> decent
>> > > >> >>>>>>>> compromise. I
>> > > >> >>>>>>>>>>> just wanted to analyse the downsides and list the
>> options to
>> > > >> >>>> overcome
>> > > >> >>>>>>>>>>> them. I also agree with you that all options seem quite
>> > > heavy
>> > > >> >>>>>>>> compared
>> > > >> >>>>>>>>>>> with your KIP. I do not understand what you mean with
>> "less
>> > > >> >>>>>>>> predictable
>> > > >> >>>>>>>>>>> for users", though.
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>> I found the discussions about the alternatives really
>> > > >> >> interesting.
>> > > >> >>>>>>>> But I
>> > > >> >>>>>>>>>>> also think that your plan sounds good and we should
>> continue
>> > > >> with
>> > > >> >>>> it!
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>> 3.
>> > > >> >>>>>>>>>>> Ah, now, I understand the reasoning behind putting
>> isolation
>> > > >> >> level
>> > > >> >>>> in
>> > > >> >>>>>>>>>>> the state store context. Thanks! Should that also be a
>> way
>> > > to
>> > > >> >> give
>> > > >> >>>>>>>> the
>> > > >> >>>>>>>>>>> the state store the opportunity to decide whether to
>> turn on
>> > > >> >>>>>>>>>>> transactions or not?
>> > > >> >>>>>>>>>>> With my comment, I was more concerned about how do you
>> know
>> > > >> if a
>> > > >> >>>>>>>>>>> checkpoint file needs to be written under EOS, if you
>> do not
>> > > >> >> have a
>> > > >> >>>>>>>> way
>> > > >> >>>>>>>>>>> to know if the state store is transactional or not. If
>> a
>> > > state
>> > > >> >>>> store
>> > > >> >>>>>>>> is
>> > > >> >>>>>>>>>>> transactional, the checkpoint file can be written
>> during
>> > > >> normal
>> > > >> >>>>>>>>>>> processing under EOS. If the state store is not
>> > > transactional,
>> > > >> >> the
>> > > >> >>>>>>>>>>> checkpoint file must not be written under EOS.
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>> 7.
>> > > >> >>>>>>>>>>> My point was about not only considering the bytes in
>> memory
>> > > in
>> > > >> >>>> config
>> > > >> >>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that
>> might
>> > > be
>> > > >> >>>>>>>> spilled
>> > > >> >>>>>>>>>>> on disk. Basically, I was wondering whether you should
>> > > remove
>> > > >> the
>> > > >> >>>>>>>>>>> "memory" in "Maximum number of memory bytes to be used
>> to
>> > > >> >>>>>>>>>>> buffer uncommitted state-store records." My thinking
>> was
>> > > that
>> > > >> >> even
>> > > >> >>>>>>>> if a
>> > > >> >>>>>>>>>>> state store spills uncommitted bytes to disk, limiting
>> the
>> > > >> >> overall
>> > > >> >>>>>>>> bytes
>> > > >> >>>>>>>>>>> might make sense. Thinking about it again and
>> considering
>> > > the
>> > > >> >>>> recent
>> > > >> >>>>>>>>>>> discussions, it does not make too much sense anymore.
>> > > >> >>>>>>>>>>> I like the name
>> statestore.transaction.buffer.max.bytes that
>> > > >> you
>> > > >> >>>>>>>> proposed.
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>> 8.
>> > > >> >>>>>>>>>>> A high-level description (without implementation
>> details) of
>> > > >> how
>> > > >> >>>>>>>> Kafka
>> > > >> >>>>>>>>>>> Streams will manage the commit of changelog
>> transactions,
>> > > >> state
>> > > >> >>>> store
>> > > >> >>>>>>>>>>> transactions and checkpointing would be great. Would be
>> > > great
>> > > >> if
>> > > >> >>>> you
>> > > >> >>>>>>>>>>> could also add some sentences about the behavior in
>> case of
>> > > a
>> > > >> >>>>>>>> failure.
>> > > >> >>>>>>>>>>> For instance how does a transactional state store
>> recover
>> > > >> after a
>> > > >> >>>>>>>>>>> failure or what happens with the transaction buffer,
>> etc.
>> > > >> (that
>> > > >> >> is
>> > > >> >>>>>>>> what
>> > > >> >>>>>>>>>>> I meant by "fail-over" in point 9.)
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>> Best,
>> > > >> >>>>>>>>>>> Bruno
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
>> > > >> >>>>>>>>>>>> Hi Bruno,
>> > > >> >>>>>>>>>>>>
>> > > >> >>>>>>>>>>>> 1.
>> > > >> >>>>>>>>>>>> Isn't this exactly the same issue that
>> WriteBatchWithIndex
>> > > >> >>>>>>>> transactions
>> > > >> >>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
>> configured
>> > > >> memory
>> > > >> >>>>>>>> needs to
>> > > >> >>>>>>>>>>>> trigger an early commit?
>> > > >> >>>>>>>>>>>>
>> > > >> >>>>>>>>>>>> 2.
>> > > >> >>>>>>>>>>>> This is one of my big concerns. Ultimately, any
>> approach
>> > > >> based
>> > > >> >> on
>> > > >> >>>>>>>>>>> cracking
>> > > >> >>>>>>>>>>>> open RocksDB internals and using it in ways it's not
>> really
>> > > >> >>>> designed
>> > > >> >>>>>>>>>>> for is
>> > > >> >>>>>>>>>>>> likely to have some unforseen performance or
>> consistency
>> > > >> issues.
>> > > >> >>>>>>>>>>>>
>> > > >> >>>>>>>>>>>> 3.
>> > > >> >>>>>>>>>>>> What's your motivation for removing these early
>> commits?
>> > > >> While
>> > > >> >> not
>> > > >> >>>>>>>>>>> ideal, I
>> > > >> >>>>>>>>>>>> think they're a decent compromise to ensure
>> consistency
>> > > >> whilst
>> > > >> >>>>>>>>>>> maintaining
>> > > >> >>>>>>>>>>>> good and predictable performance.
>> > > >> >>>>>>>>>>>> All 3 of your suggested ideas seem *very*
>> complicated, and
>> > > >> might
>> > > >> >>>>>>>>>>> actually
>> > > >> >>>>>>>>>>>> make behaviour less predictable for users as a
>> consequence.
>> > > >> >>>>>>>>>>>>
>> > > >> >>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is
>> growing a
>> > > >> bit
>> > > >> >>>> out
>> > > >> >>>>>>>> of
>> > > >> >>>>>>>>>>>> control. While it's good to discuss ideas for future
>> > > >> >>>> improvements, I
>> > > >> >>>>>>>>>>> think
>> > > >> >>>>>>>>>>>> it's important to narrow the scope down to a design
>> that
>> > > >> >> achieves
>> > > >> >>>>>>>> the
>> > > >> >>>>>>>>>>> most
>> > > >> >>>>>>>>>>>> pressing objectives (constant sized restorations
>> during
>> > > dirty
>> > > >> >>>>>>>>>>>> close/unexpected errors). Any design that this KIP
>> produces
>> > > >> can
>> > > >> >>>>>>>>>>> ultimately
>> > > >> >>>>>>>>>>>> be changed in the future, especially if the bulk of
>> it is
>> > > >> >> internal
>> > > >> >>>>>>>>>>>> behaviour.
>> > > >> >>>>>>>>>>>>
>> > > >> >>>>>>>>>>>> I'm going to spend some time next week trying to
>> re-work
>> > > the
>> > > >> >>>>>>>> original
>> > > >> >>>>>>>>>>>> WriteBatchWithIndex design to remove the
>> newTransaction()
>> > > >> >> method,
>> > > >> >>>>>>>> such
>> > > >> >>>>>>>>>>> that
>> > > >> >>>>>>>>>>>> it's just an implementation detail of RocksDBStore.
>> That
>> > > >> way, if
>> > > >> >>>> we
>> > > >> >>>>>>>>>>> want to
>> > > >> >>>>>>>>>>>> replace WBWI with something in the future, like the
>> SST
>> > > file
>> > > >> >>>>>>>> management
>> > > >> >>>>>>>>>>>> outlined by John, then we can do so with little/no API
>> > > >> changes.
>> > > >> >>>>>>>>>>>>
>> > > >> >>>>>>>>>>>> Regards,
>> > > >> >>>>>>>>>>>>
>> > > >> >>>>>>>>>>>> Nick
>> > > >> >>>>>>>>>>>>
>> > > >> >>>>>>>>>>>
>> > > >> >>>>>>>>>>
>> > > >> >>>>>>>>>
>> > > >> >>>>>>>>
>> > > >> >>>>>>>
>> > > >> >>>>>
>> > > >> >>>>
>> > > >> >>>
>> > > >> >>
>> > > >> >
>> > > >>
>> > > >
>> > >
>>
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi everyone,

I agree that having IsolationLevel be determined at query-time is the ideal
design, but there are a few sticking points:

1.
There needs to be some way to communicate the IsolationLevel down to the
RocksDBStore itself, so that the query can respect it. Since stores are
"layered" in functionality (i.e. ChangeLoggingStore, MeteredStore, etc.),
we need some way to deliver that information to the bottom layer. For IQv2,
we can use the existing State#query() method, but IQv1 has no way to do
this.

A simple approach, which would potentially open up other options, would be
to add something like: ReadOnlyKeyValueStore<K, V>
readOnlyView(IsolationLevel isolationLevel) to ReadOnlyKeyValueStore (and
similar to ReadOnlyWindowStore, ReadOnlySessionStore, etc.).

2.
As mentioned above, RocksDB WriteBatches are not thread-safe, which causes
a problem if we want to provide READ_UNCOMMITTED Iterators. I also had a
look at RocksDB Transactions[1], but they solve a very different problem,
and have the same thread-safety issue.

One possible approach that I mentioned is chaining WriteBatches: every time
a new Interactive Query is received (i.e. readOnlyView, see above, is
called) we "freeze" the existing WriteBatch, and start a new one for new
writes. The Interactive Query queries the "chain" of previous WriteBatches
+ the underlying database; while the StreamThread starts writing to the
*new* WriteBatch. On-commit, the StreamThread would write *all*
WriteBatches in the chain to the database (that have not yet been written).

WriteBatches would be closed/freed only when they have been both committed,
and all open Interactive Queries on them have been closed. This would
require some reference counting.

Obviously a drawback of this approach is the potential for increased memory
usage: if an Interactive Query is long-lived, for example by doing a full
scan over a large database, or even just pausing in the middle of an
iteration, then the existing chain of WriteBatches could be kept around for
a long time, potentially forever.

--

A.
Going off on a tangent, it looks like in addition to supporting
READ_COMMITTED queries, we could go further and support REPEATABLE_READ
queries (i.e. where subsequent reads to the same key in the same
Interactive Query are guaranteed to yield the same value) by making use of
RocksDB Snapshots[2]. These are fairly lightweight, so the performance
impact is likely to be negligible, but they do require that the Interactive
Query session can be explicitly closed.

This could be achieved if we made the above readOnlyView interface look
more like:

interface ReadOnlyKeyValueView<K, V> implements ReadOnlyKeyValueStore<K,
V>, AutoCloseable {}

interface ReadOnlyKeyValueStore<K, V> {
    ...
    ReadOnlyKeyValueView<K, V> readOnlyView(IsolationLevel isolationLevel);
}

But this would be a breaking change, as existing IQv1 queries are
guaranteed to never call store.close(), and therefore these would leak
memory under REPEATABLE_READ.

B.
One thing that's notable: MyRocks states that they support READ_COMMITTED
and REPEATABLE_READ, but they make no mention of READ_UNCOMMITTED[3][4].
This could be because doing so is technically difficult/impossible using
the primitives available in RocksDB.

--

Lucas, to address your points:

U1.
It's only "SHOULD" to permit alternative (i.e. non-RocksDB) implementations
of StateStore that do not support atomic writes. Obviously in those cases,
the guarantees Kafka Streams provides/expects would be relaxed. Do you
think we should require all implementations to support atomic writes?

U2.
Stores can support multiple IsolationLevels. As we've discussed above, the
ideal scenario would be to specify the IsolationLevel at query-time.
Failing that, I think the second-best approach is to define the
IsolationLevel for *all* queries based on the processing.mode, which is
what the default StateStoreContext#isolationLevel() achieves. Would you
prefer an alternative?

While the existing implementation is equivalent to READ_UNCOMMITTED, this
can yield unexpected results/errors under EOS, if a transaction is rolled
back. While this would be a change in behaviour for users, it would look
more like a bug fix than a breaking change. That said, we *could* make it
configurable, and default to the existing behaviour (READ_UNCOMMITTED)
instead of inferring it from the processing.mode?

N1, N2.
These were only primitives to avoid boxing costs, but since this is not a
performance sensitive area, it should be fine to change if that's desirable.

N3.
It's because the store "manages its own offsets", which includes both
committing the offset, *and providing it* via getCommittedOffset().
Personally, I think "managesOffsets" conveys this best, but I don't mind
changing it if the nomenclature is unclear.

Sorry for the massive emails/essays!
--
Nick

1: https://github.com/facebook/rocksdb/wiki/Transactions
2: https://github.com/facebook/rocksdb/wiki/Snapshot
3: https://github.com/facebook/mysql-5.6/wiki/Transaction-Isolation
4: https://mariadb.com/kb/en/myrocks-transactional-isolation/

On Mon, 18 Sept 2023 at 16:19, Lucas Brutschy
<lb...@confluent.io.invalid> wrote:

> Hi Nick,
>
> since I last read it in April, the KIP has become much cleaner and
> easier to read. Great work!
>
> It feels to me the last big open point is whether we can implement
> isolation level as a query parameter. I understand that there are
> implementation concerns, but as Colt says, it would be a great
> addition, and would also simplify the migration path for this change.
> Is the implementation problem you mentioned caused by the WriteBatch
> not having a notion of a snapshot, as the underlying DB iterator does?
> In that case, I am not sure a chain of WriteBatches as you propose
> would fully solve the problem, but maybe I didn't dig enough into the
> details to fully understand it.
>
> If it's not possible to implement it now, would it be an option to
> make sure in this KIP that we do not fully close the door on per-query
> isolation levels in the interface, as it may be possible to implement
> the missing primitives in RocksDB or Speedb in the future.
>
> Understanding:
>
> * U1) Why is it only "SHOULD" for changelogOffsets to be persisted
> atomically with the records?
> * U2) Don't understand the default implementation of `isolationLevel`.
> The isolation level should be a property of the underlying store, and
> not be defined by the default config? Existing stores probably don't
> guarantee READ_COMMITTED, so the default should be to return
> READ_UNCOMMITTED.
>
> Nits:
> * N1) Could `getComittedOffset` use an `OptionalLong` return type, to
> avoid the `null`?
> * N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
> return type, to avoid the `-1`?
> * N3) I don't understand why `managesOffsets` uses the 'manage' verb,
> whereas all other methods use the "commits" verb. I'd suggest
> `commitsOffsets`.
>
> Either way, it feels this KIP is very close to the finish line, I'm
> looking forward to seeing this in production!
>
> Cheers,
> Lucas
>
> On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <co...@littlehorse.io> wrote:
> >
> > > Making IsolationLevel a query-time constraint, rather than linking it
> to
> > the processing.guarantee.
> >
> > As I understand it, would this allow even a user of EOS to control
> whether
> > reading committed or uncommitted records? If so, I am highly in favor of
> > this.
> >
> > I know that I was one of the early people to point out the current
> > shortcoming that IQ reads uncommitted records, but just this morning I
> > realized a pattern we use which means that (for certain queries) our
> system
> > needs to be able to read uncommitted records, which is the current
> behavior
> > of Kafka Streams in EOS.***
> >
> > If IsolationLevel being a query-time decision allows for this, then that
> > would be amazing. I would also vote that the default behavior should be
> for
> > reading uncommitted records, because it is totally possible for a valid
> > application to depend on that behavior, and breaking it in a minor
> release
> > might be a bit strong.
> >
> > *** (Note, for the curious reader....) Our use-case/query pattern is a
> bit
> > complex, but reading "uncommitted" records is actually safe in our case
> > 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 soon as the processor processes the result. If they
> > can't read uncommitted records, they may receive a "201 - Created"
> > response, immediately followed by a "404 - 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 <ni...@gmail.com>
> wrote:
> >
> > > Addendum:
> > >
> > > I think we would also face the same problem with the approach John
> outlined
> > > earlier (using the record cache as a transaction buffer and flushing it
> > > straight to SST files). This is because the record cache (the
> ThreadCache
> > > class) is not thread-safe, so every commit would invalidate open IQ
> > > Iterators in the same way that RocksDB WriteBatches do.
> > > --
> > > Nick
> > >
> > > On Wed, 13 Sept 2023 at 16:58, Nick Telford <ni...@gmail.com>
> > > 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.
> > > > 2. Making IsolationLevel a query-time constraint, rather than
> linking it
> > > > to the processing.guarantee.
> > > >
> > > > There's a wrinkle that makes this a challenge: Interactive Queries
> that
> > > > open an Iterator, when using transactions and READ_UNCOMMITTED.
> > > > The problem is that under READ_UNCOMMITTED, queries need to be able
> to
> > > > read records from the currently uncommitted transaction buffer
> > > > (WriteBatch). This includes for Iterators, which should iterate both
> the
> > > > transaction buffer and underlying database (using
> > > > WriteBatch#iteratorWithBase()).
> > > >
> > > > The issue is that when the StreamThread commits, it writes the
> current
> > > > WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing the
> > > > WriteBatch while an Interactive Query holds an open Iterator on it
> will
> > > > invalidate the Iterator. Worse, it turns out that Iterators over a
> > > > WriteBatch become invalidated not just when the WriteBatch is
> cleared,
> > > but
> > > > also when the Iterators' current key receives a new write.
> > > >
> > > > Now that I'm writing this, I remember that this is the major reason
> that
> > > I
> > > > switched the original design from having a query-time IsolationLevel
> to
> > > > having the IsolationLevel linked to the transactionality of the
> stores
> > > > themselves.
> > > >
> > > > It *might* be possible to resolve this, by having a "chain" of
> > > > WriteBatches, with the StreamThread switching to a new WriteBatch
> > > whenever
> > > > a new Interactive Query attempts to read from the database, but that
> > > could
> > > > cause some performance problems/memory pressure when subjected to a
> high
> > > > Interactive Query load. It would also reduce the efficiency of
> > > WriteBatches
> > > > on-commit, as we'd have to write N WriteBatches, where N is the
> number of
> > > > Interactive Queries since the last commit.
> > > >
> > > > I realise this is getting into the weeds of the implementation, and
> you'd
> > > > rather we focus on the API for now, but I think it's important to
> > > consider
> > > > how to implement the desired API, in case we come up with an API that
> > > > cannot be implemented efficiently, or even at all!
> > > >
> > > > Thoughts?
> > > > --
> > > > Nick
> > > >
> > > > On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <ca...@apache.org>
> wrote:
> > > >
> > > >> Hi Nick,
> > > >>
> > > >> 6.
> > > >> Of course, you are right! My bad!
> > > >> Wiping out the state in the downgrading case is fine.
> > > >>
> > > >>
> > > >> 3a.
> > > >> Focus on the public facing changes for the KIP. We will manage 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.
> > > >>
> > > >>
> > > >> Best,
> > > >> Bruno
> > > >>
> > > >> On 9/13/23 11:57 AM, Nick Telford wrote:
> > > >> > Hi Bruno,
> > > >> >
> > > >> > Thanks for getting back to me!
> > > >> >
> > > >> > 2.
> > > >> > The fact that implementations can always track estimated memory
> usage
> > > in
> > > >> > the wrapper is a good point. I can remove -1 as an option, and
> I'll
> > > >> clarify
> > > >> > the JavaDoc that 0 is not just for non-transactional stores,
> which is
> > > >> > currently misleading.
> > > >> >
> > > >> > 6.
> > > >> > The problem with catching the exception in the downgrade process
> is
> > > that
> > > >> > would require new code in the Kafka version being downgraded to.
> Since
> > > >> > users could conceivably downgrade to almost *any* older version of
> > > Kafka
> > > >> > Streams, I'm not sure how we could add that code?
> > > >> > The only way I can think of doing it would be to provide a
> dedicated
> > > >> > downgrade tool, that goes through every local store and removes
> the
> > > >> > offsets column families. But that seems like an unnecessary
> amount of
> > > >> extra
> > > >> > code to maintain just to handle a somewhat niche situation, when
> the
> > > >> > alternative (automatically wipe and restore stores) should be
> > > >> acceptable.
> > > >> >
> > > >> > 1, 4, 5: Agreed. I'll make the changes you've requested.
> > > >> >
> > > >> > 3a.
> > > >> > I agree that IsolationLevel makes more sense at query-time, and I
> > > >> actually
> > > >> > initially attempted to place the IsolationLevel at query-time,
> but I
> > > ran
> > > >> > into some problems:
> > > >> > - The key issue is that, under ALOS we're not staging writes in
> > > >> > transactions, so can't perform writes at the READ_COMMITTED
> isolation
> > > >> > level. However, this may be addressed if we decide to *always* use
> > > >> > transactions as discussed under 3b.
> > > >> > - IQv1 and IQv2 have quite different implementations. I remember
> > > having
> > > >> > some difficulty understanding the IQv1 internals, which made it
> > > >> difficult
> > > >> > to determine what needed to be changed. However, I *think* this
> can be
> > > >> > addressed for both implementations by wrapping the RocksDBStore
> in an
> > > >> > IsolationLevel-dependent wrapper, that overrides read methods
> (get,
> > > >> etc.)
> > > >> > to either read directly from the database or from the ongoing
> > > >> transaction.
> > > >> > But IQv1 might still be difficult.
> > > >> > - If IsolationLevel becomes a query constraint, then all other
> > > >> StateStores
> > > >> > will need to respect it, including the in-memory stores. This
> would
> > > >> require
> > > >> > us to adapt in-memory stores to stage their writes so they can be
> > > >> isolated
> > > >> > from READ_COMMITTTED queries. It would also become an important
> > > >> > consideration for third-party stores on upgrade, as without
> changes,
> > > >> they
> > > >> > would not support READ_COMMITTED queries correctly.
> > > >> >
> > > >> > Ultimately, I may need some help making the necessary change to
> IQv1
> > > to
> > > >> > support this, but I don't think it's fundamentally impossible, if
> we
> > > >> want
> > > >> > to pursue this route.
> > > >> >
> > > >> > 3b.
> > > >> > The main reason I chose to keep ALOS un-transactional was to
> minimize
> > > >> > behavioural change for most users (I believe most Streams users
> use
> > > the
> > > >> > default configuration, which is ALOS). That said, it's clear that
> if
> > > >> ALOS
> > > >> > also used transactional stores, the only change in behaviour
> would be
> > > >> that
> > > >> > it would become *more correct*, which could be considered a "bug
> fix"
> > > by
> > > >> > users, rather than a change they need to handle.
> > > >> >
> > > >> > I believe that performance using transactions (aka. RocksDB
> > > >> WriteBatches)
> > > >> > should actually be *better* than the un-batched write-path that is
> > > >> > currently used[1]. The only "performance" consideration will be
> the
> > > >> > increased memory usage that transactions require. Given the
> > > mitigations
> > > >> for
> > > >> > this memory that we have in place, I would expect that this is
> not a
> > > >> > problem for most users.
> > > >> >
> > > >> > If we're happy to do so, we can make ALOS also use transactions.
> > > >> >
> > > >> > Regards,
> > > >> > Nick
> > > >> >
> > > >> > Link 1:
> > > >> >
> > >
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > >> >
> > > >> > On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <ca...@apache.org>
> > > >> wrote:
> > > >> >
> > > >> >> Hi Nick,
> > > >> >>
> > > >> >> Thanks for the updates and sorry for the delay on my side!
> > > >> >>
> > > >> >>
> > > >> >> 1.
> > > >> >> Making the default implementation for flush() a no-op sounds
> good to
> > > >> me.
> > > >> >>
> > > >> >>
> > > >> >> 2.
> > > >> >> I think what was bugging me here is that a third-party state
> store
> > > >> needs
> > > >> >> to implement the state store interface. That means they need to
> > > >> >> implement a wrapper around the actual state store as we do for
> > > RocksDB
> > > >> >> with RocksDBStore. So, a third-party state store can always
> estimate
> > > >> the
> > > >> >> uncommitted bytes, if it wants, because the wrapper can record
> the
> > > >> added
> > > >> >> bytes.
> > > >> >> One case I can think of where returning -1 makes sense is when
> > > Streams
> > > >> >> does not need to estimate the size of the write batch and trigger
> > > >> >> extraordinary commits, because the third-party state store takes
> care
> > > >> of
> > > >> >> memory. But in that case the method could also just return 0.
> Even
> > > that
> > > >> >> case would be better solved with a method that returns whether
> the
> > > >> state
> > > >> >> store manages itself the memory used for uncommitted bytes or
> not.
> > > >> >> Said that, I am fine with keeping the -1 return value, I was just
> > > >> >> wondering when and if it will be used.
> > > >> >>
> > > >> >> Regarding returning 0 for transactional state stores when the
> batch
> > > is
> > > >> >> empty, I was just wondering because you explicitly stated
> > > >> >>
> > > >> >> "or {@code 0} if this StateStore does not support transactions."
> > > >> >>
> > > >> >> So it seemed to me returning 0 could only happen for
> > > non-transactional
> > > >> >> state stores.
> > > >> >>
> > > >> >>
> > > >> >> 3.
> > > >> >>
> > > >> >> a) What do you think if we move the isolation level to IQ (v1 and
> > > v2)?
> > > >> >> In the end this is the only component that really needs to
> specify
> > > the
> > > >> >> isolation level. It is similar to the Kafka consumer that can
> choose
> > > >> >> with what isolation level to read the input topic.
> > > >> >> For IQv1 the isolation level should go into
> StoreQueryParameters. For
> > > >> >> IQv2, I would add it to the Query interface.
> > > >> >>
> > > >> >> b) Point a) raises the question what should happen during
> > > at-least-once
> > > >> >> processing when the state store does not use transactions? John
> in
> > > the
> > > >> >> past proposed to also use transactions on state stores for
> > > >> >> at-least-once. I like that idea, because it avoids aggregating
> the
> > > same
> > > >> >> records over and over again in the case of a failure. We had a
> case
> > > in
> > > >> >> the past where a Streams applications in at-least-once mode was
> > > failing
> > > >> >> continuously for some reasons I do not remember before
> committing the
> > > >> >> offsets. After each failover, the app aggregated again and again
> the
> > > >> >> same records. Of course the aggregate increased to very wrong
> values
> > > >> >> just because of the failover. With transactions on the state
> stores
> > > we
> > > >> >> could have avoided this. The app would have output the same
> aggregate
> > > >> >> multiple times (i.e., after each failover) but at least the
> value of
> > > >> the
> > > >> >> aggregate would not depend on the number of failovers.
> Outputting the
> > > >> >> same aggregate multiple times would be incorrect under
> exactly-once
> > > but
> > > >> >> it is OK for at-least-once.
> > > >> >> If it makes sense to add a config to turn on and off
> transactions on
> > > >> >> state stores under at-least-once or just use transactions in any
> case
> > > >> is
> > > >> >> a question we should also discuss in this KIP. It depends a bit
> on
> > > the
> > > >> >> performance trade-off. Maybe to be safe, I would add a config.
> > > >> >>
> > > >> >>
> > > >> >> 4.
> > > >> >> Your points are all valid. I tend to say to keep the metrics
> around
> > > >> >> flush() until we remove flush() completely from the interface.
> Calls
> > > to
> > > >> >> flush() might still exist since existing processors might still
> call
> > > >> >> flush() explicitly as you mentioned in 1). For sure, we need to
> > > >> document
> > > >> >> how the metrics change due to the transactions in the upgrade
> notes.
> > > >> >>
> > > >> >>
> > > >> >> 5.
> > > >> >> I see. Then you should describe how the .position files are
> handled
> > > in
> > > >> >> a dedicated section of the KIP or incorporate the description in
> the
> > > >> >> "Atomic Checkpointing" section instead of only mentioning it in
> the
> > > >> >> "Compatibility, 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:
> > > >> >>
> > > >> >>
> > > >> >>
> > > >>
> > >
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > > >> >>
> > > >> >> It is a bit brittle, but it works.
> > > >> >>
> > > >> >>
> > > >> >> Best,
> > > >> >> Bruno
> > > >> >>
> > > >> >>
> > > >> >> On 8/24/23 12:18 PM, Nick Telford wrote:
> > > >> >>> Hi Bruno,
> > > >> >>>
> > > >> >>> Thanks for taking the time to review the KIP. I'm back from
> leave
> > > now
> > > >> and
> > > >> >>> intend to move this forwards as quickly as I can.
> > > >> >>>
> > > >> >>> Addressing your points:
> > > >> >>>
> > > >> >>> 1.
> > > >> >>> Because flush() is part of the StateStore API, it's exposed to
> > > custom
> > > >> >>> Processors, which might be making calls to flush(). This was
> > > actually
> > > >> the
> > > >> >>> case in a few integration tests.
> > > >> >>> To maintain as much compatibility as possible, I'd prefer not to
> > > make
> > > >> >> this
> > > >> >>> an UnsupportedOperationException, as it will cause previously
> > > working
> > > >> >>> Processors to start throwing exceptions at runtime.
> > > >> >>> I agree that it doesn't make sense for it to proxy commit(),
> though,
> > > >> as
> > > >> >>> that would cause it to violate the "StateStores commit only
> when the
> > > >> Task
> > > >> >>> commits" rule.
> > > >> >>> Instead, I think we should make this a no-op. That way, existing
> > > user
> > > >> >>> Processors will continue to work as-before, without violation of
> > > store
> > > >> >>> consistency that would be caused by premature flush/commit of
> > > >> StateStore
> > > >> >>> data to disk.
> > > >> >>> What do you think?
> > > >> >>>
> > > >> >>> 2.
> > > >> >>> As stated in the JavaDoc, when a StateStore implementation is
> > > >> >>> transactional, but is unable to estimate the uncommitted memory
> > > usage,
> > > >> >> the
> > > >> >>> method will return -1.
> > > >> >>> The intention here is to permit third-party implementations
> that may
> > > >> not
> > > >> >> be
> > > >> >>> able to estimate memory usage.
> > > >> >>>
> > > >> >>> Yes, it will be 0 when nothing has been written to the store
> yet. I
> > > >> >> thought
> > > >> >>> that was implied by "This method will return an approximation
> of the
> > > >> >> memory
> > > >> >>> would be freed by the next call to {@link #commit(Map)}" and
> > > "@return
> > > >> The
> > > >> >>> approximate size of all records awaiting {@link #commit(Map)}",
> > > >> however,
> > > >> >> I
> > > >> >>> can add it explicitly to the JavaDoc if you think this is
> unclear?
> > > >> >>>
> > > >> >>> 3.
> > > >> >>> I realise this is probably the most contentious point in my
> design,
> > > >> and
> > > >> >> I'm
> > > >> >>> open to changing it if I'm unable to convince you of the
> benefits.
> > > >> >>> Nevertheless, here's my argument:
> > > >> >>> The Interactive Query (IQ) API(s) are directly provided
> StateStores
> > > to
> > > >> >>> query, and it may be important for users to programmatically
> know
> > > >> which
> > > >> >>> mode the StateStore is operating under. If we simply provide an
> > > >> >>> "eosEnabled" boolean (as used throughout the internal streams
> > > >> engine), or
> > > >> >>> similar, then users will need to understand the operation and
> > > >> >> consequences
> > > >> >>> of each available processing mode and how it pertains to their
> > > >> >> StateStore.
> > > >> >>>
> > > >> >>> Interactive Query users aren't the only people that care about
> the
> > > >> >>> processing.mode/IsolationLevel of a StateStore: implementers of
> > > custom
> > > >> >>> StateStores also need to understand the behaviour expected of
> their
> > > >> >>> implementation. KIP-892 introduces some assumptions into the
> Streams
> > > >> >> Engine
> > > >> >>> about how StateStores operate under each processing mode, and
> it's
> > > >> >>> important that custom implementations adhere to those
> assumptions in
> > > >> >> order
> > > >> >>> to maintain the consistency guarantees.
> > > >> >>>
> > > >> >>> IsolationLevels provide a high-level contract on the behaviour
> of
> > > the
> > > >> >>> StateStore: a user knows that under READ_COMMITTED, they will
> see
> > > >> writes
> > > >> >>> only after the Task has committed, and under READ_UNCOMMITTED
> they
> > > >> will
> > > >> >> see
> > > >> >>> writes immediately. No understanding of the details of each
> > > >> >> processing.mode
> > > >> >>> is required, either for IQ users or StateStore implementers.
> > > >> >>>
> > > >> >>> An argument can be made that these contractual guarantees can
> simply
> > > >> be
> > > >> >>> documented for the processing.mode (i.e. that exactly-once and
> > > >> >>> exactly-once-v2 behave like READ_COMMITTED and at-least-once
> behaves
> > > >> like
> > > >> >>> READ_UNCOMMITTED), but there are several small issues with this
> I'd
> > > >> >> prefer
> > > >> >>> to avoid:
> > > >> >>>
> > > >> >>>      - Where would we document these contracts, in a way that is
> > > >> difficult
> > > >> >>>      for users/implementers to miss/ignore?
> > > >> >>>      - It's not clear to users that the processing mode is
> > > >> communicating
> > > >> >>>      an expectation of read isolation, unless they read the
> > > >> >> documentation. Users
> > > >> >>>      rarely consult documentation unless they feel they need
> to, so
> > > >> it's
> > > >> >> likely
> > > >> >>>      this detail would get missed by many users.
> > > >> >>>      - It tightly couples processing modes to read isolation.
> Adding
> > > >> new
> > > >> >>>      processing modes, or changing the read isolation of
> existing
> > > >> >> processing
> > > >> >>>      modes would be difficult/impossible.
> > > >> >>>
> > > >> >>> Ultimately, the cost of introducing IsolationLevels is just a
> single
> > > >> >>> method, since we re-use the existing IsolationLevel enum from
> Kafka.
> > > >> This
> > > >> >>> gives us a clear place to document the contractual guarantees
> > > expected
> > > >> >>> of/provided by StateStores, that is accessible both by the
> > > StateStore
> > > >> >>> itself, and by IQ users.
> > > >> >>>
> > > >> >>> (Writing this I've just realised that the StateStore and IQ APIs
> > > >> actually
> > > >> >>> don't provide access to StateStoreContext that IQ users would
> have
> > > >> direct
> > > >> >>> access to... Perhaps StateStore should expose isolationLevel()
> > > itself
> > > >> >> too?)
> > > >> >>>
> > > >> >>> 4.
> > > >> >>> Yeah, I'm not comfortable renaming the metrics in-place either,
> as
> > > >> it's a
> > > >> >>> backwards incompatible change. My concern is that, if we leave
> the
> > > >> >> existing
> > > >> >>> "flush" metrics in place, they will be confusing to users. Right
> > > now,
> > > >> >>> "flush" metrics record explicit flushes to disk, but under
> KIP-892,
> > > >> even
> > > >> >> a
> > > >> >>> commit() will not explicitly flush data to disk - RocksDB will
> > > decide
> > > >> on
> > > >> >>> when to flush memtables to disk itself.
> > > >> >>>
> > > >> >>> If we keep the existing "flush" metrics, we'd have two options,
> > > which
> > > >> >> both
> > > >> >>> seem pretty bad to me:
> > > >> >>>
> > > >> >>>      1. Have them record calls to commit(), which would be
> > > >> misleading, as
> > > >> >>>      data is no longer explicitly "flushed" to disk by this
> call.
> > > >> >>>      2. Have them record nothing at all, which is equivalent to
> > > >> removing
> > > >> >> the
> > > >> >>>      metrics, except that users will see the metric still
> exists and
> > > >> so
> > > >> >> assume
> > > >> >>>      that the metric is correct, and that there's a problem with
> > > their
> > > >> >> system
> > > >> >>>      when there isn't.
> > > >> >>>
> > > >> >>> I agree that removing them is also a bad solution, and I'd like
> some
> > > >> >>> guidance on the best path forward here.
> > > >> >>>
> > > >> >>> 5.
> > > >> >>> Position files are updated on every write to a StateStore.
> Since our
> > > >> >> writes
> > > >> >>> are now buffered until commit(), we can't update the Position
> file
> > > >> until
> > > >> >>> commit() has been called, otherwise it would be inconsistent
> with
> > > the
> > > >> >> data
> > > >> >>> in the event of a rollback. Consequently, we need to manage
> these
> > > >> offsets
> > > >> >>> the same way we manage the checkpoint offsets, and ensure
> they're
> > > only
> > > >> >>> written on commit().
> > > >> >>>
> > > >> >>> 6.
> > > >> >>> Agreed, although I'm not exactly sure yet what tests to write.
> How
> > > >> >> explicit
> > > >> >>> do we need to be here in the KIP?
> > > >> >>>
> > > >> >>> As for upgrade/downgrade: upgrade is designed to be seamless,
> and we
> > > >> >> should
> > > >> >>> definitely add some tests around that. Downgrade, it transpires,
> > > isn't
> > > >> >>> currently possible, as the extra column family for offset
> storage is
> > > >> >>> incompatible with the pre-KIP-892 implementation: when you open
> a
> > > >> RocksDB
> > > >> >>> database, you must open all available column families or
> receive an
> > > >> >> error.
> > > >> >>> What currently happens on downgrade is that it attempts to open
> the
> > > >> >> store,
> > > >> >>> throws an error about the offsets column family not being
> opened,
> > > >> which
> > > >> >>> triggers a wipe and rebuild of the Task. Given that downgrades
> > > should
> > > >> be
> > > >> >>> uncommon, I think this is acceptable behaviour, as the
> end-state is
> > > >> >>> consistent, even if it results in an undesirable state restore.
> > > >> >>>
> > > >> >>> Should I document the upgrade/downgrade behaviour explicitly in
> the
> > > >> KIP?
> > > >> >>>
> > > >> >>> --
> > > >> >>>
> > > >> >>> Regards,
> > > >> >>> Nick
> > > >> >>>
> > > >> >>>
> > > >> >>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <cadonna@apache.org
> >
> > > >> wrote:
> > > >> >>>
> > > >> >>>> Hi Nick!
> > > >> >>>>
> > > >> >>>> Thanks for the updates!
> > > >> >>>>
> > > >> >>>> 1.
> > > >> >>>> Why does StateStore#flush() default to
> > > >> >>>> StateStore#commit(Collections.emptyMap())?
> > > >> >>>> Since calls to flush() will not exist anymore after this KIP is
> > > >> >>>> released, I would rather throw an unsupported operation
> exception
> > > by
> > > >> >>>> default.
> > > >> >>>>
> > > >> >>>>
> > > >> >>>> 2.
> > > >> >>>> When would a state store return -1 from
> > > >> >>>> StateStore#approximateNumUncommittedBytes() while being
> > > >> transactional?
> > > >> >>>>
> > > >> >>>> Wouldn't StateStore#approximateNumUncommittedBytes() also
> return 0
> > > if
> > > >> >>>> the state store is transactional but nothing has been written
> to
> > > the
> > > >> >>>> state store yet?
> > > >> >>>>
> > > >> >>>>
> > > >> >>>> 3.
> > > >> >>>> Sorry for bringing this up again. Does this KIP really need to
> > > >> introduce
> > > >> >>>> StateStoreContext#isolationLevel()? StateStoreContext has
> already
> > > >> >>>> appConfigs() which basically exposes the same information,
> i.e., if
> > > >> EOS
> > > >> >>>> is enabled or not.
> > > >> >>>> In one of your previous e-mails you wrote:
> > > >> >>>>
> > > >> >>>> "My idea was to try to keep the StateStore interface as loosely
> > > >> coupled
> > > >> >>>> from the Streams engine as possible, to give implementers more
> > > >> freedom,
> > > >> >>>> and reduce the amount of internal knowledge required."
> > > >> >>>>
> > > >> >>>> While I understand the intent, I doubt that it decreases the
> > > >> coupling of
> > > >> >>>> a StateStore interface and the Streams engine. READ_COMMITTED
> only
> > > >> >>>> applies to IQ but not to reads by processors. Thus,
> implementers
> > > >> need to
> > > >> >>>> understand how Streams accesses the state stores.
> > > >> >>>>
> > > >> >>>> I would like to hear what others think about this.
> > > >> >>>>
> > > >> >>>>
> > > >> >>>> 4.
> > > >> >>>> Great exposing new metrics for transactional state stores!
> > > However, I
> > > >> >>>> would prefer to add new metrics and deprecate (in the docs)
> the 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 also need to adapt/add integration tests
> besides
> > > unit
> > > >> >>>> tests. Additionally, we probably need integration or system
> tests
> > > to
> > > >> >>>> verify that upgrades and downgrades between transactional and
> > > >> >>>> non-transactional state stores work as expected.
> > > >> >>>>
> > > >> >>>>
> > > >> >>>> Best,
> > > >> >>>> Bruno
> > > >> >>>>
> > > >> >>>>
> > > >> >>>>
> > > >> >>>>
> > > >> >>>>
> > > >> >>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > > >> >>>>> One more thing: I noted John's suggestion in the KIP, under
> > > >> "Rejected
> > > >> >>>>> Alternatives". I still think it's an idea worth pursuing, but
> I
> > > >> believe
> > > >> >>>>> that it's out of the scope of this KIP, because it solves a
> > > >> different
> > > >> >> set
> > > >> >>>>> of problems to this KIP, and the scope of this one has already
> > > grown
> > > >> >>>> quite
> > > >> >>>>> large!
> > > >> >>>>>
> > > >> >>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > > nick.telford@gmail.com>
> > > >> >>>> wrote:
> > > >> >>>>>
> > > >> >>>>>> Hi everyone,
> > > >> >>>>>>
> > > >> >>>>>> I've updated the KIP (
> > > >> >>>>>>
> > > >> >>>>
> > > >> >>
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > >> >>>> )
> > > >> >>>>>> with the latest changes; mostly bringing back "Atomic
> > > >> Checkpointing"
> > > >> >>>> (for
> > > >> >>>>>> what feels like the 10th time!). I think the one thing
> missing is
> > > >> some
> > > >> >>>>>> changes to metrics (notably the store "flush" metrics will
> need
> > > to
> > > >> be
> > > >> >>>>>> renamed to "commit").
> > > >> >>>>>>
> > > >> >>>>>> The reason I brought back Atomic Checkpointing was to
> decouple
> > > >> store
> > > >> >>>> flush
> > > >> >>>>>> from store commit. This is important, because with
> Transactional
> > > >> >>>>>> StateStores, we now need to call "flush" on *every* Task
> commit,
> > > >> and
> > > >> >> not
> > > >> >>>>>> just when the StateStore is closing, otherwise our
> transaction
> > > >> buffer
> > > >> >>>> will
> > > >> >>>>>> never be written and persisted, instead growing unbounded! I
> > > >> >>>> experimented
> > > >> >>>>>> with some simple solutions, like forcing a store flush
> whenever
> > > the
> > > >> >>>>>> transaction buffer was likely to exceed its configured size,
> but
> > > >> this
> > > >> >>>> was
> > > >> >>>>>> brittle: it prevented the transaction buffer from being
> > > configured
> > > >> to
> > > >> >> be
> > > >> >>>>>> unbounded, and it still would have required explicit flushes
> of
> > > >> >> RocksDB,
> > > >> >>>>>> yielding sub-optimal performance and memory utilization.
> > > >> >>>>>>
> > > >> >>>>>> I deemed Atomic Checkpointing to be the "right" way to
> resolve
> > > this
> > > >> >>>>>> problem. By ensuring that the changelog offsets that
> correspond
> > > to
> > > >> the
> > > >> >>>> most
> > > >> >>>>>> recently written records are always atomically written to the
> > > >> >> StateStore
> > > >> >>>>>> (by writing them to the same transaction buffer), we can
> avoid
> > > >> >> forcibly
> > > >> >>>>>> flushing the RocksDB memtables to disk, letting RocksDB flush
> > > them
> > > >> >> only
> > > >> >>>>>> when necessary, without losing any of our consistency
> guarantees.
> > > >> See
> > > >> >>>> the
> > > >> >>>>>> updated KIP for more info.
> > > >> >>>>>>
> > > >> >>>>>> I have fully implemented these changes, although I'm still
> not
> > > >> >> entirely
> > > >> >>>>>> happy with the implementation for segmented StateStores, so I
> > > plan
> > > >> to
> > > >> >>>>>> refactor that. Despite that, all tests pass. If you'd like
> to try
> > > >> out
> > > >> >> or
> > > >> >>>>>> review this highly experimental and incomplete branch, it's
> > > >> available
> > > >> >>>> here:
> > > >> >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0.
> Note:
> > > >> it's
> > > >> >>>> built
> > > >> >>>>>> against Kafka 3.5.0 so that I had a stable base to build and
> test
> > > >> it
> > > >> >> on,
> > > >> >>>>>> and to enable easy apples-to-apples comparisons in a live
> > > >> >> environment. I
> > > >> >>>>>> plan to rebase it against trunk once it's nearer completion
> and
> > > has
> > > >> >> been
> > > >> >>>>>> proven on our main application.
> > > >> >>>>>>
> > > >> >>>>>> I would really appreciate help in reviewing and testing:
> > > >> >>>>>> - Segmented (Versioned, Session and Window) stores
> > > >> >>>>>> - Global stores
> > > >> >>>>>>
> > > >> >>>>>> As I do not currently use either of these, so my primary test
> > > >> >>>> environment
> > > >> >>>>>> doesn't test these areas.
> > > >> >>>>>>
> > > >> >>>>>> I'm going on Parental Leave starting next week for a few
> weeks,
> > > so
> > > >> >> will
> > > >> >>>>>> not have time to move this forward until late August. That
> said,
> > > >> your
> > > >> >>>>>> feedback is welcome and appreciated, I just won't be able to
> > > >> respond
> > > >> >> as
> > > >> >>>>>> quickly as usual.
> > > >> >>>>>>
> > > >> >>>>>> Regards,
> > > >> >>>>>> Nick
> > > >> >>>>>>
> > > >> >>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > > nick.telford@gmail.com>
> > > >> >>>> wrote:
> > > >> >>>>>>
> > > >> >>>>>>> Hi Bruno
> > > >> >>>>>>>
> > > >> >>>>>>> Yes, that's correct, although the impact on IQ is not
> something
> > > I
> > > >> had
> > > >> >>>>>>> considered.
> > > >> >>>>>>>
> > > >> >>>>>>> What about atomically updating the state store from the
> > > >> transaction
> > > >> >>>>>>>> buffer every commit interval and writing the checkpoint
> (thus,
> > > >> >>>> flushing
> > > >> >>>>>>>> the memtable) every configured amount of data and/or
> number of
> > > >> >> commit
> > > >> >>>>>>>> intervals?
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>>> I'm not quite sure I follow. Are you suggesting that we add
> an
> > > >> >>>> additional
> > > >> >>>>>>> config for the max number of commit intervals between
> > > checkpoints?
> > > >> >> That
> > > >> >>>>>>> way, we would checkpoint *either* when the transaction
> buffers
> > > are
> > > >> >>>> nearly
> > > >> >>>>>>> full, *OR* whenever a certain number of commit intervals
> have
> > > >> >> elapsed,
> > > >> >>>>>>> whichever comes first?
> > > >> >>>>>>>
> > > >> >>>>>>> That certainly seems reasonable, although this re-ignites an
> > > >> earlier
> > > >> >>>>>>> debate about whether a config should be measured in "number
> of
> > > >> commit
> > > >> >>>>>>> intervals", instead of just an absolute time.
> > > >> >>>>>>>
> > > >> >>>>>>> FWIW, I realised that this issue is the reason I was
> pursuing
> > > the
> > > >> >>>> Atomic
> > > >> >>>>>>> Checkpoints, as it de-couples memtable flush from
> checkpointing,
> > > >> >> which
> > > >> >>>>>>> enables us to just checkpoint on every commit without any
> > > >> performance
> > > >> >>>>>>> impact. Atomic Checkpointing is definitely the "best"
> solution,
> > > >> but
> > > >> >>>> I'm not
> > > >> >>>>>>> sure if this is enough to bring it back into this KIP.
> > > >> >>>>>>>
> > > >> >>>>>>> I'm currently working on moving all the transactional logic
> > > >> directly
> > > >> >>>> into
> > > >> >>>>>>> RocksDBStore itself, which does away with the
> > > >> >> StateStore#newTransaction
> > > >> >>>>>>> method, and reduces the number of new classes introduced,
> > > >> >> significantly
> > > >> >>>>>>> reducing the complexity. If it works, and the complexity is
> > > >> >> drastically
> > > >> >>>>>>> reduced, I may try bringing back Atomic Checkpoints into
> this
> > > KIP.
> > > >> >>>>>>>
> > > >> >>>>>>> Regards,
> > > >> >>>>>>> Nick
> > > >> >>>>>>>
> > > >> >>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <
> cadonna@apache.org>
> > > >> >> wrote:
> > > >> >>>>>>>
> > > >> >>>>>>>> Hi Nick,
> > > >> >>>>>>>>
> > > >> >>>>>>>> Thanks for the insights! Very interesting!
> > > >> >>>>>>>>
> > > >> >>>>>>>> As far as I understand, you want to atomically update the
> state
> > > >> >> store
> > > >> >>>>>>>> from the transaction buffer, flush the memtable of a state
> > > store
> > > >> and
> > > >> >>>>>>>> write the checkpoint not after the commit time elapsed but
> > > after
> > > >> the
> > > >> >>>>>>>> transaction buffer reached a size that would lead to
> exceeding
> > > >> >>>>>>>> statestore.transaction.buffer.max.bytes before the next
> commit
> > > >> >>>> interval
> > > >> >>>>>>>> ends.
> > > >> >>>>>>>> That means, the Kafka transaction would commit every commit
> > > >> interval
> > > >> >>>> but
> > > >> >>>>>>>> the state store will only be atomically updated roughly
> every
> > > >> >>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ
> would
> > > >> then
> > > >> >>>> only
> > > >> >>>>>>>> see new data roughly every
> > > >> statestore.transaction.buffer.max.bytes.
> > > >> >>>>>>>> After a failure the state store needs to restore up to
> > > >> >>>>>>>> statestore.transaction.buffer.max.bytes.
> > > >> >>>>>>>>
> > > >> >>>>>>>> Is this correct?
> > > >> >>>>>>>>
> > > >> >>>>>>>> What about atomically updating the state store from the
> > > >> transaction
> > > >> >>>>>>>> buffer every commit interval and writing the checkpoint
> (thus,
> > > >> >>>> flushing
> > > >> >>>>>>>> the memtable) every configured amount of data and/or
> number of
> > > >> >> commit
> > > >> >>>>>>>> intervals? In such a way, we would have the same delay for
> > > >> records
> > > >> >>>>>>>> appearing in output topics and IQ because both would appear
> > > when
> > > >> the
> > > >> >>>>>>>> Kafka transaction is committed. However, after a failure
> the
> > > >> state
> > > >> >>>> store
> > > >> >>>>>>>> still needs to restore up to
> > > >> statestore.transaction.buffer.max.bytes
> > > >> >>>> and
> > > >> >>>>>>>> it might restore data that is already in the state store
> > > because
> > > >> the
> > > >> >>>>>>>> checkpoint lags behind the last stable offset (i.e. the
> last
> > > >> >> committed
> > > >> >>>>>>>> offset) of the changelog topics. Restoring data that is
> already
> > > >> in
> > > >> >> the
> > > >> >>>>>>>> state store is idempotent, so eos should not violated.
> > > >> >>>>>>>> This solution needs at least one new config to specify
> when a
> > > >> >>>> checkpoint
> > > >> >>>>>>>> should be written.
> > > >> >>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>> A small correction to your previous e-mail that does not
> change
> > > >> >>>> anything
> > > >> >>>>>>>> you said: Under alos the default commit interval is 30
> seconds,
> > > >> not
> > > >> >>>> five
> > > >> >>>>>>>> seconds.
> > > >> >>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>> Best,
> > > >> >>>>>>>> Bruno
> > > >> >>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > > >> >>>>>>>>> Hi everyone,
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> I've begun performance testing my branch on our staging
> > > >> >> environment,
> > > >> >>>>>>>>> putting it through its paces in our non-trivial
> application.
> > > I'm
> > > >> >>>>>>>> already
> > > >> >>>>>>>>> observing the same increased flush rate that we saw the
> last
> > > >> time
> > > >> >> we
> > > >> >>>>>>>>> attempted to use a version of this KIP, but this time, I
> > > think I
> > > >> >> know
> > > >> >>>>>>>> why.
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at
> the end
> > > >> of
> > > >> >> the
> > > >> >>>>>>>> Task
> > > >> >>>>>>>>> commit process, has the following behaviour:
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>        - Under ALOS: checkpoint the state stores. This
> > > includes
> > > >> >>>>>>>>>        flushing memtables in RocksDB. This is acceptable
> > > >> because the
> > > >> >>>>>>>> default
> > > >> >>>>>>>>>        commit.interval.ms is 5 seconds, so forcibly
> flushing
> > > >> >> memtables
> > > >> >>>>>>>> every 5
> > > >> >>>>>>>>>        seconds is acceptable for most applications.
> > > >> >>>>>>>>>        - Under EOS: checkpointing is not done, *unless*
> it's
> > > >> being
> > > >> >>>>>>>> forced, due
> > > >> >>>>>>>>>        to e.g. the Task closing or being revoked. This
> means
> > > >> that
> > > >> >> under
> > > >> >>>>>>>> normal
> > > >> >>>>>>>>>        processing conditions, the state stores will not be
> > > >> >>>> checkpointed,
> > > >> >>>>>>>> and will
> > > >> >>>>>>>>>        not have memtables flushed at all , unless RocksDB
> > > >> decides to
> > > >> >>>>>>>> flush them on
> > > >> >>>>>>>>>        its own. Checkpointing stores and force-flushing
> their
> > > >> >> memtables
> > > >> >>>>>>>> is only
> > > >> >>>>>>>>>        done when a Task is being closed.
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least
> > > *some*
> > > >> >>>> normal
> > > >> >>>>>>>>> Task commits, in order to write the RocksDB transaction
> > > buffers
> > > >> to
> > > >> >>>> the
> > > >> >>>>>>>>> state stores, and to ensure the offsets are synced to
> disk to
> > > >> >> prevent
> > > >> >>>>>>>>> restores from getting out of hand. Consequently, my
> current
> > > >> >>>>>>>> implementation
> > > >> >>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is
> far too
> > > >> >>>>>>>> frequent.
> > > >> >>>>>>>>> This causes checkpoints every 10,000 records, which is a
> > > change
> > > >> in
> > > >> >>>>>>>> flush
> > > >> >>>>>>>>> behaviour, potentially causing performance problems for
> some
> > > >> >>>>>>>> applications.
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> I'm looking into possible solutions, and I'm currently
> leaning
> > > >> >>>> towards
> > > >> >>>>>>>>> using the statestore.transaction.buffer.max.bytes
> > > configuration
> > > >> to
> > > >> >>>>>>>>> checkpoint Tasks once we are likely to exceed it. This
> would
> > > >> >>>>>>>> complement the
> > > >> >>>>>>>>> existing "early Task commit" functionality that this
> > > >> configuration
> > > >> >>>>>>>>> provides, in the following way:
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>        - Currently, we use
> > > >> statestore.transaction.buffer.max.bytes
> > > >> >> to
> > > >> >>>>>>>> force an
> > > >> >>>>>>>>>        early Task commit if processing more records would
> > > cause
> > > >> our
> > > >> >>>> state
> > > >> >>>>>>>> store
> > > >> >>>>>>>>>        transactions to exceed the memory assigned to them.
> > > >> >>>>>>>>>        - New functionality: when a Task *does* commit, we
> will
> > > >> not
> > > >> >>>>>>>> checkpoint
> > > >> >>>>>>>>>        the stores (and hence flush the transaction
> buffers)
> > > >> unless
> > > >> >> we
> > > >> >>>>>>>> expect to
> > > >> >>>>>>>>>        cross the statestore.transaction.buffer.max.bytes
> > > >> threshold
> > > >> >>>> before
> > > >> >>>>>>>> the next
> > > >> >>>>>>>>>        commit
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> I'm also open to suggestions.
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> Regards,
> > > >> >>>>>>>>> Nick
> > > >> >>>>>>>>>
> > > >> >>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> > > >> nick.telford@gmail.com
> > > >> >>>
> > > >> >>>>>>>> wrote:
> > > >> >>>>>>>>>
> > > >> >>>>>>>>>> Hi Bruno!
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 3.
> > > >> >>>>>>>>>> By "less predictable for users", I meant in terms of
> > > >> understanding
> > > >> >>>> the
> > > >> >>>>>>>>>> performance profile under various circumstances. The more
> > > >> complex
> > > >> >>>> the
> > > >> >>>>>>>>>> solution, the more difficult it would be for users to
> > > >> understand
> > > >> >> the
> > > >> >>>>>>>>>> performance they see. For example, spilling records to
> disk
> > > >> when
> > > >> >> the
> > > >> >>>>>>>>>> transaction buffer reaches a threshold would, I expect,
> > > reduce
> > > >> >> write
> > > >> >>>>>>>>>> throughput. This reduction in write throughput could be
> > > >> >> unexpected,
> > > >> >>>>>>>> and
> > > >> >>>>>>>>>> potentially difficult to diagnose/understand for users.
> > > >> >>>>>>>>>> At the moment, I think the "early commit" concept is
> > > relatively
> > > >> >>>>>>>>>> straightforward; it's easy to document, and conceptually
> > > fairly
> > > >> >>>>>>>> obvious to
> > > >> >>>>>>>>>> users. We could probably add a metric to make it easier
> to
> > > >> >>>> understand
> > > >> >>>>>>>> when
> > > >> >>>>>>>>>> it happens though.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 3. (the second one)
> > > >> >>>>>>>>>> The IsolationLevel is *essentially* an indirect way of
> > > telling
> > > >> >>>>>>>> StateStores
> > > >> >>>>>>>>>> whether they should be transactional. READ_COMMITTED
> > > >> essentially
> > > >> >>>>>>>> requires
> > > >> >>>>>>>>>> transactions, because it dictates that two threads
> calling
> > > >> >>>>>>>>>> `newTransaction()` should not see writes from the other
> > > >> >> transaction
> > > >> >>>>>>>> until
> > > >> >>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all
> bets are
> > > >> off,
> > > >> >>>> and
> > > >> >>>>>>>>>> stores can allow threads to observe written records at
> any
> > > >> time,
> > > >> >>>>>>>> which is
> > > >> >>>>>>>>>> essentially "no transactions". That said, StateStores are
> > > free
> > > >> to
> > > >> >>>>>>>> implement
> > > >> >>>>>>>>>> these guarantees however they can, which is a bit more
> > > relaxed
> > > >> >> than
> > > >> >>>>>>>>>> dictating "you must use transactions". For example, with
> > > >> RocksDB
> > > >> >> we
> > > >> >>>>>>>> would
> > > >> >>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
> > > "transactions",
> > > >> >>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But
> with
> > > >> other
> > > >> >>>>>>>> storage
> > > >> >>>>>>>>>> engines, it might be preferable to *always* use
> transactions,
> > > >> even
> > > >> >>>>>>>> when
> > > >> >>>>>>>>>> unnecessary; or there may be storage engines that don't
> > > provide
> > > >> >>>>>>>>>> transactions, but the isolation guarantees can be met
> using a
> > > >> >>>>>>>> different
> > > >> >>>>>>>>>> technique.
> > > >> >>>>>>>>>> My idea was to try to keep the StateStore interface as
> > > loosely
> > > >> >>>> coupled
> > > >> >>>>>>>>>> from the Streams engine as possible, to give implementers
> > > more
> > > >> >>>>>>>> freedom, and
> > > >> >>>>>>>>>> reduce the amount of internal knowledge required.
> > > >> >>>>>>>>>> That said, I understand that "IsolationLevel" might not
> be
> > > the
> > > >> >> right
> > > >> >>>>>>>>>> abstraction, and we can always make it much more
> explicit if
> > > >> >>>>>>>> required, e.g.
> > > >> >>>>>>>>>> boolean transactional()
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> 7-8.
> > > >> >>>>>>>>>> I can make these changes either later today or tomorrow.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Small update:
> > > >> >>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of
> issues
> > > >> that
> > > >> >>>>>>>> needed
> > > >> >>>>>>>>>> addressing. Currently, all the tests pass, which is
> > > promising,
> > > >> but
> > > >> >>>> it
> > > >> >>>>>>>> will
> > > >> >>>>>>>>>> need to undergo some performance testing. I haven't (yet)
> > > >> worked
> > > >> >> on
> > > >> >>>>>>>>>> removing the `newTransaction()` stuff, but I would expect
> > > that,
> > > >> >>>>>>>>>> behaviourally, it should make no difference. The branch
> is
> > > >> >> available
> > > >> >>>>>>>> at
> > > >> >>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if
> > > anyone
> > > >> is
> > > >> >>>>>>>>>> interested in taking an early look.
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> Regards,
> > > >> >>>>>>>>>> Nick
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> > > >> cadonna@apache.org>
> > > >> >>>>>>>> wrote:
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>>> Hi Nick,
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> 1.
> > > >> >>>>>>>>>>> Yeah, I agree with you. That was actually also my
> point. I
> > > >> >>>> understood
> > > >> >>>>>>>>>>> that John was proposing the ingestion path as a way to
> avoid
> > > >> the
> > > >> >>>>>>>> early
> > > >> >>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> 2.
> > > >> >>>>>>>>>>> I agree with John here, that actually it is public API.
> My
> > > >> >> question
> > > >> >>>>>>>> is
> > > >> >>>>>>>>>>> how this usage pattern affects normal processing.
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> 3.
> > > >> >>>>>>>>>>> My concern is that checking for the size of the
> transaction
> > > >> >> buffer
> > > >> >>>>>>>> and
> > > >> >>>>>>>>>>> maybe triggering an early commit affects the whole
> > > processing
> > > >> of
> > > >> >>>>>>>> Kafka
> > > >> >>>>>>>>>>> Streams. The transactionality of a state store is not
> > > >> confined to
> > > >> >>>> the
> > > >> >>>>>>>>>>> state store itself, but spills over and changes the
> behavior
> > > >> of
> > > >> >>>> other
> > > >> >>>>>>>>>>> parts of the system. I agree with you that it is a
> decent
> > > >> >>>>>>>> compromise. I
> > > >> >>>>>>>>>>> just wanted to analyse the downsides and list the
> options to
> > > >> >>>> overcome
> > > >> >>>>>>>>>>> them. I also agree with you that all options seem quite
> > > heavy
> > > >> >>>>>>>> compared
> > > >> >>>>>>>>>>> with your KIP. I do not understand what you mean with
> "less
> > > >> >>>>>>>> predictable
> > > >> >>>>>>>>>>> for users", though.
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> I found the discussions about the alternatives really
> > > >> >> interesting.
> > > >> >>>>>>>> But I
> > > >> >>>>>>>>>>> also think that your plan sounds good and we should
> continue
> > > >> with
> > > >> >>>> it!
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> 3.
> > > >> >>>>>>>>>>> Ah, now, I understand the reasoning behind putting
> isolation
> > > >> >> level
> > > >> >>>> in
> > > >> >>>>>>>>>>> the state store context. Thanks! Should that also be a
> way
> > > to
> > > >> >> give
> > > >> >>>>>>>> the
> > > >> >>>>>>>>>>> the state store the opportunity to decide whether to
> turn on
> > > >> >>>>>>>>>>> transactions or not?
> > > >> >>>>>>>>>>> With my comment, I was more concerned about how do you
> know
> > > >> if a
> > > >> >>>>>>>>>>> checkpoint file needs to be written under EOS, if you
> do not
> > > >> >> have a
> > > >> >>>>>>>> way
> > > >> >>>>>>>>>>> to know if the state store is transactional or not. If a
> > > state
> > > >> >>>> store
> > > >> >>>>>>>> is
> > > >> >>>>>>>>>>> transactional, the checkpoint file can be written during
> > > >> normal
> > > >> >>>>>>>>>>> processing under EOS. If the state store is not
> > > transactional,
> > > >> >> the
> > > >> >>>>>>>>>>> checkpoint file must not be written under EOS.
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> 7.
> > > >> >>>>>>>>>>> My point was about not only considering the bytes in
> memory
> > > in
> > > >> >>>> config
> > > >> >>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that
> might
> > > be
> > > >> >>>>>>>> spilled
> > > >> >>>>>>>>>>> on disk. Basically, I was wondering whether you should
> > > remove
> > > >> the
> > > >> >>>>>>>>>>> "memory" in "Maximum number of memory bytes to be used
> to
> > > >> >>>>>>>>>>> buffer uncommitted state-store records." My thinking was
> > > that
> > > >> >> even
> > > >> >>>>>>>> if a
> > > >> >>>>>>>>>>> state store spills uncommitted bytes to disk, limiting
> the
> > > >> >> overall
> > > >> >>>>>>>> bytes
> > > >> >>>>>>>>>>> might make sense. Thinking about it again and
> considering
> > > the
> > > >> >>>> recent
> > > >> >>>>>>>>>>> discussions, it does not make too much sense anymore.
> > > >> >>>>>>>>>>> I like the name statestore.transaction.buffer.max.bytes
> that
> > > >> you
> > > >> >>>>>>>> proposed.
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> 8.
> > > >> >>>>>>>>>>> A high-level description (without implementation
> details) of
> > > >> how
> > > >> >>>>>>>> Kafka
> > > >> >>>>>>>>>>> Streams will manage the commit of changelog
> transactions,
> > > >> state
> > > >> >>>> store
> > > >> >>>>>>>>>>> transactions and checkpointing would be great. Would be
> > > great
> > > >> if
> > > >> >>>> you
> > > >> >>>>>>>>>>> could also add some sentences about the behavior in
> case of
> > > a
> > > >> >>>>>>>> failure.
> > > >> >>>>>>>>>>> For instance how does a transactional state store
> recover
> > > >> after a
> > > >> >>>>>>>>>>> failure or what happens with the transaction buffer,
> etc.
> > > >> (that
> > > >> >> is
> > > >> >>>>>>>> what
> > > >> >>>>>>>>>>> I meant by "fail-over" in point 9.)
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> Best,
> > > >> >>>>>>>>>>> Bruno
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > > >> >>>>>>>>>>>> Hi Bruno,
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>> 1.
> > > >> >>>>>>>>>>>> Isn't this exactly the same issue that
> WriteBatchWithIndex
> > > >> >>>>>>>> transactions
> > > >> >>>>>>>>>>>> have, whereby exceeding (or likely to exceed)
> configured
> > > >> memory
> > > >> >>>>>>>> needs to
> > > >> >>>>>>>>>>>> trigger an early commit?
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>> 2.
> > > >> >>>>>>>>>>>> This is one of my big concerns. Ultimately, any
> approach
> > > >> based
> > > >> >> on
> > > >> >>>>>>>>>>> cracking
> > > >> >>>>>>>>>>>> open RocksDB internals and using it in ways it's not
> really
> > > >> >>>> designed
> > > >> >>>>>>>>>>> for is
> > > >> >>>>>>>>>>>> likely to have some unforseen performance or
> consistency
> > > >> issues.
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>> 3.
> > > >> >>>>>>>>>>>> What's your motivation for removing these early
> commits?
> > > >> While
> > > >> >> not
> > > >> >>>>>>>>>>> ideal, I
> > > >> >>>>>>>>>>>> think they're a decent compromise to ensure consistency
> > > >> whilst
> > > >> >>>>>>>>>>> maintaining
> > > >> >>>>>>>>>>>> good and predictable performance.
> > > >> >>>>>>>>>>>> All 3 of your suggested ideas seem *very* complicated,
> and
> > > >> might
> > > >> >>>>>>>>>>> actually
> > > >> >>>>>>>>>>>> make behaviour less predictable for users as a
> consequence.
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is
> growing a
> > > >> bit
> > > >> >>>> out
> > > >> >>>>>>>> of
> > > >> >>>>>>>>>>>> control. While it's good to discuss ideas for future
> > > >> >>>> improvements, I
> > > >> >>>>>>>>>>> think
> > > >> >>>>>>>>>>>> it's important to narrow the scope down to a design
> that
> > > >> >> achieves
> > > >> >>>>>>>> the
> > > >> >>>>>>>>>>> most
> > > >> >>>>>>>>>>>> pressing objectives (constant sized restorations during
> > > dirty
> > > >> >>>>>>>>>>>> close/unexpected errors). Any design that this KIP
> produces
> > > >> can
> > > >> >>>>>>>>>>> ultimately
> > > >> >>>>>>>>>>>> be changed in the future, especially if the bulk of it
> is
> > > >> >> internal
> > > >> >>>>>>>>>>>> behaviour.
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>> I'm going to spend some time next week trying to
> re-work
> > > the
> > > >> >>>>>>>> original
> > > >> >>>>>>>>>>>> WriteBatchWithIndex design to remove the
> newTransaction()
> > > >> >> method,
> > > >> >>>>>>>> such
> > > >> >>>>>>>>>>> that
> > > >> >>>>>>>>>>>> it's just an implementation detail of RocksDBStore.
> That
> > > >> way, if
> > > >> >>>> we
> > > >> >>>>>>>>>>> want to
> > > >> >>>>>>>>>>>> replace WBWI with something in the future, like the SST
> > > file
> > > >> >>>>>>>> management
> > > >> >>>>>>>>>>>> outlined by John, then we can do so with little/no API
> > > >> changes.
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>> Regards,
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>> Nick
> > > >> >>>>>>>>>>>>
> > > >> >>>>>>>>>>>
> > > >> >>>>>>>>>>
> > > >> >>>>>>>>>
> > > >> >>>>>>>>
> > > >> >>>>>>>
> > > >> >>>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>
> > > >> >
> > > >>
> > > >
> > >
>

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

Posted by Lucas Brutschy <lb...@confluent.io.INVALID>.
Hi Nick,

since I last read it in April, the KIP has become much cleaner and
easier to read. Great work!

It feels to me the last big open point is whether we can implement
isolation level as a query parameter. I understand that there are
implementation concerns, but as Colt says, it would be a great
addition, and would also simplify the migration path for this change.
Is the implementation problem you mentioned caused by the WriteBatch
not having a notion of a snapshot, as the underlying DB iterator does?
In that case, I am not sure a chain of WriteBatches as you propose
would fully solve the problem, but maybe I didn't dig enough into the
details to fully understand it.

If it's not possible to implement it now, would it be an option to
make sure in this KIP that we do not fully close the door on per-query
isolation levels in the interface, as it may be possible to implement
the missing primitives in RocksDB or Speedb in the future.

Understanding:

* U1) Why is it only "SHOULD" for changelogOffsets to be persisted
atomically with the records?
* U2) Don't understand the default implementation of `isolationLevel`.
The isolation level should be a property of the underlying store, and
not be defined by the default config? Existing stores probably don't
guarantee READ_COMMITTED, so the default should be to return
READ_UNCOMMITTED.

Nits:
* N1) Could `getComittedOffset` use an `OptionalLong` return type, to
avoid the `null`?
* N2) Could `apporixmateNumUncomittedBytes` use an `OptionalLong`
return type, to avoid the `-1`?
* N3) I don't understand why `managesOffsets` uses the 'manage' verb,
whereas all other methods use the "commits" verb. I'd suggest
`commitsOffsets`.

Either way, it feels this KIP is very close to the finish line, I'm
looking forward to seeing this in production!

Cheers,
Lucas

On Mon, Sep 18, 2023 at 6:57 AM Colt McNealy <co...@littlehorse.io> wrote:
>
> > Making IsolationLevel a query-time constraint, rather than linking it to
> the processing.guarantee.
>
> As I understand it, would this allow even a user of EOS to control whether
> reading committed or uncommitted records? If so, I am highly in favor of
> this.
>
> I know that I was one of the early people to point out the current
> shortcoming that IQ reads uncommitted records, but just this morning I
> realized a pattern we use which means that (for certain queries) our system
> needs to be able to read uncommitted records, which is the current behavior
> of Kafka Streams in EOS.***
>
> If IsolationLevel being a query-time decision allows for this, then that
> would be amazing. I would also vote that the default behavior should be for
> reading uncommitted records, because it is totally possible for a valid
> application to depend on that behavior, and breaking it in a minor release
> might be a bit strong.
>
> *** (Note, for the curious reader....) Our use-case/query pattern is a bit
> complex, but reading "uncommitted" records is actually safe in our case
> 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 soon as the processor processes the result. If they
> can't read uncommitted records, they may receive a "201 - Created"
> response, immediately followed by a "404 - 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 <ni...@gmail.com> wrote:
>
> > Addendum:
> >
> > I think we would also face the same problem with the approach John outlined
> > earlier (using the record cache as a transaction buffer and flushing it
> > straight to SST files). This is because the record cache (the ThreadCache
> > class) is not thread-safe, so every commit would invalidate open IQ
> > Iterators in the same way that RocksDB WriteBatches do.
> > --
> > Nick
> >
> > On Wed, 13 Sept 2023 at 16:58, Nick Telford <ni...@gmail.com>
> > 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.
> > > 2. Making IsolationLevel a query-time constraint, rather than linking it
> > > to the processing.guarantee.
> > >
> > > There's a wrinkle that makes this a challenge: Interactive Queries that
> > > open an Iterator, when using transactions and READ_UNCOMMITTED.
> > > The problem is that under READ_UNCOMMITTED, queries need to be able to
> > > read records from the currently uncommitted transaction buffer
> > > (WriteBatch). This includes for Iterators, which should iterate both the
> > > transaction buffer and underlying database (using
> > > WriteBatch#iteratorWithBase()).
> > >
> > > The issue is that when the StreamThread commits, it writes the current
> > > WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing the
> > > WriteBatch while an Interactive Query holds an open Iterator on it will
> > > invalidate the Iterator. Worse, it turns out that Iterators over a
> > > WriteBatch become invalidated not just when the WriteBatch is cleared,
> > but
> > > also when the Iterators' current key receives a new write.
> > >
> > > Now that I'm writing this, I remember that this is the major reason that
> > I
> > > switched the original design from having a query-time IsolationLevel to
> > > having the IsolationLevel linked to the transactionality of the stores
> > > themselves.
> > >
> > > It *might* be possible to resolve this, by having a "chain" of
> > > WriteBatches, with the StreamThread switching to a new WriteBatch
> > whenever
> > > a new Interactive Query attempts to read from the database, but that
> > could
> > > cause some performance problems/memory pressure when subjected to a high
> > > Interactive Query load. It would also reduce the efficiency of
> > WriteBatches
> > > on-commit, as we'd have to write N WriteBatches, where N is the number of
> > > Interactive Queries since the last commit.
> > >
> > > I realise this is getting into the weeds of the implementation, and you'd
> > > rather we focus on the API for now, but I think it's important to
> > consider
> > > how to implement the desired API, in case we come up with an API that
> > > cannot be implemented efficiently, or even at all!
> > >
> > > Thoughts?
> > > --
> > > Nick
> > >
> > > On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <ca...@apache.org> wrote:
> > >
> > >> Hi Nick,
> > >>
> > >> 6.
> > >> Of course, you are right! My bad!
> > >> Wiping out the state in the downgrading case is fine.
> > >>
> > >>
> > >> 3a.
> > >> Focus on the public facing changes for the KIP. We will manage 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.
> > >>
> > >>
> > >> Best,
> > >> Bruno
> > >>
> > >> On 9/13/23 11:57 AM, Nick Telford wrote:
> > >> > Hi Bruno,
> > >> >
> > >> > Thanks for getting back to me!
> > >> >
> > >> > 2.
> > >> > The fact that implementations can always track estimated memory usage
> > in
> > >> > the wrapper is a good point. I can remove -1 as an option, and I'll
> > >> clarify
> > >> > the JavaDoc that 0 is not just for non-transactional stores, which is
> > >> > currently misleading.
> > >> >
> > >> > 6.
> > >> > The problem with catching the exception in the downgrade process is
> > that
> > >> > would require new code in the Kafka version being downgraded to. Since
> > >> > users could conceivably downgrade to almost *any* older version of
> > Kafka
> > >> > Streams, I'm not sure how we could add that code?
> > >> > The only way I can think of doing it would be to provide a dedicated
> > >> > downgrade tool, that goes through every local store and removes the
> > >> > offsets column families. But that seems like an unnecessary amount of
> > >> extra
> > >> > code to maintain just to handle a somewhat niche situation, when the
> > >> > alternative (automatically wipe and restore stores) should be
> > >> acceptable.
> > >> >
> > >> > 1, 4, 5: Agreed. I'll make the changes you've requested.
> > >> >
> > >> > 3a.
> > >> > I agree that IsolationLevel makes more sense at query-time, and I
> > >> actually
> > >> > initially attempted to place the IsolationLevel at query-time, but I
> > ran
> > >> > into some problems:
> > >> > - The key issue is that, under ALOS we're not staging writes in
> > >> > transactions, so can't perform writes at the READ_COMMITTED isolation
> > >> > level. However, this may be addressed if we decide to *always* use
> > >> > transactions as discussed under 3b.
> > >> > - IQv1 and IQv2 have quite different implementations. I remember
> > having
> > >> > some difficulty understanding the IQv1 internals, which made it
> > >> difficult
> > >> > to determine what needed to be changed. However, I *think* this can be
> > >> > addressed for both implementations by wrapping the RocksDBStore in an
> > >> > IsolationLevel-dependent wrapper, that overrides read methods (get,
> > >> etc.)
> > >> > to either read directly from the database or from the ongoing
> > >> transaction.
> > >> > But IQv1 might still be difficult.
> > >> > - If IsolationLevel becomes a query constraint, then all other
> > >> StateStores
> > >> > will need to respect it, including the in-memory stores. This would
> > >> require
> > >> > us to adapt in-memory stores to stage their writes so they can be
> > >> isolated
> > >> > from READ_COMMITTTED queries. It would also become an important
> > >> > consideration for third-party stores on upgrade, as without changes,
> > >> they
> > >> > would not support READ_COMMITTED queries correctly.
> > >> >
> > >> > Ultimately, I may need some help making the necessary change to IQv1
> > to
> > >> > support this, but I don't think it's fundamentally impossible, if we
> > >> want
> > >> > to pursue this route.
> > >> >
> > >> > 3b.
> > >> > The main reason I chose to keep ALOS un-transactional was to minimize
> > >> > behavioural change for most users (I believe most Streams users use
> > the
> > >> > default configuration, which is ALOS). That said, it's clear that if
> > >> ALOS
> > >> > also used transactional stores, the only change in behaviour would be
> > >> that
> > >> > it would become *more correct*, which could be considered a "bug fix"
> > by
> > >> > users, rather than a change they need to handle.
> > >> >
> > >> > I believe that performance using transactions (aka. RocksDB
> > >> WriteBatches)
> > >> > should actually be *better* than the un-batched write-path that is
> > >> > currently used[1]. The only "performance" consideration will be the
> > >> > increased memory usage that transactions require. Given the
> > mitigations
> > >> for
> > >> > this memory that we have in place, I would expect that this is not a
> > >> > problem for most users.
> > >> >
> > >> > If we're happy to do so, we can make ALOS also use transactions.
> > >> >
> > >> > Regards,
> > >> > Nick
> > >> >
> > >> > Link 1:
> > >> >
> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > >> >
> > >> > On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <ca...@apache.org>
> > >> wrote:
> > >> >
> > >> >> Hi Nick,
> > >> >>
> > >> >> Thanks for the updates and sorry for the delay on my side!
> > >> >>
> > >> >>
> > >> >> 1.
> > >> >> Making the default implementation for flush() a no-op sounds good to
> > >> me.
> > >> >>
> > >> >>
> > >> >> 2.
> > >> >> I think what was bugging me here is that a third-party state store
> > >> needs
> > >> >> to implement the state store interface. That means they need to
> > >> >> implement a wrapper around the actual state store as we do for
> > RocksDB
> > >> >> with RocksDBStore. So, a third-party state store can always estimate
> > >> the
> > >> >> uncommitted bytes, if it wants, because the wrapper can record the
> > >> added
> > >> >> bytes.
> > >> >> One case I can think of where returning -1 makes sense is when
> > Streams
> > >> >> does not need to estimate the size of the write batch and trigger
> > >> >> extraordinary commits, because the third-party state store takes care
> > >> of
> > >> >> memory. But in that case the method could also just return 0. Even
> > that
> > >> >> case would be better solved with a method that returns whether the
> > >> state
> > >> >> store manages itself the memory used for uncommitted bytes or not.
> > >> >> Said that, I am fine with keeping the -1 return value, I was just
> > >> >> wondering when and if it will be used.
> > >> >>
> > >> >> Regarding returning 0 for transactional state stores when the batch
> > is
> > >> >> empty, I was just wondering because you explicitly stated
> > >> >>
> > >> >> "or {@code 0} if this StateStore does not support transactions."
> > >> >>
> > >> >> So it seemed to me returning 0 could only happen for
> > non-transactional
> > >> >> state stores.
> > >> >>
> > >> >>
> > >> >> 3.
> > >> >>
> > >> >> a) What do you think if we move the isolation level to IQ (v1 and
> > v2)?
> > >> >> In the end this is the only component that really needs to specify
> > the
> > >> >> isolation level. It is similar to the Kafka consumer that can choose
> > >> >> with what isolation level to read the input topic.
> > >> >> For IQv1 the isolation level should go into StoreQueryParameters. For
> > >> >> IQv2, I would add it to the Query interface.
> > >> >>
> > >> >> b) Point a) raises the question what should happen during
> > at-least-once
> > >> >> processing when the state store does not use transactions? John in
> > the
> > >> >> past proposed to also use transactions on state stores for
> > >> >> at-least-once. I like that idea, because it avoids aggregating the
> > same
> > >> >> records over and over again in the case of a failure. We had a case
> > in
> > >> >> the past where a Streams applications in at-least-once mode was
> > failing
> > >> >> continuously for some reasons I do not remember before committing the
> > >> >> offsets. After each failover, the app aggregated again and again the
> > >> >> same records. Of course the aggregate increased to very wrong values
> > >> >> just because of the failover. With transactions on the state stores
> > we
> > >> >> could have avoided this. The app would have output the same aggregate
> > >> >> multiple times (i.e., after each failover) but at least the value of
> > >> the
> > >> >> aggregate would not depend on the number of failovers. Outputting the
> > >> >> same aggregate multiple times would be incorrect under exactly-once
> > but
> > >> >> it is OK for at-least-once.
> > >> >> If it makes sense to add a config to turn on and off transactions on
> > >> >> state stores under at-least-once or just use transactions in any case
> > >> is
> > >> >> a question we should also discuss in this KIP. It depends a bit on
> > the
> > >> >> performance trade-off. Maybe to be safe, I would add a config.
> > >> >>
> > >> >>
> > >> >> 4.
> > >> >> Your points are all valid. I tend to say to keep the metrics around
> > >> >> flush() until we remove flush() completely from the interface. Calls
> > to
> > >> >> flush() might still exist since existing processors might still call
> > >> >> flush() explicitly as you mentioned in 1). For sure, we need to
> > >> document
> > >> >> how the metrics change due to the transactions in the upgrade notes.
> > >> >>
> > >> >>
> > >> >> 5.
> > >> >> I see. Then you should describe how the .position files are handled
> > in
> > >> >> a dedicated section of the KIP or incorporate the description in the
> > >> >> "Atomic Checkpointing" section instead of only mentioning it in the
> > >> >> "Compatibility, 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:
> > >> >>
> > >> >>
> > >> >>
> > >>
> > https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> > >> >>
> > >> >> It is a bit brittle, but it works.
> > >> >>
> > >> >>
> > >> >> Best,
> > >> >> Bruno
> > >> >>
> > >> >>
> > >> >> On 8/24/23 12:18 PM, Nick Telford wrote:
> > >> >>> Hi Bruno,
> > >> >>>
> > >> >>> Thanks for taking the time to review the KIP. I'm back from leave
> > now
> > >> and
> > >> >>> intend to move this forwards as quickly as I can.
> > >> >>>
> > >> >>> Addressing your points:
> > >> >>>
> > >> >>> 1.
> > >> >>> Because flush() is part of the StateStore API, it's exposed to
> > custom
> > >> >>> Processors, which might be making calls to flush(). This was
> > actually
> > >> the
> > >> >>> case in a few integration tests.
> > >> >>> To maintain as much compatibility as possible, I'd prefer not to
> > make
> > >> >> this
> > >> >>> an UnsupportedOperationException, as it will cause previously
> > working
> > >> >>> Processors to start throwing exceptions at runtime.
> > >> >>> I agree that it doesn't make sense for it to proxy commit(), though,
> > >> as
> > >> >>> that would cause it to violate the "StateStores commit only when the
> > >> Task
> > >> >>> commits" rule.
> > >> >>> Instead, I think we should make this a no-op. That way, existing
> > user
> > >> >>> Processors will continue to work as-before, without violation of
> > store
> > >> >>> consistency that would be caused by premature flush/commit of
> > >> StateStore
> > >> >>> data to disk.
> > >> >>> What do you think?
> > >> >>>
> > >> >>> 2.
> > >> >>> As stated in the JavaDoc, when a StateStore implementation is
> > >> >>> transactional, but is unable to estimate the uncommitted memory
> > usage,
> > >> >> the
> > >> >>> method will return -1.
> > >> >>> The intention here is to permit third-party implementations that may
> > >> not
> > >> >> be
> > >> >>> able to estimate memory usage.
> > >> >>>
> > >> >>> Yes, it will be 0 when nothing has been written to the store yet. I
> > >> >> thought
> > >> >>> that was implied by "This method will return an approximation of the
> > >> >> memory
> > >> >>> would be freed by the next call to {@link #commit(Map)}" and
> > "@return
> > >> The
> > >> >>> approximate size of all records awaiting {@link #commit(Map)}",
> > >> however,
> > >> >> I
> > >> >>> can add it explicitly to the JavaDoc if you think this is unclear?
> > >> >>>
> > >> >>> 3.
> > >> >>> I realise this is probably the most contentious point in my design,
> > >> and
> > >> >> I'm
> > >> >>> open to changing it if I'm unable to convince you of the benefits.
> > >> >>> Nevertheless, here's my argument:
> > >> >>> The Interactive Query (IQ) API(s) are directly provided StateStores
> > to
> > >> >>> query, and it may be important for users to programmatically know
> > >> which
> > >> >>> mode the StateStore is operating under. If we simply provide an
> > >> >>> "eosEnabled" boolean (as used throughout the internal streams
> > >> engine), or
> > >> >>> similar, then users will need to understand the operation and
> > >> >> consequences
> > >> >>> of each available processing mode and how it pertains to their
> > >> >> StateStore.
> > >> >>>
> > >> >>> Interactive Query users aren't the only people that care about the
> > >> >>> processing.mode/IsolationLevel of a StateStore: implementers of
> > custom
> > >> >>> StateStores also need to understand the behaviour expected of their
> > >> >>> implementation. KIP-892 introduces some assumptions into the Streams
> > >> >> Engine
> > >> >>> about how StateStores operate under each processing mode, and it's
> > >> >>> important that custom implementations adhere to those assumptions in
> > >> >> order
> > >> >>> to maintain the consistency guarantees.
> > >> >>>
> > >> >>> IsolationLevels provide a high-level contract on the behaviour of
> > the
> > >> >>> StateStore: a user knows that under READ_COMMITTED, they will see
> > >> writes
> > >> >>> only after the Task has committed, and under READ_UNCOMMITTED they
> > >> will
> > >> >> see
> > >> >>> writes immediately. No understanding of the details of each
> > >> >> processing.mode
> > >> >>> is required, either for IQ users or StateStore implementers.
> > >> >>>
> > >> >>> An argument can be made that these contractual guarantees can simply
> > >> be
> > >> >>> documented for the processing.mode (i.e. that exactly-once and
> > >> >>> exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves
> > >> like
> > >> >>> READ_UNCOMMITTED), but there are several small issues with this I'd
> > >> >> prefer
> > >> >>> to avoid:
> > >> >>>
> > >> >>>      - Where would we document these contracts, in a way that is
> > >> difficult
> > >> >>>      for users/implementers to miss/ignore?
> > >> >>>      - It's not clear to users that the processing mode is
> > >> communicating
> > >> >>>      an expectation of read isolation, unless they read the
> > >> >> documentation. Users
> > >> >>>      rarely consult documentation unless they feel they need to, so
> > >> it's
> > >> >> likely
> > >> >>>      this detail would get missed by many users.
> > >> >>>      - It tightly couples processing modes to read isolation. Adding
> > >> new
> > >> >>>      processing modes, or changing the read isolation of existing
> > >> >> processing
> > >> >>>      modes would be difficult/impossible.
> > >> >>>
> > >> >>> Ultimately, the cost of introducing IsolationLevels is just a single
> > >> >>> method, since we re-use the existing IsolationLevel enum from Kafka.
> > >> This
> > >> >>> gives us a clear place to document the contractual guarantees
> > expected
> > >> >>> of/provided by StateStores, that is accessible both by the
> > StateStore
> > >> >>> itself, and by IQ users.
> > >> >>>
> > >> >>> (Writing this I've just realised that the StateStore and IQ APIs
> > >> actually
> > >> >>> don't provide access to StateStoreContext that IQ users would have
> > >> direct
> > >> >>> access to... Perhaps StateStore should expose isolationLevel()
> > itself
> > >> >> too?)
> > >> >>>
> > >> >>> 4.
> > >> >>> Yeah, I'm not comfortable renaming the metrics in-place either, as
> > >> it's a
> > >> >>> backwards incompatible change. My concern is that, if we leave the
> > >> >> existing
> > >> >>> "flush" metrics in place, they will be confusing to users. Right
> > now,
> > >> >>> "flush" metrics record explicit flushes to disk, but under KIP-892,
> > >> even
> > >> >> a
> > >> >>> commit() will not explicitly flush data to disk - RocksDB will
> > decide
> > >> on
> > >> >>> when to flush memtables to disk itself.
> > >> >>>
> > >> >>> If we keep the existing "flush" metrics, we'd have two options,
> > which
> > >> >> both
> > >> >>> seem pretty bad to me:
> > >> >>>
> > >> >>>      1. Have them record calls to commit(), which would be
> > >> misleading, as
> > >> >>>      data is no longer explicitly "flushed" to disk by this call.
> > >> >>>      2. Have them record nothing at all, which is equivalent to
> > >> removing
> > >> >> the
> > >> >>>      metrics, except that users will see the metric still exists and
> > >> so
> > >> >> assume
> > >> >>>      that the metric is correct, and that there's a problem with
> > their
> > >> >> system
> > >> >>>      when there isn't.
> > >> >>>
> > >> >>> I agree that removing them is also a bad solution, and I'd like some
> > >> >>> guidance on the best path forward here.
> > >> >>>
> > >> >>> 5.
> > >> >>> Position files are updated on every write to a StateStore. Since our
> > >> >> writes
> > >> >>> are now buffered until commit(), we can't update the Position file
> > >> until
> > >> >>> commit() has been called, otherwise it would be inconsistent with
> > the
> > >> >> data
> > >> >>> in the event of a rollback. Consequently, we need to manage these
> > >> offsets
> > >> >>> the same way we manage the checkpoint offsets, and ensure they're
> > only
> > >> >>> written on commit().
> > >> >>>
> > >> >>> 6.
> > >> >>> Agreed, although I'm not exactly sure yet what tests to write. How
> > >> >> explicit
> > >> >>> do we need to be here in the KIP?
> > >> >>>
> > >> >>> As for upgrade/downgrade: upgrade is designed to be seamless, and we
> > >> >> should
> > >> >>> definitely add some tests around that. Downgrade, it transpires,
> > isn't
> > >> >>> currently possible, as the extra column family for offset storage is
> > >> >>> incompatible with the pre-KIP-892 implementation: when you open a
> > >> RocksDB
> > >> >>> database, you must open all available column families or receive an
> > >> >> error.
> > >> >>> What currently happens on downgrade is that it attempts to open the
> > >> >> store,
> > >> >>> throws an error about the offsets column family not being opened,
> > >> which
> > >> >>> triggers a wipe and rebuild of the Task. Given that downgrades
> > should
> > >> be
> > >> >>> uncommon, I think this is acceptable behaviour, as the end-state is
> > >> >>> consistent, even if it results in an undesirable state restore.
> > >> >>>
> > >> >>> Should I document the upgrade/downgrade behaviour explicitly in the
> > >> KIP?
> > >> >>>
> > >> >>> --
> > >> >>>
> > >> >>> Regards,
> > >> >>> Nick
> > >> >>>
> > >> >>>
> > >> >>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org>
> > >> wrote:
> > >> >>>
> > >> >>>> Hi Nick!
> > >> >>>>
> > >> >>>> Thanks for the updates!
> > >> >>>>
> > >> >>>> 1.
> > >> >>>> Why does StateStore#flush() default to
> > >> >>>> StateStore#commit(Collections.emptyMap())?
> > >> >>>> Since calls to flush() will not exist anymore after this KIP is
> > >> >>>> released, I would rather throw an unsupported operation exception
> > by
> > >> >>>> default.
> > >> >>>>
> > >> >>>>
> > >> >>>> 2.
> > >> >>>> When would a state store return -1 from
> > >> >>>> StateStore#approximateNumUncommittedBytes() while being
> > >> transactional?
> > >> >>>>
> > >> >>>> Wouldn't StateStore#approximateNumUncommittedBytes() also return 0
> > if
> > >> >>>> the state store is transactional but nothing has been written to
> > the
> > >> >>>> state store yet?
> > >> >>>>
> > >> >>>>
> > >> >>>> 3.
> > >> >>>> Sorry for bringing this up again. Does this KIP really need to
> > >> introduce
> > >> >>>> StateStoreContext#isolationLevel()? StateStoreContext has already
> > >> >>>> appConfigs() which basically exposes the same information, i.e., if
> > >> EOS
> > >> >>>> is enabled or not.
> > >> >>>> In one of your previous e-mails you wrote:
> > >> >>>>
> > >> >>>> "My idea was to try to keep the StateStore interface as loosely
> > >> coupled
> > >> >>>> from the Streams engine as possible, to give implementers more
> > >> freedom,
> > >> >>>> and reduce the amount of internal knowledge required."
> > >> >>>>
> > >> >>>> While I understand the intent, I doubt that it decreases the
> > >> coupling of
> > >> >>>> a StateStore interface and the Streams engine. READ_COMMITTED only
> > >> >>>> applies to IQ but not to reads by processors. Thus, implementers
> > >> need to
> > >> >>>> understand how Streams accesses the state stores.
> > >> >>>>
> > >> >>>> I would like to hear what others think about this.
> > >> >>>>
> > >> >>>>
> > >> >>>> 4.
> > >> >>>> Great exposing new metrics for transactional state stores!
> > However, I
> > >> >>>> would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides
> > unit
> > >> >>>> tests. Additionally, we probably need integration or system tests
> > to
> > >> >>>> verify that upgrades and downgrades between transactional and
> > >> >>>> non-transactional state stores work as expected.
> > >> >>>>
> > >> >>>>
> > >> >>>> Best,
> > >> >>>> Bruno
> > >> >>>>
> > >> >>>>
> > >> >>>>
> > >> >>>>
> > >> >>>>
> > >> >>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> > >> >>>>> One more thing: I noted John's suggestion in the KIP, under
> > >> "Rejected
> > >> >>>>> Alternatives". I still think it's an idea worth pursuing, but I
> > >> believe
> > >> >>>>> that it's out of the scope of this KIP, because it solves a
> > >> different
> > >> >> set
> > >> >>>>> of problems to this KIP, and the scope of this one has already
> > grown
> > >> >>>> quite
> > >> >>>>> large!
> > >> >>>>>
> > >> >>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> > nick.telford@gmail.com>
> > >> >>>> wrote:
> > >> >>>>>
> > >> >>>>>> Hi everyone,
> > >> >>>>>>
> > >> >>>>>> I've updated the KIP (
> > >> >>>>>>
> > >> >>>>
> > >> >>
> > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > >> >>>> )
> > >> >>>>>> with the latest changes; mostly bringing back "Atomic
> > >> Checkpointing"
> > >> >>>> (for
> > >> >>>>>> what feels like the 10th time!). I think the one thing missing is
> > >> some
> > >> >>>>>> changes to metrics (notably the store "flush" metrics will need
> > to
> > >> be
> > >> >>>>>> renamed to "commit").
> > >> >>>>>>
> > >> >>>>>> The reason I brought back Atomic Checkpointing was to decouple
> > >> store
> > >> >>>> flush
> > >> >>>>>> from store commit. This is important, because with Transactional
> > >> >>>>>> StateStores, we now need to call "flush" on *every* Task commit,
> > >> and
> > >> >> not
> > >> >>>>>> just when the StateStore is closing, otherwise our transaction
> > >> buffer
> > >> >>>> will
> > >> >>>>>> never be written and persisted, instead growing unbounded! I
> > >> >>>> experimented
> > >> >>>>>> with some simple solutions, like forcing a store flush whenever
> > the
> > >> >>>>>> transaction buffer was likely to exceed its configured size, but
> > >> this
> > >> >>>> was
> > >> >>>>>> brittle: it prevented the transaction buffer from being
> > configured
> > >> to
> > >> >> be
> > >> >>>>>> unbounded, and it still would have required explicit flushes of
> > >> >> RocksDB,
> > >> >>>>>> yielding sub-optimal performance and memory utilization.
> > >> >>>>>>
> > >> >>>>>> I deemed Atomic Checkpointing to be the "right" way to resolve
> > this
> > >> >>>>>> problem. By ensuring that the changelog offsets that correspond
> > to
> > >> the
> > >> >>>> most
> > >> >>>>>> recently written records are always atomically written to the
> > >> >> StateStore
> > >> >>>>>> (by writing them to the same transaction buffer), we can avoid
> > >> >> forcibly
> > >> >>>>>> flushing the RocksDB memtables to disk, letting RocksDB flush
> > them
> > >> >> only
> > >> >>>>>> when necessary, without losing any of our consistency guarantees.
> > >> See
> > >> >>>> the
> > >> >>>>>> updated KIP for more info.
> > >> >>>>>>
> > >> >>>>>> I have fully implemented these changes, although I'm still not
> > >> >> entirely
> > >> >>>>>> happy with the implementation for segmented StateStores, so I
> > plan
> > >> to
> > >> >>>>>> refactor that. Despite that, all tests pass. If you'd like to try
> > >> out
> > >> >> or
> > >> >>>>>> review this highly experimental and incomplete branch, it's
> > >> available
> > >> >>>> here:
> > >> >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note:
> > >> it's
> > >> >>>> built
> > >> >>>>>> against Kafka 3.5.0 so that I had a stable base to build and test
> > >> it
> > >> >> on,
> > >> >>>>>> and to enable easy apples-to-apples comparisons in a live
> > >> >> environment. I
> > >> >>>>>> plan to rebase it against trunk once it's nearer completion and
> > has
> > >> >> been
> > >> >>>>>> proven on our main application.
> > >> >>>>>>
> > >> >>>>>> I would really appreciate help in reviewing and testing:
> > >> >>>>>> - Segmented (Versioned, Session and Window) stores
> > >> >>>>>> - Global stores
> > >> >>>>>>
> > >> >>>>>> As I do not currently use either of these, so my primary test
> > >> >>>> environment
> > >> >>>>>> doesn't test these areas.
> > >> >>>>>>
> > >> >>>>>> I'm going on Parental Leave starting next week for a few weeks,
> > so
> > >> >> will
> > >> >>>>>> not have time to move this forward until late August. That said,
> > >> your
> > >> >>>>>> feedback is welcome and appreciated, I just won't be able to
> > >> respond
> > >> >> as
> > >> >>>>>> quickly as usual.
> > >> >>>>>>
> > >> >>>>>> Regards,
> > >> >>>>>> Nick
> > >> >>>>>>
> > >> >>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> > nick.telford@gmail.com>
> > >> >>>> wrote:
> > >> >>>>>>
> > >> >>>>>>> Hi Bruno
> > >> >>>>>>>
> > >> >>>>>>> Yes, that's correct, although the impact on IQ is not something
> > I
> > >> had
> > >> >>>>>>> considered.
> > >> >>>>>>>
> > >> >>>>>>> What about atomically updating the state store from the
> > >> transaction
> > >> >>>>>>>> buffer every commit interval and writing the checkpoint (thus,
> > >> >>>> flushing
> > >> >>>>>>>> the memtable) every configured amount of data and/or number of
> > >> >> commit
> > >> >>>>>>>> intervals?
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>>> I'm not quite sure I follow. Are you suggesting that we add an
> > >> >>>> additional
> > >> >>>>>>> config for the max number of commit intervals between
> > checkpoints?
> > >> >> That
> > >> >>>>>>> way, we would checkpoint *either* when the transaction buffers
> > are
> > >> >>>> nearly
> > >> >>>>>>> full, *OR* whenever a certain number of commit intervals have
> > >> >> elapsed,
> > >> >>>>>>> whichever comes first?
> > >> >>>>>>>
> > >> >>>>>>> That certainly seems reasonable, although this re-ignites an
> > >> earlier
> > >> >>>>>>> debate about whether a config should be measured in "number of
> > >> commit
> > >> >>>>>>> intervals", instead of just an absolute time.
> > >> >>>>>>>
> > >> >>>>>>> FWIW, I realised that this issue is the reason I was pursuing
> > the
> > >> >>>> Atomic
> > >> >>>>>>> Checkpoints, as it de-couples memtable flush from checkpointing,
> > >> >> which
> > >> >>>>>>> enables us to just checkpoint on every commit without any
> > >> performance
> > >> >>>>>>> impact. Atomic Checkpointing is definitely the "best" solution,
> > >> but
> > >> >>>> I'm not
> > >> >>>>>>> sure if this is enough to bring it back into this KIP.
> > >> >>>>>>>
> > >> >>>>>>> I'm currently working on moving all the transactional logic
> > >> directly
> > >> >>>> into
> > >> >>>>>>> RocksDBStore itself, which does away with the
> > >> >> StateStore#newTransaction
> > >> >>>>>>> method, and reduces the number of new classes introduced,
> > >> >> significantly
> > >> >>>>>>> reducing the complexity. If it works, and the complexity is
> > >> >> drastically
> > >> >>>>>>> reduced, I may try bringing back Atomic Checkpoints into this
> > KIP.
> > >> >>>>>>>
> > >> >>>>>>> Regards,
> > >> >>>>>>> Nick
> > >> >>>>>>>
> > >> >>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org>
> > >> >> wrote:
> > >> >>>>>>>
> > >> >>>>>>>> Hi Nick,
> > >> >>>>>>>>
> > >> >>>>>>>> Thanks for the insights! Very interesting!
> > >> >>>>>>>>
> > >> >>>>>>>> As far as I understand, you want to atomically update the state
> > >> >> store
> > >> >>>>>>>> from the transaction buffer, flush the memtable of a state
> > store
> > >> and
> > >> >>>>>>>> write the checkpoint not after the commit time elapsed but
> > after
> > >> the
> > >> >>>>>>>> transaction buffer reached a size that would lead to exceeding
> > >> >>>>>>>> statestore.transaction.buffer.max.bytes before the next commit
> > >> >>>> interval
> > >> >>>>>>>> ends.
> > >> >>>>>>>> That means, the Kafka transaction would commit every commit
> > >> interval
> > >> >>>> but
> > >> >>>>>>>> the state store will only be atomically updated roughly every
> > >> >>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ would
> > >> then
> > >> >>>> only
> > >> >>>>>>>> see new data roughly every
> > >> statestore.transaction.buffer.max.bytes.
> > >> >>>>>>>> After a failure the state store needs to restore up to
> > >> >>>>>>>> statestore.transaction.buffer.max.bytes.
> > >> >>>>>>>>
> > >> >>>>>>>> Is this correct?
> > >> >>>>>>>>
> > >> >>>>>>>> What about atomically updating the state store from the
> > >> transaction
> > >> >>>>>>>> buffer every commit interval and writing the checkpoint (thus,
> > >> >>>> flushing
> > >> >>>>>>>> the memtable) every configured amount of data and/or number of
> > >> >> commit
> > >> >>>>>>>> intervals? In such a way, we would have the same delay for
> > >> records
> > >> >>>>>>>> appearing in output topics and IQ because both would appear
> > when
> > >> the
> > >> >>>>>>>> Kafka transaction is committed. However, after a failure the
> > >> state
> > >> >>>> store
> > >> >>>>>>>> still needs to restore up to
> > >> statestore.transaction.buffer.max.bytes
> > >> >>>> and
> > >> >>>>>>>> it might restore data that is already in the state store
> > because
> > >> the
> > >> >>>>>>>> checkpoint lags behind the last stable offset (i.e. the last
> > >> >> committed
> > >> >>>>>>>> offset) of the changelog topics. Restoring data that is already
> > >> in
> > >> >> the
> > >> >>>>>>>> state store is idempotent, so eos should not violated.
> > >> >>>>>>>> This solution needs at least one new config to specify when a
> > >> >>>> checkpoint
> > >> >>>>>>>> should be written.
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>> A small correction to your previous e-mail that does not change
> > >> >>>> anything
> > >> >>>>>>>> you said: Under alos the default commit interval is 30 seconds,
> > >> not
> > >> >>>> five
> > >> >>>>>>>> seconds.
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>> Best,
> > >> >>>>>>>> Bruno
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> > >> >>>>>>>>> Hi everyone,
> > >> >>>>>>>>>
> > >> >>>>>>>>> I've begun performance testing my branch on our staging
> > >> >> environment,
> > >> >>>>>>>>> putting it through its paces in our non-trivial application.
> > I'm
> > >> >>>>>>>> already
> > >> >>>>>>>>> observing the same increased flush rate that we saw the last
> > >> time
> > >> >> we
> > >> >>>>>>>>> attempted to use a version of this KIP, but this time, I
> > think I
> > >> >> know
> > >> >>>>>>>> why.
> > >> >>>>>>>>>
> > >> >>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end
> > >> of
> > >> >> the
> > >> >>>>>>>> Task
> > >> >>>>>>>>> commit process, has the following behaviour:
> > >> >>>>>>>>>
> > >> >>>>>>>>>        - Under ALOS: checkpoint the state stores. This
> > includes
> > >> >>>>>>>>>        flushing memtables in RocksDB. This is acceptable
> > >> because the
> > >> >>>>>>>> default
> > >> >>>>>>>>>        commit.interval.ms is 5 seconds, so forcibly flushing
> > >> >> memtables
> > >> >>>>>>>> every 5
> > >> >>>>>>>>>        seconds is acceptable for most applications.
> > >> >>>>>>>>>        - Under EOS: checkpointing is not done, *unless* it's
> > >> being
> > >> >>>>>>>> forced, due
> > >> >>>>>>>>>        to e.g. the Task closing or being revoked. This means
> > >> that
> > >> >> under
> > >> >>>>>>>> normal
> > >> >>>>>>>>>        processing conditions, the state stores will not be
> > >> >>>> checkpointed,
> > >> >>>>>>>> and will
> > >> >>>>>>>>>        not have memtables flushed at all , unless RocksDB
> > >> decides to
> > >> >>>>>>>> flush them on
> > >> >>>>>>>>>        its own. Checkpointing stores and force-flushing their
> > >> >> memtables
> > >> >>>>>>>> is only
> > >> >>>>>>>>>        done when a Task is being closed.
> > >> >>>>>>>>>
> > >> >>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least
> > *some*
> > >> >>>> normal
> > >> >>>>>>>>> Task commits, in order to write the RocksDB transaction
> > buffers
> > >> to
> > >> >>>> the
> > >> >>>>>>>>> state stores, and to ensure the offsets are synced to disk to
> > >> >> prevent
> > >> >>>>>>>>> restores from getting out of hand. Consequently, my current
> > >> >>>>>>>> implementation
> > >> >>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is far too
> > >> >>>>>>>> frequent.
> > >> >>>>>>>>> This causes checkpoints every 10,000 records, which is a
> > change
> > >> in
> > >> >>>>>>>> flush
> > >> >>>>>>>>> behaviour, potentially causing performance problems for some
> > >> >>>>>>>> applications.
> > >> >>>>>>>>>
> > >> >>>>>>>>> I'm looking into possible solutions, and I'm currently leaning
> > >> >>>> towards
> > >> >>>>>>>>> using the statestore.transaction.buffer.max.bytes
> > configuration
> > >> to
> > >> >>>>>>>>> checkpoint Tasks once we are likely to exceed it. This would
> > >> >>>>>>>> complement the
> > >> >>>>>>>>> existing "early Task commit" functionality that this
> > >> configuration
> > >> >>>>>>>>> provides, in the following way:
> > >> >>>>>>>>>
> > >> >>>>>>>>>        - Currently, we use
> > >> statestore.transaction.buffer.max.bytes
> > >> >> to
> > >> >>>>>>>> force an
> > >> >>>>>>>>>        early Task commit if processing more records would
> > cause
> > >> our
> > >> >>>> state
> > >> >>>>>>>> store
> > >> >>>>>>>>>        transactions to exceed the memory assigned to them.
> > >> >>>>>>>>>        - New functionality: when a Task *does* commit, we will
> > >> not
> > >> >>>>>>>> checkpoint
> > >> >>>>>>>>>        the stores (and hence flush the transaction buffers)
> > >> unless
> > >> >> we
> > >> >>>>>>>> expect to
> > >> >>>>>>>>>        cross the statestore.transaction.buffer.max.bytes
> > >> threshold
> > >> >>>> before
> > >> >>>>>>>> the next
> > >> >>>>>>>>>        commit
> > >> >>>>>>>>>
> > >> >>>>>>>>> I'm also open to suggestions.
> > >> >>>>>>>>>
> > >> >>>>>>>>> Regards,
> > >> >>>>>>>>> Nick
> > >> >>>>>>>>>
> > >> >>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> > >> nick.telford@gmail.com
> > >> >>>
> > >> >>>>>>>> wrote:
> > >> >>>>>>>>>
> > >> >>>>>>>>>> Hi Bruno!
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 3.
> > >> >>>>>>>>>> By "less predictable for users", I meant in terms of
> > >> understanding
> > >> >>>> the
> > >> >>>>>>>>>> performance profile under various circumstances. The more
> > >> complex
> > >> >>>> the
> > >> >>>>>>>>>> solution, the more difficult it would be for users to
> > >> understand
> > >> >> the
> > >> >>>>>>>>>> performance they see. For example, spilling records to disk
> > >> when
> > >> >> the
> > >> >>>>>>>>>> transaction buffer reaches a threshold would, I expect,
> > reduce
> > >> >> write
> > >> >>>>>>>>>> throughput. This reduction in write throughput could be
> > >> >> unexpected,
> > >> >>>>>>>> and
> > >> >>>>>>>>>> potentially difficult to diagnose/understand for users.
> > >> >>>>>>>>>> At the moment, I think the "early commit" concept is
> > relatively
> > >> >>>>>>>>>> straightforward; it's easy to document, and conceptually
> > fairly
> > >> >>>>>>>> obvious to
> > >> >>>>>>>>>> users. We could probably add a metric to make it easier to
> > >> >>>> understand
> > >> >>>>>>>> when
> > >> >>>>>>>>>> it happens though.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 3. (the second one)
> > >> >>>>>>>>>> The IsolationLevel is *essentially* an indirect way of
> > telling
> > >> >>>>>>>> StateStores
> > >> >>>>>>>>>> whether they should be transactional. READ_COMMITTED
> > >> essentially
> > >> >>>>>>>> requires
> > >> >>>>>>>>>> transactions, because it dictates that two threads calling
> > >> >>>>>>>>>> `newTransaction()` should not see writes from the other
> > >> >> transaction
> > >> >>>>>>>> until
> > >> >>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all bets are
> > >> off,
> > >> >>>> and
> > >> >>>>>>>>>> stores can allow threads to observe written records at any
> > >> time,
> > >> >>>>>>>> which is
> > >> >>>>>>>>>> essentially "no transactions". That said, StateStores are
> > free
> > >> to
> > >> >>>>>>>> implement
> > >> >>>>>>>>>> these guarantees however they can, which is a bit more
> > relaxed
> > >> >> than
> > >> >>>>>>>>>> dictating "you must use transactions". For example, with
> > >> RocksDB
> > >> >> we
> > >> >>>>>>>> would
> > >> >>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
> > "transactions",
> > >> >>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But with
> > >> other
> > >> >>>>>>>> storage
> > >> >>>>>>>>>> engines, it might be preferable to *always* use transactions,
> > >> even
> > >> >>>>>>>> when
> > >> >>>>>>>>>> unnecessary; or there may be storage engines that don't
> > provide
> > >> >>>>>>>>>> transactions, but the isolation guarantees can be met using a
> > >> >>>>>>>> different
> > >> >>>>>>>>>> technique.
> > >> >>>>>>>>>> My idea was to try to keep the StateStore interface as
> > loosely
> > >> >>>> coupled
> > >> >>>>>>>>>> from the Streams engine as possible, to give implementers
> > more
> > >> >>>>>>>> freedom, and
> > >> >>>>>>>>>> reduce the amount of internal knowledge required.
> > >> >>>>>>>>>> That said, I understand that "IsolationLevel" might not be
> > the
> > >> >> right
> > >> >>>>>>>>>> abstraction, and we can always make it much more explicit if
> > >> >>>>>>>> required, e.g.
> > >> >>>>>>>>>> boolean transactional()
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> 7-8.
> > >> >>>>>>>>>> I can make these changes either later today or tomorrow.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Small update:
> > >> >>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of issues
> > >> that
> > >> >>>>>>>> needed
> > >> >>>>>>>>>> addressing. Currently, all the tests pass, which is
> > promising,
> > >> but
> > >> >>>> it
> > >> >>>>>>>> will
> > >> >>>>>>>>>> need to undergo some performance testing. I haven't (yet)
> > >> worked
> > >> >> on
> > >> >>>>>>>>>> removing the `newTransaction()` stuff, but I would expect
> > that,
> > >> >>>>>>>>>> behaviourally, it should make no difference. The branch is
> > >> >> available
> > >> >>>>>>>> at
> > >> >>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if
> > anyone
> > >> is
> > >> >>>>>>>>>> interested in taking an early look.
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> Regards,
> > >> >>>>>>>>>> Nick
> > >> >>>>>>>>>>
> > >> >>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> > >> cadonna@apache.org>
> > >> >>>>>>>> wrote:
> > >> >>>>>>>>>>
> > >> >>>>>>>>>>> Hi Nick,
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> 1.
> > >> >>>>>>>>>>> Yeah, I agree with you. That was actually also my point. I
> > >> >>>> understood
> > >> >>>>>>>>>>> that John was proposing the ingestion path as a way to avoid
> > >> the
> > >> >>>>>>>> early
> > >> >>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> 2.
> > >> >>>>>>>>>>> I agree with John here, that actually it is public API. My
> > >> >> question
> > >> >>>>>>>> is
> > >> >>>>>>>>>>> how this usage pattern affects normal processing.
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> 3.
> > >> >>>>>>>>>>> My concern is that checking for the size of the transaction
> > >> >> buffer
> > >> >>>>>>>> and
> > >> >>>>>>>>>>> maybe triggering an early commit affects the whole
> > processing
> > >> of
> > >> >>>>>>>> Kafka
> > >> >>>>>>>>>>> Streams. The transactionality of a state store is not
> > >> confined to
> > >> >>>> the
> > >> >>>>>>>>>>> state store itself, but spills over and changes the behavior
> > >> of
> > >> >>>> other
> > >> >>>>>>>>>>> parts of the system. I agree with you that it is a decent
> > >> >>>>>>>> compromise. I
> > >> >>>>>>>>>>> just wanted to analyse the downsides and list the options to
> > >> >>>> overcome
> > >> >>>>>>>>>>> them. I also agree with you that all options seem quite
> > heavy
> > >> >>>>>>>> compared
> > >> >>>>>>>>>>> with your KIP. I do not understand what you mean with "less
> > >> >>>>>>>> predictable
> > >> >>>>>>>>>>> for users", though.
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> I found the discussions about the alternatives really
> > >> >> interesting.
> > >> >>>>>>>> But I
> > >> >>>>>>>>>>> also think that your plan sounds good and we should continue
> > >> with
> > >> >>>> it!
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> 3.
> > >> >>>>>>>>>>> Ah, now, I understand the reasoning behind putting isolation
> > >> >> level
> > >> >>>> in
> > >> >>>>>>>>>>> the state store context. Thanks! Should that also be a way
> > to
> > >> >> give
> > >> >>>>>>>> the
> > >> >>>>>>>>>>> the state store the opportunity to decide whether to turn on
> > >> >>>>>>>>>>> transactions or not?
> > >> >>>>>>>>>>> With my comment, I was more concerned about how do you know
> > >> if a
> > >> >>>>>>>>>>> checkpoint file needs to be written under EOS, if you do not
> > >> >> have a
> > >> >>>>>>>> way
> > >> >>>>>>>>>>> to know if the state store is transactional or not. If a
> > state
> > >> >>>> store
> > >> >>>>>>>> is
> > >> >>>>>>>>>>> transactional, the checkpoint file can be written during
> > >> normal
> > >> >>>>>>>>>>> processing under EOS. If the state store is not
> > transactional,
> > >> >> the
> > >> >>>>>>>>>>> checkpoint file must not be written under EOS.
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> 7.
> > >> >>>>>>>>>>> My point was about not only considering the bytes in memory
> > in
> > >> >>>> config
> > >> >>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that might
> > be
> > >> >>>>>>>> spilled
> > >> >>>>>>>>>>> on disk. Basically, I was wondering whether you should
> > remove
> > >> the
> > >> >>>>>>>>>>> "memory" in "Maximum number of memory bytes to be used to
> > >> >>>>>>>>>>> buffer uncommitted state-store records." My thinking was
> > that
> > >> >> even
> > >> >>>>>>>> if a
> > >> >>>>>>>>>>> state store spills uncommitted bytes to disk, limiting the
> > >> >> overall
> > >> >>>>>>>> bytes
> > >> >>>>>>>>>>> might make sense. Thinking about it again and considering
> > the
> > >> >>>> recent
> > >> >>>>>>>>>>> discussions, it does not make too much sense anymore.
> > >> >>>>>>>>>>> I like the name statestore.transaction.buffer.max.bytes that
> > >> you
> > >> >>>>>>>> proposed.
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> 8.
> > >> >>>>>>>>>>> A high-level description (without implementation details) of
> > >> how
> > >> >>>>>>>> Kafka
> > >> >>>>>>>>>>> Streams will manage the commit of changelog transactions,
> > >> state
> > >> >>>> store
> > >> >>>>>>>>>>> transactions and checkpointing would be great. Would be
> > great
> > >> if
> > >> >>>> you
> > >> >>>>>>>>>>> could also add some sentences about the behavior in case of
> > a
> > >> >>>>>>>> failure.
> > >> >>>>>>>>>>> For instance how does a transactional state store recover
> > >> after a
> > >> >>>>>>>>>>> failure or what happens with the transaction buffer, etc.
> > >> (that
> > >> >> is
> > >> >>>>>>>> what
> > >> >>>>>>>>>>> I meant by "fail-over" in point 9.)
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> Best,
> > >> >>>>>>>>>>> Bruno
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > >> >>>>>>>>>>>> Hi Bruno,
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>> 1.
> > >> >>>>>>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
> > >> >>>>>>>> transactions
> > >> >>>>>>>>>>>> have, whereby exceeding (or likely to exceed) configured
> > >> memory
> > >> >>>>>>>> needs to
> > >> >>>>>>>>>>>> trigger an early commit?
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>> 2.
> > >> >>>>>>>>>>>> This is one of my big concerns. Ultimately, any approach
> > >> based
> > >> >> on
> > >> >>>>>>>>>>> cracking
> > >> >>>>>>>>>>>> open RocksDB internals and using it in ways it's not really
> > >> >>>> designed
> > >> >>>>>>>>>>> for is
> > >> >>>>>>>>>>>> likely to have some unforseen performance or consistency
> > >> issues.
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>> 3.
> > >> >>>>>>>>>>>> What's your motivation for removing these early commits?
> > >> While
> > >> >> not
> > >> >>>>>>>>>>> ideal, I
> > >> >>>>>>>>>>>> think they're a decent compromise to ensure consistency
> > >> whilst
> > >> >>>>>>>>>>> maintaining
> > >> >>>>>>>>>>>> good and predictable performance.
> > >> >>>>>>>>>>>> All 3 of your suggested ideas seem *very* complicated, and
> > >> might
> > >> >>>>>>>>>>> actually
> > >> >>>>>>>>>>>> make behaviour less predictable for users as a consequence.
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is growing a
> > >> bit
> > >> >>>> out
> > >> >>>>>>>> of
> > >> >>>>>>>>>>>> control. While it's good to discuss ideas for future
> > >> >>>> improvements, I
> > >> >>>>>>>>>>> think
> > >> >>>>>>>>>>>> it's important to narrow the scope down to a design that
> > >> >> achieves
> > >> >>>>>>>> the
> > >> >>>>>>>>>>> most
> > >> >>>>>>>>>>>> pressing objectives (constant sized restorations during
> > dirty
> > >> >>>>>>>>>>>> close/unexpected errors). Any design that this KIP produces
> > >> can
> > >> >>>>>>>>>>> ultimately
> > >> >>>>>>>>>>>> be changed in the future, especially if the bulk of it is
> > >> >> internal
> > >> >>>>>>>>>>>> behaviour.
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>> I'm going to spend some time next week trying to re-work
> > the
> > >> >>>>>>>> original
> > >> >>>>>>>>>>>> WriteBatchWithIndex design to remove the newTransaction()
> > >> >> method,
> > >> >>>>>>>> such
> > >> >>>>>>>>>>> that
> > >> >>>>>>>>>>>> it's just an implementation detail of RocksDBStore. That
> > >> way, if
> > >> >>>> we
> > >> >>>>>>>>>>> want to
> > >> >>>>>>>>>>>> replace WBWI with something in the future, like the SST
> > file
> > >> >>>>>>>> management
> > >> >>>>>>>>>>>> outlined by John, then we can do so with little/no API
> > >> changes.
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>> Regards,
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>> Nick
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>>
> > >> >>>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>
> > >> >>>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >> >
> > >>
> > >
> >

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

Posted by Colt McNealy <co...@littlehorse.io>.
> Making IsolationLevel a query-time constraint, rather than linking it to
the processing.guarantee.

As I understand it, would this allow even a user of EOS to control whether
reading committed or uncommitted records? If so, I am highly in favor of
this.

I know that I was one of the early people to point out the current
shortcoming that IQ reads uncommitted records, but just this morning I
realized a pattern we use which means that (for certain queries) our system
needs to be able to read uncommitted records, which is the current behavior
of Kafka Streams in EOS.***

If IsolationLevel being a query-time decision allows for this, then that
would be amazing. I would also vote that the default behavior should be for
reading uncommitted records, because it is totally possible for a valid
application to depend on that behavior, and breaking it in a minor release
might be a bit strong.

*** (Note, for the curious reader....) Our use-case/query pattern is a bit
complex, but reading "uncommitted" records is actually safe in our case
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 soon as the processor processes the result. If they
can't read uncommitted records, they may receive a "201 - Created"
response, immediately followed by a "404 - 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 <ni...@gmail.com> wrote:

> Addendum:
>
> I think we would also face the same problem with the approach John outlined
> earlier (using the record cache as a transaction buffer and flushing it
> straight to SST files). This is because the record cache (the ThreadCache
> class) is not thread-safe, so every commit would invalidate open IQ
> Iterators in the same way that RocksDB WriteBatches do.
> --
> Nick
>
> On Wed, 13 Sept 2023 at 16:58, Nick Telford <ni...@gmail.com>
> 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.
> > 2. Making IsolationLevel a query-time constraint, rather than linking it
> > to the processing.guarantee.
> >
> > There's a wrinkle that makes this a challenge: Interactive Queries that
> > open an Iterator, when using transactions and READ_UNCOMMITTED.
> > The problem is that under READ_UNCOMMITTED, queries need to be able to
> > read records from the currently uncommitted transaction buffer
> > (WriteBatch). This includes for Iterators, which should iterate both the
> > transaction buffer and underlying database (using
> > WriteBatch#iteratorWithBase()).
> >
> > The issue is that when the StreamThread commits, it writes the current
> > WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing the
> > WriteBatch while an Interactive Query holds an open Iterator on it will
> > invalidate the Iterator. Worse, it turns out that Iterators over a
> > WriteBatch become invalidated not just when the WriteBatch is cleared,
> but
> > also when the Iterators' current key receives a new write.
> >
> > Now that I'm writing this, I remember that this is the major reason that
> I
> > switched the original design from having a query-time IsolationLevel to
> > having the IsolationLevel linked to the transactionality of the stores
> > themselves.
> >
> > It *might* be possible to resolve this, by having a "chain" of
> > WriteBatches, with the StreamThread switching to a new WriteBatch
> whenever
> > a new Interactive Query attempts to read from the database, but that
> could
> > cause some performance problems/memory pressure when subjected to a high
> > Interactive Query load. It would also reduce the efficiency of
> WriteBatches
> > on-commit, as we'd have to write N WriteBatches, where N is the number of
> > Interactive Queries since the last commit.
> >
> > I realise this is getting into the weeds of the implementation, and you'd
> > rather we focus on the API for now, but I think it's important to
> consider
> > how to implement the desired API, in case we come up with an API that
> > cannot be implemented efficiently, or even at all!
> >
> > Thoughts?
> > --
> > Nick
> >
> > On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <ca...@apache.org> wrote:
> >
> >> Hi Nick,
> >>
> >> 6.
> >> Of course, you are right! My bad!
> >> Wiping out the state in the downgrading case is fine.
> >>
> >>
> >> 3a.
> >> Focus on the public facing changes for the KIP. We will manage 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.
> >>
> >>
> >> Best,
> >> Bruno
> >>
> >> On 9/13/23 11:57 AM, Nick Telford wrote:
> >> > Hi Bruno,
> >> >
> >> > Thanks for getting back to me!
> >> >
> >> > 2.
> >> > The fact that implementations can always track estimated memory usage
> in
> >> > the wrapper is a good point. I can remove -1 as an option, and I'll
> >> clarify
> >> > the JavaDoc that 0 is not just for non-transactional stores, which is
> >> > currently misleading.
> >> >
> >> > 6.
> >> > The problem with catching the exception in the downgrade process is
> that
> >> > would require new code in the Kafka version being downgraded to. Since
> >> > users could conceivably downgrade to almost *any* older version of
> Kafka
> >> > Streams, I'm not sure how we could add that code?
> >> > The only way I can think of doing it would be to provide a dedicated
> >> > downgrade tool, that goes through every local store and removes the
> >> > offsets column families. But that seems like an unnecessary amount of
> >> extra
> >> > code to maintain just to handle a somewhat niche situation, when the
> >> > alternative (automatically wipe and restore stores) should be
> >> acceptable.
> >> >
> >> > 1, 4, 5: Agreed. I'll make the changes you've requested.
> >> >
> >> > 3a.
> >> > I agree that IsolationLevel makes more sense at query-time, and I
> >> actually
> >> > initially attempted to place the IsolationLevel at query-time, but I
> ran
> >> > into some problems:
> >> > - The key issue is that, under ALOS we're not staging writes in
> >> > transactions, so can't perform writes at the READ_COMMITTED isolation
> >> > level. However, this may be addressed if we decide to *always* use
> >> > transactions as discussed under 3b.
> >> > - IQv1 and IQv2 have quite different implementations. I remember
> having
> >> > some difficulty understanding the IQv1 internals, which made it
> >> difficult
> >> > to determine what needed to be changed. However, I *think* this can be
> >> > addressed for both implementations by wrapping the RocksDBStore in an
> >> > IsolationLevel-dependent wrapper, that overrides read methods (get,
> >> etc.)
> >> > to either read directly from the database or from the ongoing
> >> transaction.
> >> > But IQv1 might still be difficult.
> >> > - If IsolationLevel becomes a query constraint, then all other
> >> StateStores
> >> > will need to respect it, including the in-memory stores. This would
> >> require
> >> > us to adapt in-memory stores to stage their writes so they can be
> >> isolated
> >> > from READ_COMMITTTED queries. It would also become an important
> >> > consideration for third-party stores on upgrade, as without changes,
> >> they
> >> > would not support READ_COMMITTED queries correctly.
> >> >
> >> > Ultimately, I may need some help making the necessary change to IQv1
> to
> >> > support this, but I don't think it's fundamentally impossible, if we
> >> want
> >> > to pursue this route.
> >> >
> >> > 3b.
> >> > The main reason I chose to keep ALOS un-transactional was to minimize
> >> > behavioural change for most users (I believe most Streams users use
> the
> >> > default configuration, which is ALOS). That said, it's clear that if
> >> ALOS
> >> > also used transactional stores, the only change in behaviour would be
> >> that
> >> > it would become *more correct*, which could be considered a "bug fix"
> by
> >> > users, rather than a change they need to handle.
> >> >
> >> > I believe that performance using transactions (aka. RocksDB
> >> WriteBatches)
> >> > should actually be *better* than the un-batched write-path that is
> >> > currently used[1]. The only "performance" consideration will be the
> >> > increased memory usage that transactions require. Given the
> mitigations
> >> for
> >> > this memory that we have in place, I would expect that this is not a
> >> > problem for most users.
> >> >
> >> > If we're happy to do so, we can make ALOS also use transactions.
> >> >
> >> > Regards,
> >> > Nick
> >> >
> >> > Link 1:
> >> >
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >> >
> >> > On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <ca...@apache.org>
> >> wrote:
> >> >
> >> >> Hi Nick,
> >> >>
> >> >> Thanks for the updates and sorry for the delay on my side!
> >> >>
> >> >>
> >> >> 1.
> >> >> Making the default implementation for flush() a no-op sounds good to
> >> me.
> >> >>
> >> >>
> >> >> 2.
> >> >> I think what was bugging me here is that a third-party state store
> >> needs
> >> >> to implement the state store interface. That means they need to
> >> >> implement a wrapper around the actual state store as we do for
> RocksDB
> >> >> with RocksDBStore. So, a third-party state store can always estimate
> >> the
> >> >> uncommitted bytes, if it wants, because the wrapper can record the
> >> added
> >> >> bytes.
> >> >> One case I can think of where returning -1 makes sense is when
> Streams
> >> >> does not need to estimate the size of the write batch and trigger
> >> >> extraordinary commits, because the third-party state store takes care
> >> of
> >> >> memory. But in that case the method could also just return 0. Even
> that
> >> >> case would be better solved with a method that returns whether the
> >> state
> >> >> store manages itself the memory used for uncommitted bytes or not.
> >> >> Said that, I am fine with keeping the -1 return value, I was just
> >> >> wondering when and if it will be used.
> >> >>
> >> >> Regarding returning 0 for transactional state stores when the batch
> is
> >> >> empty, I was just wondering because you explicitly stated
> >> >>
> >> >> "or {@code 0} if this StateStore does not support transactions."
> >> >>
> >> >> So it seemed to me returning 0 could only happen for
> non-transactional
> >> >> state stores.
> >> >>
> >> >>
> >> >> 3.
> >> >>
> >> >> a) What do you think if we move the isolation level to IQ (v1 and
> v2)?
> >> >> In the end this is the only component that really needs to specify
> the
> >> >> isolation level. It is similar to the Kafka consumer that can choose
> >> >> with what isolation level to read the input topic.
> >> >> For IQv1 the isolation level should go into StoreQueryParameters. For
> >> >> IQv2, I would add it to the Query interface.
> >> >>
> >> >> b) Point a) raises the question what should happen during
> at-least-once
> >> >> processing when the state store does not use transactions? John in
> the
> >> >> past proposed to also use transactions on state stores for
> >> >> at-least-once. I like that idea, because it avoids aggregating the
> same
> >> >> records over and over again in the case of a failure. We had a case
> in
> >> >> the past where a Streams applications in at-least-once mode was
> failing
> >> >> continuously for some reasons I do not remember before committing the
> >> >> offsets. After each failover, the app aggregated again and again the
> >> >> same records. Of course the aggregate increased to very wrong values
> >> >> just because of the failover. With transactions on the state stores
> we
> >> >> could have avoided this. The app would have output the same aggregate
> >> >> multiple times (i.e., after each failover) but at least the value of
> >> the
> >> >> aggregate would not depend on the number of failovers. Outputting the
> >> >> same aggregate multiple times would be incorrect under exactly-once
> but
> >> >> it is OK for at-least-once.
> >> >> If it makes sense to add a config to turn on and off transactions on
> >> >> state stores under at-least-once or just use transactions in any case
> >> is
> >> >> a question we should also discuss in this KIP. It depends a bit on
> the
> >> >> performance trade-off. Maybe to be safe, I would add a config.
> >> >>
> >> >>
> >> >> 4.
> >> >> Your points are all valid. I tend to say to keep the metrics around
> >> >> flush() until we remove flush() completely from the interface. Calls
> to
> >> >> flush() might still exist since existing processors might still call
> >> >> flush() explicitly as you mentioned in 1). For sure, we need to
> >> document
> >> >> how the metrics change due to the transactions in the upgrade notes.
> >> >>
> >> >>
> >> >> 5.
> >> >> I see. Then you should describe how the .position files are handled
> in
> >> >> a dedicated section of the KIP or incorporate the description in the
> >> >> "Atomic Checkpointing" section instead of only mentioning it in the
> >> >> "Compatibility, 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:
> >> >>
> >> >>
> >> >>
> >>
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> >> >>
> >> >> It is a bit brittle, but it works.
> >> >>
> >> >>
> >> >> Best,
> >> >> Bruno
> >> >>
> >> >>
> >> >> On 8/24/23 12:18 PM, Nick Telford wrote:
> >> >>> Hi Bruno,
> >> >>>
> >> >>> Thanks for taking the time to review the KIP. I'm back from leave
> now
> >> and
> >> >>> intend to move this forwards as quickly as I can.
> >> >>>
> >> >>> Addressing your points:
> >> >>>
> >> >>> 1.
> >> >>> Because flush() is part of the StateStore API, it's exposed to
> custom
> >> >>> Processors, which might be making calls to flush(). This was
> actually
> >> the
> >> >>> case in a few integration tests.
> >> >>> To maintain as much compatibility as possible, I'd prefer not to
> make
> >> >> this
> >> >>> an UnsupportedOperationException, as it will cause previously
> working
> >> >>> Processors to start throwing exceptions at runtime.
> >> >>> I agree that it doesn't make sense for it to proxy commit(), though,
> >> as
> >> >>> that would cause it to violate the "StateStores commit only when the
> >> Task
> >> >>> commits" rule.
> >> >>> Instead, I think we should make this a no-op. That way, existing
> user
> >> >>> Processors will continue to work as-before, without violation of
> store
> >> >>> consistency that would be caused by premature flush/commit of
> >> StateStore
> >> >>> data to disk.
> >> >>> What do you think?
> >> >>>
> >> >>> 2.
> >> >>> As stated in the JavaDoc, when a StateStore implementation is
> >> >>> transactional, but is unable to estimate the uncommitted memory
> usage,
> >> >> the
> >> >>> method will return -1.
> >> >>> The intention here is to permit third-party implementations that may
> >> not
> >> >> be
> >> >>> able to estimate memory usage.
> >> >>>
> >> >>> Yes, it will be 0 when nothing has been written to the store yet. I
> >> >> thought
> >> >>> that was implied by "This method will return an approximation of the
> >> >> memory
> >> >>> would be freed by the next call to {@link #commit(Map)}" and
> "@return
> >> The
> >> >>> approximate size of all records awaiting {@link #commit(Map)}",
> >> however,
> >> >> I
> >> >>> can add it explicitly to the JavaDoc if you think this is unclear?
> >> >>>
> >> >>> 3.
> >> >>> I realise this is probably the most contentious point in my design,
> >> and
> >> >> I'm
> >> >>> open to changing it if I'm unable to convince you of the benefits.
> >> >>> Nevertheless, here's my argument:
> >> >>> The Interactive Query (IQ) API(s) are directly provided StateStores
> to
> >> >>> query, and it may be important for users to programmatically know
> >> which
> >> >>> mode the StateStore is operating under. If we simply provide an
> >> >>> "eosEnabled" boolean (as used throughout the internal streams
> >> engine), or
> >> >>> similar, then users will need to understand the operation and
> >> >> consequences
> >> >>> of each available processing mode and how it pertains to their
> >> >> StateStore.
> >> >>>
> >> >>> Interactive Query users aren't the only people that care about the
> >> >>> processing.mode/IsolationLevel of a StateStore: implementers of
> custom
> >> >>> StateStores also need to understand the behaviour expected of their
> >> >>> implementation. KIP-892 introduces some assumptions into the Streams
> >> >> Engine
> >> >>> about how StateStores operate under each processing mode, and it's
> >> >>> important that custom implementations adhere to those assumptions in
> >> >> order
> >> >>> to maintain the consistency guarantees.
> >> >>>
> >> >>> IsolationLevels provide a high-level contract on the behaviour of
> the
> >> >>> StateStore: a user knows that under READ_COMMITTED, they will see
> >> writes
> >> >>> only after the Task has committed, and under READ_UNCOMMITTED they
> >> will
> >> >> see
> >> >>> writes immediately. No understanding of the details of each
> >> >> processing.mode
> >> >>> is required, either for IQ users or StateStore implementers.
> >> >>>
> >> >>> An argument can be made that these contractual guarantees can simply
> >> be
> >> >>> documented for the processing.mode (i.e. that exactly-once and
> >> >>> exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves
> >> like
> >> >>> READ_UNCOMMITTED), but there are several small issues with this I'd
> >> >> prefer
> >> >>> to avoid:
> >> >>>
> >> >>>      - Where would we document these contracts, in a way that is
> >> difficult
> >> >>>      for users/implementers to miss/ignore?
> >> >>>      - It's not clear to users that the processing mode is
> >> communicating
> >> >>>      an expectation of read isolation, unless they read the
> >> >> documentation. Users
> >> >>>      rarely consult documentation unless they feel they need to, so
> >> it's
> >> >> likely
> >> >>>      this detail would get missed by many users.
> >> >>>      - It tightly couples processing modes to read isolation. Adding
> >> new
> >> >>>      processing modes, or changing the read isolation of existing
> >> >> processing
> >> >>>      modes would be difficult/impossible.
> >> >>>
> >> >>> Ultimately, the cost of introducing IsolationLevels is just a single
> >> >>> method, since we re-use the existing IsolationLevel enum from Kafka.
> >> This
> >> >>> gives us a clear place to document the contractual guarantees
> expected
> >> >>> of/provided by StateStores, that is accessible both by the
> StateStore
> >> >>> itself, and by IQ users.
> >> >>>
> >> >>> (Writing this I've just realised that the StateStore and IQ APIs
> >> actually
> >> >>> don't provide access to StateStoreContext that IQ users would have
> >> direct
> >> >>> access to... Perhaps StateStore should expose isolationLevel()
> itself
> >> >> too?)
> >> >>>
> >> >>> 4.
> >> >>> Yeah, I'm not comfortable renaming the metrics in-place either, as
> >> it's a
> >> >>> backwards incompatible change. My concern is that, if we leave the
> >> >> existing
> >> >>> "flush" metrics in place, they will be confusing to users. Right
> now,
> >> >>> "flush" metrics record explicit flushes to disk, but under KIP-892,
> >> even
> >> >> a
> >> >>> commit() will not explicitly flush data to disk - RocksDB will
> decide
> >> on
> >> >>> when to flush memtables to disk itself.
> >> >>>
> >> >>> If we keep the existing "flush" metrics, we'd have two options,
> which
> >> >> both
> >> >>> seem pretty bad to me:
> >> >>>
> >> >>>      1. Have them record calls to commit(), which would be
> >> misleading, as
> >> >>>      data is no longer explicitly "flushed" to disk by this call.
> >> >>>      2. Have them record nothing at all, which is equivalent to
> >> removing
> >> >> the
> >> >>>      metrics, except that users will see the metric still exists and
> >> so
> >> >> assume
> >> >>>      that the metric is correct, and that there's a problem with
> their
> >> >> system
> >> >>>      when there isn't.
> >> >>>
> >> >>> I agree that removing them is also a bad solution, and I'd like some
> >> >>> guidance on the best path forward here.
> >> >>>
> >> >>> 5.
> >> >>> Position files are updated on every write to a StateStore. Since our
> >> >> writes
> >> >>> are now buffered until commit(), we can't update the Position file
> >> until
> >> >>> commit() has been called, otherwise it would be inconsistent with
> the
> >> >> data
> >> >>> in the event of a rollback. Consequently, we need to manage these
> >> offsets
> >> >>> the same way we manage the checkpoint offsets, and ensure they're
> only
> >> >>> written on commit().
> >> >>>
> >> >>> 6.
> >> >>> Agreed, although I'm not exactly sure yet what tests to write. How
> >> >> explicit
> >> >>> do we need to be here in the KIP?
> >> >>>
> >> >>> As for upgrade/downgrade: upgrade is designed to be seamless, and we
> >> >> should
> >> >>> definitely add some tests around that. Downgrade, it transpires,
> isn't
> >> >>> currently possible, as the extra column family for offset storage is
> >> >>> incompatible with the pre-KIP-892 implementation: when you open a
> >> RocksDB
> >> >>> database, you must open all available column families or receive an
> >> >> error.
> >> >>> What currently happens on downgrade is that it attempts to open the
> >> >> store,
> >> >>> throws an error about the offsets column family not being opened,
> >> which
> >> >>> triggers a wipe and rebuild of the Task. Given that downgrades
> should
> >> be
> >> >>> uncommon, I think this is acceptable behaviour, as the end-state is
> >> >>> consistent, even if it results in an undesirable state restore.
> >> >>>
> >> >>> Should I document the upgrade/downgrade behaviour explicitly in the
> >> KIP?
> >> >>>
> >> >>> --
> >> >>>
> >> >>> Regards,
> >> >>> Nick
> >> >>>
> >> >>>
> >> >>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org>
> >> wrote:
> >> >>>
> >> >>>> Hi Nick!
> >> >>>>
> >> >>>> Thanks for the updates!
> >> >>>>
> >> >>>> 1.
> >> >>>> Why does StateStore#flush() default to
> >> >>>> StateStore#commit(Collections.emptyMap())?
> >> >>>> Since calls to flush() will not exist anymore after this KIP is
> >> >>>> released, I would rather throw an unsupported operation exception
> by
> >> >>>> default.
> >> >>>>
> >> >>>>
> >> >>>> 2.
> >> >>>> When would a state store return -1 from
> >> >>>> StateStore#approximateNumUncommittedBytes() while being
> >> transactional?
> >> >>>>
> >> >>>> Wouldn't StateStore#approximateNumUncommittedBytes() also return 0
> if
> >> >>>> the state store is transactional but nothing has been written to
> the
> >> >>>> state store yet?
> >> >>>>
> >> >>>>
> >> >>>> 3.
> >> >>>> Sorry for bringing this up again. Does this KIP really need to
> >> introduce
> >> >>>> StateStoreContext#isolationLevel()? StateStoreContext has already
> >> >>>> appConfigs() which basically exposes the same information, i.e., if
> >> EOS
> >> >>>> is enabled or not.
> >> >>>> In one of your previous e-mails you wrote:
> >> >>>>
> >> >>>> "My idea was to try to keep the StateStore interface as loosely
> >> coupled
> >> >>>> from the Streams engine as possible, to give implementers more
> >> freedom,
> >> >>>> and reduce the amount of internal knowledge required."
> >> >>>>
> >> >>>> While I understand the intent, I doubt that it decreases the
> >> coupling of
> >> >>>> a StateStore interface and the Streams engine. READ_COMMITTED only
> >> >>>> applies to IQ but not to reads by processors. Thus, implementers
> >> need to
> >> >>>> understand how Streams accesses the state stores.
> >> >>>>
> >> >>>> I would like to hear what others think about this.
> >> >>>>
> >> >>>>
> >> >>>> 4.
> >> >>>> Great exposing new metrics for transactional state stores!
> However, I
> >> >>>> would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides
> unit
> >> >>>> tests. Additionally, we probably need integration or system tests
> to
> >> >>>> verify that upgrades and downgrades between transactional and
> >> >>>> non-transactional state stores work as expected.
> >> >>>>
> >> >>>>
> >> >>>> Best,
> >> >>>> Bruno
> >> >>>>
> >> >>>>
> >> >>>>
> >> >>>>
> >> >>>>
> >> >>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> >> >>>>> One more thing: I noted John's suggestion in the KIP, under
> >> "Rejected
> >> >>>>> Alternatives". I still think it's an idea worth pursuing, but I
> >> believe
> >> >>>>> that it's out of the scope of this KIP, because it solves a
> >> different
> >> >> set
> >> >>>>> of problems to this KIP, and the scope of this one has already
> grown
> >> >>>> quite
> >> >>>>> large!
> >> >>>>>
> >> >>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <
> nick.telford@gmail.com>
> >> >>>> wrote:
> >> >>>>>
> >> >>>>>> Hi everyone,
> >> >>>>>>
> >> >>>>>> I've updated the KIP (
> >> >>>>>>
> >> >>>>
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >> >>>> )
> >> >>>>>> with the latest changes; mostly bringing back "Atomic
> >> Checkpointing"
> >> >>>> (for
> >> >>>>>> what feels like the 10th time!). I think the one thing missing is
> >> some
> >> >>>>>> changes to metrics (notably the store "flush" metrics will need
> to
> >> be
> >> >>>>>> renamed to "commit").
> >> >>>>>>
> >> >>>>>> The reason I brought back Atomic Checkpointing was to decouple
> >> store
> >> >>>> flush
> >> >>>>>> from store commit. This is important, because with Transactional
> >> >>>>>> StateStores, we now need to call "flush" on *every* Task commit,
> >> and
> >> >> not
> >> >>>>>> just when the StateStore is closing, otherwise our transaction
> >> buffer
> >> >>>> will
> >> >>>>>> never be written and persisted, instead growing unbounded! I
> >> >>>> experimented
> >> >>>>>> with some simple solutions, like forcing a store flush whenever
> the
> >> >>>>>> transaction buffer was likely to exceed its configured size, but
> >> this
> >> >>>> was
> >> >>>>>> brittle: it prevented the transaction buffer from being
> configured
> >> to
> >> >> be
> >> >>>>>> unbounded, and it still would have required explicit flushes of
> >> >> RocksDB,
> >> >>>>>> yielding sub-optimal performance and memory utilization.
> >> >>>>>>
> >> >>>>>> I deemed Atomic Checkpointing to be the "right" way to resolve
> this
> >> >>>>>> problem. By ensuring that the changelog offsets that correspond
> to
> >> the
> >> >>>> most
> >> >>>>>> recently written records are always atomically written to the
> >> >> StateStore
> >> >>>>>> (by writing them to the same transaction buffer), we can avoid
> >> >> forcibly
> >> >>>>>> flushing the RocksDB memtables to disk, letting RocksDB flush
> them
> >> >> only
> >> >>>>>> when necessary, without losing any of our consistency guarantees.
> >> See
> >> >>>> the
> >> >>>>>> updated KIP for more info.
> >> >>>>>>
> >> >>>>>> I have fully implemented these changes, although I'm still not
> >> >> entirely
> >> >>>>>> happy with the implementation for segmented StateStores, so I
> plan
> >> to
> >> >>>>>> refactor that. Despite that, all tests pass. If you'd like to try
> >> out
> >> >> or
> >> >>>>>> review this highly experimental and incomplete branch, it's
> >> available
> >> >>>> here:
> >> >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note:
> >> it's
> >> >>>> built
> >> >>>>>> against Kafka 3.5.0 so that I had a stable base to build and test
> >> it
> >> >> on,
> >> >>>>>> and to enable easy apples-to-apples comparisons in a live
> >> >> environment. I
> >> >>>>>> plan to rebase it against trunk once it's nearer completion and
> has
> >> >> been
> >> >>>>>> proven on our main application.
> >> >>>>>>
> >> >>>>>> I would really appreciate help in reviewing and testing:
> >> >>>>>> - Segmented (Versioned, Session and Window) stores
> >> >>>>>> - Global stores
> >> >>>>>>
> >> >>>>>> As I do not currently use either of these, so my primary test
> >> >>>> environment
> >> >>>>>> doesn't test these areas.
> >> >>>>>>
> >> >>>>>> I'm going on Parental Leave starting next week for a few weeks,
> so
> >> >> will
> >> >>>>>> not have time to move this forward until late August. That said,
> >> your
> >> >>>>>> feedback is welcome and appreciated, I just won't be able to
> >> respond
> >> >> as
> >> >>>>>> quickly as usual.
> >> >>>>>>
> >> >>>>>> Regards,
> >> >>>>>> Nick
> >> >>>>>>
> >> >>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <
> nick.telford@gmail.com>
> >> >>>> wrote:
> >> >>>>>>
> >> >>>>>>> Hi Bruno
> >> >>>>>>>
> >> >>>>>>> Yes, that's correct, although the impact on IQ is not something
> I
> >> had
> >> >>>>>>> considered.
> >> >>>>>>>
> >> >>>>>>> What about atomically updating the state store from the
> >> transaction
> >> >>>>>>>> buffer every commit interval and writing the checkpoint (thus,
> >> >>>> flushing
> >> >>>>>>>> the memtable) every configured amount of data and/or number of
> >> >> commit
> >> >>>>>>>> intervals?
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>> I'm not quite sure I follow. Are you suggesting that we add an
> >> >>>> additional
> >> >>>>>>> config for the max number of commit intervals between
> checkpoints?
> >> >> That
> >> >>>>>>> way, we would checkpoint *either* when the transaction buffers
> are
> >> >>>> nearly
> >> >>>>>>> full, *OR* whenever a certain number of commit intervals have
> >> >> elapsed,
> >> >>>>>>> whichever comes first?
> >> >>>>>>>
> >> >>>>>>> That certainly seems reasonable, although this re-ignites an
> >> earlier
> >> >>>>>>> debate about whether a config should be measured in "number of
> >> commit
> >> >>>>>>> intervals", instead of just an absolute time.
> >> >>>>>>>
> >> >>>>>>> FWIW, I realised that this issue is the reason I was pursuing
> the
> >> >>>> Atomic
> >> >>>>>>> Checkpoints, as it de-couples memtable flush from checkpointing,
> >> >> which
> >> >>>>>>> enables us to just checkpoint on every commit without any
> >> performance
> >> >>>>>>> impact. Atomic Checkpointing is definitely the "best" solution,
> >> but
> >> >>>> I'm not
> >> >>>>>>> sure if this is enough to bring it back into this KIP.
> >> >>>>>>>
> >> >>>>>>> I'm currently working on moving all the transactional logic
> >> directly
> >> >>>> into
> >> >>>>>>> RocksDBStore itself, which does away with the
> >> >> StateStore#newTransaction
> >> >>>>>>> method, and reduces the number of new classes introduced,
> >> >> significantly
> >> >>>>>>> reducing the complexity. If it works, and the complexity is
> >> >> drastically
> >> >>>>>>> reduced, I may try bringing back Atomic Checkpoints into this
> KIP.
> >> >>>>>>>
> >> >>>>>>> Regards,
> >> >>>>>>> Nick
> >> >>>>>>>
> >> >>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org>
> >> >> wrote:
> >> >>>>>>>
> >> >>>>>>>> Hi Nick,
> >> >>>>>>>>
> >> >>>>>>>> Thanks for the insights! Very interesting!
> >> >>>>>>>>
> >> >>>>>>>> As far as I understand, you want to atomically update the state
> >> >> store
> >> >>>>>>>> from the transaction buffer, flush the memtable of a state
> store
> >> and
> >> >>>>>>>> write the checkpoint not after the commit time elapsed but
> after
> >> the
> >> >>>>>>>> transaction buffer reached a size that would lead to exceeding
> >> >>>>>>>> statestore.transaction.buffer.max.bytes before the next commit
> >> >>>> interval
> >> >>>>>>>> ends.
> >> >>>>>>>> That means, the Kafka transaction would commit every commit
> >> interval
> >> >>>> but
> >> >>>>>>>> the state store will only be atomically updated roughly every
> >> >>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ would
> >> then
> >> >>>> only
> >> >>>>>>>> see new data roughly every
> >> statestore.transaction.buffer.max.bytes.
> >> >>>>>>>> After a failure the state store needs to restore up to
> >> >>>>>>>> statestore.transaction.buffer.max.bytes.
> >> >>>>>>>>
> >> >>>>>>>> Is this correct?
> >> >>>>>>>>
> >> >>>>>>>> What about atomically updating the state store from the
> >> transaction
> >> >>>>>>>> buffer every commit interval and writing the checkpoint (thus,
> >> >>>> flushing
> >> >>>>>>>> the memtable) every configured amount of data and/or number of
> >> >> commit
> >> >>>>>>>> intervals? In such a way, we would have the same delay for
> >> records
> >> >>>>>>>> appearing in output topics and IQ because both would appear
> when
> >> the
> >> >>>>>>>> Kafka transaction is committed. However, after a failure the
> >> state
> >> >>>> store
> >> >>>>>>>> still needs to restore up to
> >> statestore.transaction.buffer.max.bytes
> >> >>>> and
> >> >>>>>>>> it might restore data that is already in the state store
> because
> >> the
> >> >>>>>>>> checkpoint lags behind the last stable offset (i.e. the last
> >> >> committed
> >> >>>>>>>> offset) of the changelog topics. Restoring data that is already
> >> in
> >> >> the
> >> >>>>>>>> state store is idempotent, so eos should not violated.
> >> >>>>>>>> This solution needs at least one new config to specify when a
> >> >>>> checkpoint
> >> >>>>>>>> should be written.
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> A small correction to your previous e-mail that does not change
> >> >>>> anything
> >> >>>>>>>> you said: Under alos the default commit interval is 30 seconds,
> >> not
> >> >>>> five
> >> >>>>>>>> seconds.
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> Best,
> >> >>>>>>>> Bruno
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> >> >>>>>>>>> Hi everyone,
> >> >>>>>>>>>
> >> >>>>>>>>> I've begun performance testing my branch on our staging
> >> >> environment,
> >> >>>>>>>>> putting it through its paces in our non-trivial application.
> I'm
> >> >>>>>>>> already
> >> >>>>>>>>> observing the same increased flush rate that we saw the last
> >> time
> >> >> we
> >> >>>>>>>>> attempted to use a version of this KIP, but this time, I
> think I
> >> >> know
> >> >>>>>>>> why.
> >> >>>>>>>>>
> >> >>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end
> >> of
> >> >> the
> >> >>>>>>>> Task
> >> >>>>>>>>> commit process, has the following behaviour:
> >> >>>>>>>>>
> >> >>>>>>>>>        - Under ALOS: checkpoint the state stores. This
> includes
> >> >>>>>>>>>        flushing memtables in RocksDB. This is acceptable
> >> because the
> >> >>>>>>>> default
> >> >>>>>>>>>        commit.interval.ms is 5 seconds, so forcibly flushing
> >> >> memtables
> >> >>>>>>>> every 5
> >> >>>>>>>>>        seconds is acceptable for most applications.
> >> >>>>>>>>>        - Under EOS: checkpointing is not done, *unless* it's
> >> being
> >> >>>>>>>> forced, due
> >> >>>>>>>>>        to e.g. the Task closing or being revoked. This means
> >> that
> >> >> under
> >> >>>>>>>> normal
> >> >>>>>>>>>        processing conditions, the state stores will not be
> >> >>>> checkpointed,
> >> >>>>>>>> and will
> >> >>>>>>>>>        not have memtables flushed at all , unless RocksDB
> >> decides to
> >> >>>>>>>> flush them on
> >> >>>>>>>>>        its own. Checkpointing stores and force-flushing their
> >> >> memtables
> >> >>>>>>>> is only
> >> >>>>>>>>>        done when a Task is being closed.
> >> >>>>>>>>>
> >> >>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least
> *some*
> >> >>>> normal
> >> >>>>>>>>> Task commits, in order to write the RocksDB transaction
> buffers
> >> to
> >> >>>> the
> >> >>>>>>>>> state stores, and to ensure the offsets are synced to disk to
> >> >> prevent
> >> >>>>>>>>> restores from getting out of hand. Consequently, my current
> >> >>>>>>>> implementation
> >> >>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is far too
> >> >>>>>>>> frequent.
> >> >>>>>>>>> This causes checkpoints every 10,000 records, which is a
> change
> >> in
> >> >>>>>>>> flush
> >> >>>>>>>>> behaviour, potentially causing performance problems for some
> >> >>>>>>>> applications.
> >> >>>>>>>>>
> >> >>>>>>>>> I'm looking into possible solutions, and I'm currently leaning
> >> >>>> towards
> >> >>>>>>>>> using the statestore.transaction.buffer.max.bytes
> configuration
> >> to
> >> >>>>>>>>> checkpoint Tasks once we are likely to exceed it. This would
> >> >>>>>>>> complement the
> >> >>>>>>>>> existing "early Task commit" functionality that this
> >> configuration
> >> >>>>>>>>> provides, in the following way:
> >> >>>>>>>>>
> >> >>>>>>>>>        - Currently, we use
> >> statestore.transaction.buffer.max.bytes
> >> >> to
> >> >>>>>>>> force an
> >> >>>>>>>>>        early Task commit if processing more records would
> cause
> >> our
> >> >>>> state
> >> >>>>>>>> store
> >> >>>>>>>>>        transactions to exceed the memory assigned to them.
> >> >>>>>>>>>        - New functionality: when a Task *does* commit, we will
> >> not
> >> >>>>>>>> checkpoint
> >> >>>>>>>>>        the stores (and hence flush the transaction buffers)
> >> unless
> >> >> we
> >> >>>>>>>> expect to
> >> >>>>>>>>>        cross the statestore.transaction.buffer.max.bytes
> >> threshold
> >> >>>> before
> >> >>>>>>>> the next
> >> >>>>>>>>>        commit
> >> >>>>>>>>>
> >> >>>>>>>>> I'm also open to suggestions.
> >> >>>>>>>>>
> >> >>>>>>>>> Regards,
> >> >>>>>>>>> Nick
> >> >>>>>>>>>
> >> >>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> >> nick.telford@gmail.com
> >> >>>
> >> >>>>>>>> wrote:
> >> >>>>>>>>>
> >> >>>>>>>>>> Hi Bruno!
> >> >>>>>>>>>>
> >> >>>>>>>>>> 3.
> >> >>>>>>>>>> By "less predictable for users", I meant in terms of
> >> understanding
> >> >>>> the
> >> >>>>>>>>>> performance profile under various circumstances. The more
> >> complex
> >> >>>> the
> >> >>>>>>>>>> solution, the more difficult it would be for users to
> >> understand
> >> >> the
> >> >>>>>>>>>> performance they see. For example, spilling records to disk
> >> when
> >> >> the
> >> >>>>>>>>>> transaction buffer reaches a threshold would, I expect,
> reduce
> >> >> write
> >> >>>>>>>>>> throughput. This reduction in write throughput could be
> >> >> unexpected,
> >> >>>>>>>> and
> >> >>>>>>>>>> potentially difficult to diagnose/understand for users.
> >> >>>>>>>>>> At the moment, I think the "early commit" concept is
> relatively
> >> >>>>>>>>>> straightforward; it's easy to document, and conceptually
> fairly
> >> >>>>>>>> obvious to
> >> >>>>>>>>>> users. We could probably add a metric to make it easier to
> >> >>>> understand
> >> >>>>>>>> when
> >> >>>>>>>>>> it happens though.
> >> >>>>>>>>>>
> >> >>>>>>>>>> 3. (the second one)
> >> >>>>>>>>>> The IsolationLevel is *essentially* an indirect way of
> telling
> >> >>>>>>>> StateStores
> >> >>>>>>>>>> whether they should be transactional. READ_COMMITTED
> >> essentially
> >> >>>>>>>> requires
> >> >>>>>>>>>> transactions, because it dictates that two threads calling
> >> >>>>>>>>>> `newTransaction()` should not see writes from the other
> >> >> transaction
> >> >>>>>>>> until
> >> >>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all bets are
> >> off,
> >> >>>> and
> >> >>>>>>>>>> stores can allow threads to observe written records at any
> >> time,
> >> >>>>>>>> which is
> >> >>>>>>>>>> essentially "no transactions". That said, StateStores are
> free
> >> to
> >> >>>>>>>> implement
> >> >>>>>>>>>> these guarantees however they can, which is a bit more
> relaxed
> >> >> than
> >> >>>>>>>>>> dictating "you must use transactions". For example, with
> >> RocksDB
> >> >> we
> >> >>>>>>>> would
> >> >>>>>>>>>> implement these as READ_COMMITTED == WBWI-based
> "transactions",
> >> >>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But with
> >> other
> >> >>>>>>>> storage
> >> >>>>>>>>>> engines, it might be preferable to *always* use transactions,
> >> even
> >> >>>>>>>> when
> >> >>>>>>>>>> unnecessary; or there may be storage engines that don't
> provide
> >> >>>>>>>>>> transactions, but the isolation guarantees can be met using a
> >> >>>>>>>> different
> >> >>>>>>>>>> technique.
> >> >>>>>>>>>> My idea was to try to keep the StateStore interface as
> loosely
> >> >>>> coupled
> >> >>>>>>>>>> from the Streams engine as possible, to give implementers
> more
> >> >>>>>>>> freedom, and
> >> >>>>>>>>>> reduce the amount of internal knowledge required.
> >> >>>>>>>>>> That said, I understand that "IsolationLevel" might not be
> the
> >> >> right
> >> >>>>>>>>>> abstraction, and we can always make it much more explicit if
> >> >>>>>>>> required, e.g.
> >> >>>>>>>>>> boolean transactional()
> >> >>>>>>>>>>
> >> >>>>>>>>>> 7-8.
> >> >>>>>>>>>> I can make these changes either later today or tomorrow.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Small update:
> >> >>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of issues
> >> that
> >> >>>>>>>> needed
> >> >>>>>>>>>> addressing. Currently, all the tests pass, which is
> promising,
> >> but
> >> >>>> it
> >> >>>>>>>> will
> >> >>>>>>>>>> need to undergo some performance testing. I haven't (yet)
> >> worked
> >> >> on
> >> >>>>>>>>>> removing the `newTransaction()` stuff, but I would expect
> that,
> >> >>>>>>>>>> behaviourally, it should make no difference. The branch is
> >> >> available
> >> >>>>>>>> at
> >> >>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if
> anyone
> >> is
> >> >>>>>>>>>> interested in taking an early look.
> >> >>>>>>>>>>
> >> >>>>>>>>>> Regards,
> >> >>>>>>>>>> Nick
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> >> cadonna@apache.org>
> >> >>>>>>>> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>>> Hi Nick,
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> 1.
> >> >>>>>>>>>>> Yeah, I agree with you. That was actually also my point. I
> >> >>>> understood
> >> >>>>>>>>>>> that John was proposing the ingestion path as a way to avoid
> >> the
> >> >>>>>>>> early
> >> >>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> 2.
> >> >>>>>>>>>>> I agree with John here, that actually it is public API. My
> >> >> question
> >> >>>>>>>> is
> >> >>>>>>>>>>> how this usage pattern affects normal processing.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> 3.
> >> >>>>>>>>>>> My concern is that checking for the size of the transaction
> >> >> buffer
> >> >>>>>>>> and
> >> >>>>>>>>>>> maybe triggering an early commit affects the whole
> processing
> >> of
> >> >>>>>>>> Kafka
> >> >>>>>>>>>>> Streams. The transactionality of a state store is not
> >> confined to
> >> >>>> the
> >> >>>>>>>>>>> state store itself, but spills over and changes the behavior
> >> of
> >> >>>> other
> >> >>>>>>>>>>> parts of the system. I agree with you that it is a decent
> >> >>>>>>>> compromise. I
> >> >>>>>>>>>>> just wanted to analyse the downsides and list the options to
> >> >>>> overcome
> >> >>>>>>>>>>> them. I also agree with you that all options seem quite
> heavy
> >> >>>>>>>> compared
> >> >>>>>>>>>>> with your KIP. I do not understand what you mean with "less
> >> >>>>>>>> predictable
> >> >>>>>>>>>>> for users", though.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> I found the discussions about the alternatives really
> >> >> interesting.
> >> >>>>>>>> But I
> >> >>>>>>>>>>> also think that your plan sounds good and we should continue
> >> with
> >> >>>> it!
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> 3.
> >> >>>>>>>>>>> Ah, now, I understand the reasoning behind putting isolation
> >> >> level
> >> >>>> in
> >> >>>>>>>>>>> the state store context. Thanks! Should that also be a way
> to
> >> >> give
> >> >>>>>>>> the
> >> >>>>>>>>>>> the state store the opportunity to decide whether to turn on
> >> >>>>>>>>>>> transactions or not?
> >> >>>>>>>>>>> With my comment, I was more concerned about how do you know
> >> if a
> >> >>>>>>>>>>> checkpoint file needs to be written under EOS, if you do not
> >> >> have a
> >> >>>>>>>> way
> >> >>>>>>>>>>> to know if the state store is transactional or not. If a
> state
> >> >>>> store
> >> >>>>>>>> is
> >> >>>>>>>>>>> transactional, the checkpoint file can be written during
> >> normal
> >> >>>>>>>>>>> processing under EOS. If the state store is not
> transactional,
> >> >> the
> >> >>>>>>>>>>> checkpoint file must not be written under EOS.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> 7.
> >> >>>>>>>>>>> My point was about not only considering the bytes in memory
> in
> >> >>>> config
> >> >>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that might
> be
> >> >>>>>>>> spilled
> >> >>>>>>>>>>> on disk. Basically, I was wondering whether you should
> remove
> >> the
> >> >>>>>>>>>>> "memory" in "Maximum number of memory bytes to be used to
> >> >>>>>>>>>>> buffer uncommitted state-store records." My thinking was
> that
> >> >> even
> >> >>>>>>>> if a
> >> >>>>>>>>>>> state store spills uncommitted bytes to disk, limiting the
> >> >> overall
> >> >>>>>>>> bytes
> >> >>>>>>>>>>> might make sense. Thinking about it again and considering
> the
> >> >>>> recent
> >> >>>>>>>>>>> discussions, it does not make too much sense anymore.
> >> >>>>>>>>>>> I like the name statestore.transaction.buffer.max.bytes that
> >> you
> >> >>>>>>>> proposed.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> 8.
> >> >>>>>>>>>>> A high-level description (without implementation details) of
> >> how
> >> >>>>>>>> Kafka
> >> >>>>>>>>>>> Streams will manage the commit of changelog transactions,
> >> state
> >> >>>> store
> >> >>>>>>>>>>> transactions and checkpointing would be great. Would be
> great
> >> if
> >> >>>> you
> >> >>>>>>>>>>> could also add some sentences about the behavior in case of
> a
> >> >>>>>>>> failure.
> >> >>>>>>>>>>> For instance how does a transactional state store recover
> >> after a
> >> >>>>>>>>>>> failure or what happens with the transaction buffer, etc.
> >> (that
> >> >> is
> >> >>>>>>>> what
> >> >>>>>>>>>>> I meant by "fail-over" in point 9.)
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Best,
> >> >>>>>>>>>>> Bruno
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> >> >>>>>>>>>>>> Hi Bruno,
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> 1.
> >> >>>>>>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
> >> >>>>>>>> transactions
> >> >>>>>>>>>>>> have, whereby exceeding (or likely to exceed) configured
> >> memory
> >> >>>>>>>> needs to
> >> >>>>>>>>>>>> trigger an early commit?
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> 2.
> >> >>>>>>>>>>>> This is one of my big concerns. Ultimately, any approach
> >> based
> >> >> on
> >> >>>>>>>>>>> cracking
> >> >>>>>>>>>>>> open RocksDB internals and using it in ways it's not really
> >> >>>> designed
> >> >>>>>>>>>>> for is
> >> >>>>>>>>>>>> likely to have some unforseen performance or consistency
> >> issues.
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> 3.
> >> >>>>>>>>>>>> What's your motivation for removing these early commits?
> >> While
> >> >> not
> >> >>>>>>>>>>> ideal, I
> >> >>>>>>>>>>>> think they're a decent compromise to ensure consistency
> >> whilst
> >> >>>>>>>>>>> maintaining
> >> >>>>>>>>>>>> good and predictable performance.
> >> >>>>>>>>>>>> All 3 of your suggested ideas seem *very* complicated, and
> >> might
> >> >>>>>>>>>>> actually
> >> >>>>>>>>>>>> make behaviour less predictable for users as a consequence.
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is growing a
> >> bit
> >> >>>> out
> >> >>>>>>>> of
> >> >>>>>>>>>>>> control. While it's good to discuss ideas for future
> >> >>>> improvements, I
> >> >>>>>>>>>>> think
> >> >>>>>>>>>>>> it's important to narrow the scope down to a design that
> >> >> achieves
> >> >>>>>>>> the
> >> >>>>>>>>>>> most
> >> >>>>>>>>>>>> pressing objectives (constant sized restorations during
> dirty
> >> >>>>>>>>>>>> close/unexpected errors). Any design that this KIP produces
> >> can
> >> >>>>>>>>>>> ultimately
> >> >>>>>>>>>>>> be changed in the future, especially if the bulk of it is
> >> >> internal
> >> >>>>>>>>>>>> behaviour.
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> I'm going to spend some time next week trying to re-work
> the
> >> >>>>>>>> original
> >> >>>>>>>>>>>> WriteBatchWithIndex design to remove the newTransaction()
> >> >> method,
> >> >>>>>>>> such
> >> >>>>>>>>>>> that
> >> >>>>>>>>>>>> it's just an implementation detail of RocksDBStore. That
> >> way, if
> >> >>>> we
> >> >>>>>>>>>>> want to
> >> >>>>>>>>>>>> replace WBWI with something in the future, like the SST
> file
> >> >>>>>>>> management
> >> >>>>>>>>>>>> outlined by John, then we can do so with little/no API
> >> changes.
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> Regards,
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> Nick
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>
> >> >>>>
> >> >>>
> >> >>
> >> >
> >>
> >
>

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

Posted by Nick Telford <ni...@gmail.com>.
Addendum:

I think we would also face the same problem with the approach John outlined
earlier (using the record cache as a transaction buffer and flushing it
straight to SST files). This is because the record cache (the ThreadCache
class) is not thread-safe, so every commit would invalidate open IQ
Iterators in the same way that RocksDB WriteBatches do.
--
Nick

On Wed, 13 Sept 2023 at 16:58, Nick Telford <ni...@gmail.com> 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.
> 2. Making IsolationLevel a query-time constraint, rather than linking it
> to the processing.guarantee.
>
> There's a wrinkle that makes this a challenge: Interactive Queries that
> open an Iterator, when using transactions and READ_UNCOMMITTED.
> The problem is that under READ_UNCOMMITTED, queries need to be able to
> read records from the currently uncommitted transaction buffer
> (WriteBatch). This includes for Iterators, which should iterate both the
> transaction buffer and underlying database (using
> WriteBatch#iteratorWithBase()).
>
> The issue is that when the StreamThread commits, it writes the current
> WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing the
> WriteBatch while an Interactive Query holds an open Iterator on it will
> invalidate the Iterator. Worse, it turns out that Iterators over a
> WriteBatch become invalidated not just when the WriteBatch is cleared, but
> also when the Iterators' current key receives a new write.
>
> Now that I'm writing this, I remember that this is the major reason that I
> switched the original design from having a query-time IsolationLevel to
> having the IsolationLevel linked to the transactionality of the stores
> themselves.
>
> It *might* be possible to resolve this, by having a "chain" of
> WriteBatches, with the StreamThread switching to a new WriteBatch whenever
> a new Interactive Query attempts to read from the database, but that could
> cause some performance problems/memory pressure when subjected to a high
> Interactive Query load. It would also reduce the efficiency of WriteBatches
> on-commit, as we'd have to write N WriteBatches, where N is the number of
> Interactive Queries since the last commit.
>
> I realise this is getting into the weeds of the implementation, and you'd
> rather we focus on the API for now, but I think it's important to consider
> how to implement the desired API, in case we come up with an API that
> cannot be implemented efficiently, or even at all!
>
> Thoughts?
> --
> Nick
>
> On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <ca...@apache.org> wrote:
>
>> Hi Nick,
>>
>> 6.
>> Of course, you are right! My bad!
>> Wiping out the state in the downgrading case is fine.
>>
>>
>> 3a.
>> Focus on the public facing changes for the KIP. We will manage 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.
>>
>>
>> Best,
>> Bruno
>>
>> On 9/13/23 11:57 AM, Nick Telford wrote:
>> > Hi Bruno,
>> >
>> > Thanks for getting back to me!
>> >
>> > 2.
>> > The fact that implementations can always track estimated memory usage in
>> > the wrapper is a good point. I can remove -1 as an option, and I'll
>> clarify
>> > the JavaDoc that 0 is not just for non-transactional stores, which is
>> > currently misleading.
>> >
>> > 6.
>> > The problem with catching the exception in the downgrade process is that
>> > would require new code in the Kafka version being downgraded to. Since
>> > users could conceivably downgrade to almost *any* older version of Kafka
>> > Streams, I'm not sure how we could add that code?
>> > The only way I can think of doing it would be to provide a dedicated
>> > downgrade tool, that goes through every local store and removes the
>> > offsets column families. But that seems like an unnecessary amount of
>> extra
>> > code to maintain just to handle a somewhat niche situation, when the
>> > alternative (automatically wipe and restore stores) should be
>> acceptable.
>> >
>> > 1, 4, 5: Agreed. I'll make the changes you've requested.
>> >
>> > 3a.
>> > I agree that IsolationLevel makes more sense at query-time, and I
>> actually
>> > initially attempted to place the IsolationLevel at query-time, but I ran
>> > into some problems:
>> > - The key issue is that, under ALOS we're not staging writes in
>> > transactions, so can't perform writes at the READ_COMMITTED isolation
>> > level. However, this may be addressed if we decide to *always* use
>> > transactions as discussed under 3b.
>> > - IQv1 and IQv2 have quite different implementations. I remember having
>> > some difficulty understanding the IQv1 internals, which made it
>> difficult
>> > to determine what needed to be changed. However, I *think* this can be
>> > addressed for both implementations by wrapping the RocksDBStore in an
>> > IsolationLevel-dependent wrapper, that overrides read methods (get,
>> etc.)
>> > to either read directly from the database or from the ongoing
>> transaction.
>> > But IQv1 might still be difficult.
>> > - If IsolationLevel becomes a query constraint, then all other
>> StateStores
>> > will need to respect it, including the in-memory stores. This would
>> require
>> > us to adapt in-memory stores to stage their writes so they can be
>> isolated
>> > from READ_COMMITTTED queries. It would also become an important
>> > consideration for third-party stores on upgrade, as without changes,
>> they
>> > would not support READ_COMMITTED queries correctly.
>> >
>> > Ultimately, I may need some help making the necessary change to IQv1 to
>> > support this, but I don't think it's fundamentally impossible, if we
>> want
>> > to pursue this route.
>> >
>> > 3b.
>> > The main reason I chose to keep ALOS un-transactional was to minimize
>> > behavioural change for most users (I believe most Streams users use the
>> > default configuration, which is ALOS). That said, it's clear that if
>> ALOS
>> > also used transactional stores, the only change in behaviour would be
>> that
>> > it would become *more correct*, which could be considered a "bug fix" by
>> > users, rather than a change they need to handle.
>> >
>> > I believe that performance using transactions (aka. RocksDB
>> WriteBatches)
>> > should actually be *better* than the un-batched write-path that is
>> > currently used[1]. The only "performance" consideration will be the
>> > increased memory usage that transactions require. Given the mitigations
>> for
>> > this memory that we have in place, I would expect that this is not a
>> > problem for most users.
>> >
>> > If we're happy to do so, we can make ALOS also use transactions.
>> >
>> > Regards,
>> > Nick
>> >
>> > Link 1:
>> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>> >
>> > On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <ca...@apache.org>
>> wrote:
>> >
>> >> Hi Nick,
>> >>
>> >> Thanks for the updates and sorry for the delay on my side!
>> >>
>> >>
>> >> 1.
>> >> Making the default implementation for flush() a no-op sounds good to
>> me.
>> >>
>> >>
>> >> 2.
>> >> I think what was bugging me here is that a third-party state store
>> needs
>> >> to implement the state store interface. That means they need to
>> >> implement a wrapper around the actual state store as we do for RocksDB
>> >> with RocksDBStore. So, a third-party state store can always estimate
>> the
>> >> uncommitted bytes, if it wants, because the wrapper can record the
>> added
>> >> bytes.
>> >> One case I can think of where returning -1 makes sense is when Streams
>> >> does not need to estimate the size of the write batch and trigger
>> >> extraordinary commits, because the third-party state store takes care
>> of
>> >> memory. But in that case the method could also just return 0. Even that
>> >> case would be better solved with a method that returns whether the
>> state
>> >> store manages itself the memory used for uncommitted bytes or not.
>> >> Said that, I am fine with keeping the -1 return value, I was just
>> >> wondering when and if it will be used.
>> >>
>> >> Regarding returning 0 for transactional state stores when the batch is
>> >> empty, I was just wondering because you explicitly stated
>> >>
>> >> "or {@code 0} if this StateStore does not support transactions."
>> >>
>> >> So it seemed to me returning 0 could only happen for non-transactional
>> >> state stores.
>> >>
>> >>
>> >> 3.
>> >>
>> >> a) What do you think if we move the isolation level to IQ (v1 and v2)?
>> >> In the end this is the only component that really needs to specify the
>> >> isolation level. It is similar to the Kafka consumer that can choose
>> >> with what isolation level to read the input topic.
>> >> For IQv1 the isolation level should go into StoreQueryParameters. For
>> >> IQv2, I would add it to the Query interface.
>> >>
>> >> b) Point a) raises the question what should happen during at-least-once
>> >> processing when the state store does not use transactions? John in the
>> >> past proposed to also use transactions on state stores for
>> >> at-least-once. I like that idea, because it avoids aggregating the same
>> >> records over and over again in the case of a failure. We had a case in
>> >> the past where a Streams applications in at-least-once mode was failing
>> >> continuously for some reasons I do not remember before committing the
>> >> offsets. After each failover, the app aggregated again and again the
>> >> same records. Of course the aggregate increased to very wrong values
>> >> just because of the failover. With transactions on the state stores we
>> >> could have avoided this. The app would have output the same aggregate
>> >> multiple times (i.e., after each failover) but at least the value of
>> the
>> >> aggregate would not depend on the number of failovers. Outputting the
>> >> same aggregate multiple times would be incorrect under exactly-once but
>> >> it is OK for at-least-once.
>> >> If it makes sense to add a config to turn on and off transactions on
>> >> state stores under at-least-once or just use transactions in any case
>> is
>> >> a question we should also discuss in this KIP. It depends a bit on the
>> >> performance trade-off. Maybe to be safe, I would add a config.
>> >>
>> >>
>> >> 4.
>> >> Your points are all valid. I tend to say to keep the metrics around
>> >> flush() until we remove flush() completely from the interface. Calls to
>> >> flush() might still exist since existing processors might still call
>> >> flush() explicitly as you mentioned in 1). For sure, we need to
>> document
>> >> how the metrics change due to the transactions in the upgrade notes.
>> >>
>> >>
>> >> 5.
>> >> I see. Then you should describe how the .position files are handled  in
>> >> a dedicated section of the KIP or incorporate the description in the
>> >> "Atomic Checkpointing" section instead of only mentioning it in the
>> >> "Compatibility, 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:
>> >>
>> >>
>> >>
>> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
>> >>
>> >> It is a bit brittle, but it works.
>> >>
>> >>
>> >> Best,
>> >> Bruno
>> >>
>> >>
>> >> On 8/24/23 12:18 PM, Nick Telford wrote:
>> >>> Hi Bruno,
>> >>>
>> >>> Thanks for taking the time to review the KIP. I'm back from leave now
>> and
>> >>> intend to move this forwards as quickly as I can.
>> >>>
>> >>> Addressing your points:
>> >>>
>> >>> 1.
>> >>> Because flush() is part of the StateStore API, it's exposed to custom
>> >>> Processors, which might be making calls to flush(). This was actually
>> the
>> >>> case in a few integration tests.
>> >>> To maintain as much compatibility as possible, I'd prefer not to make
>> >> this
>> >>> an UnsupportedOperationException, as it will cause previously working
>> >>> Processors to start throwing exceptions at runtime.
>> >>> I agree that it doesn't make sense for it to proxy commit(), though,
>> as
>> >>> that would cause it to violate the "StateStores commit only when the
>> Task
>> >>> commits" rule.
>> >>> Instead, I think we should make this a no-op. That way, existing user
>> >>> Processors will continue to work as-before, without violation of store
>> >>> consistency that would be caused by premature flush/commit of
>> StateStore
>> >>> data to disk.
>> >>> What do you think?
>> >>>
>> >>> 2.
>> >>> As stated in the JavaDoc, when a StateStore implementation is
>> >>> transactional, but is unable to estimate the uncommitted memory usage,
>> >> the
>> >>> method will return -1.
>> >>> The intention here is to permit third-party implementations that may
>> not
>> >> be
>> >>> able to estimate memory usage.
>> >>>
>> >>> Yes, it will be 0 when nothing has been written to the store yet. I
>> >> thought
>> >>> that was implied by "This method will return an approximation of the
>> >> memory
>> >>> would be freed by the next call to {@link #commit(Map)}" and "@return
>> The
>> >>> approximate size of all records awaiting {@link #commit(Map)}",
>> however,
>> >> I
>> >>> can add it explicitly to the JavaDoc if you think this is unclear?
>> >>>
>> >>> 3.
>> >>> I realise this is probably the most contentious point in my design,
>> and
>> >> I'm
>> >>> open to changing it if I'm unable to convince you of the benefits.
>> >>> Nevertheless, here's my argument:
>> >>> The Interactive Query (IQ) API(s) are directly provided StateStores to
>> >>> query, and it may be important for users to programmatically know
>> which
>> >>> mode the StateStore is operating under. If we simply provide an
>> >>> "eosEnabled" boolean (as used throughout the internal streams
>> engine), or
>> >>> similar, then users will need to understand the operation and
>> >> consequences
>> >>> of each available processing mode and how it pertains to their
>> >> StateStore.
>> >>>
>> >>> Interactive Query users aren't the only people that care about the
>> >>> processing.mode/IsolationLevel of a StateStore: implementers of custom
>> >>> StateStores also need to understand the behaviour expected of their
>> >>> implementation. KIP-892 introduces some assumptions into the Streams
>> >> Engine
>> >>> about how StateStores operate under each processing mode, and it's
>> >>> important that custom implementations adhere to those assumptions in
>> >> order
>> >>> to maintain the consistency guarantees.
>> >>>
>> >>> IsolationLevels provide a high-level contract on the behaviour of the
>> >>> StateStore: a user knows that under READ_COMMITTED, they will see
>> writes
>> >>> only after the Task has committed, and under READ_UNCOMMITTED they
>> will
>> >> see
>> >>> writes immediately. No understanding of the details of each
>> >> processing.mode
>> >>> is required, either for IQ users or StateStore implementers.
>> >>>
>> >>> An argument can be made that these contractual guarantees can simply
>> be
>> >>> documented for the processing.mode (i.e. that exactly-once and
>> >>> exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves
>> like
>> >>> READ_UNCOMMITTED), but there are several small issues with this I'd
>> >> prefer
>> >>> to avoid:
>> >>>
>> >>>      - Where would we document these contracts, in a way that is
>> difficult
>> >>>      for users/implementers to miss/ignore?
>> >>>      - It's not clear to users that the processing mode is
>> communicating
>> >>>      an expectation of read isolation, unless they read the
>> >> documentation. Users
>> >>>      rarely consult documentation unless they feel they need to, so
>> it's
>> >> likely
>> >>>      this detail would get missed by many users.
>> >>>      - It tightly couples processing modes to read isolation. Adding
>> new
>> >>>      processing modes, or changing the read isolation of existing
>> >> processing
>> >>>      modes would be difficult/impossible.
>> >>>
>> >>> Ultimately, the cost of introducing IsolationLevels is just a single
>> >>> method, since we re-use the existing IsolationLevel enum from Kafka.
>> This
>> >>> gives us a clear place to document the contractual guarantees expected
>> >>> of/provided by StateStores, that is accessible both by the StateStore
>> >>> itself, and by IQ users.
>> >>>
>> >>> (Writing this I've just realised that the StateStore and IQ APIs
>> actually
>> >>> don't provide access to StateStoreContext that IQ users would have
>> direct
>> >>> access to... Perhaps StateStore should expose isolationLevel() itself
>> >> too?)
>> >>>
>> >>> 4.
>> >>> Yeah, I'm not comfortable renaming the metrics in-place either, as
>> it's a
>> >>> backwards incompatible change. My concern is that, if we leave the
>> >> existing
>> >>> "flush" metrics in place, they will be confusing to users. Right now,
>> >>> "flush" metrics record explicit flushes to disk, but under KIP-892,
>> even
>> >> a
>> >>> commit() will not explicitly flush data to disk - RocksDB will decide
>> on
>> >>> when to flush memtables to disk itself.
>> >>>
>> >>> If we keep the existing "flush" metrics, we'd have two options, which
>> >> both
>> >>> seem pretty bad to me:
>> >>>
>> >>>      1. Have them record calls to commit(), which would be
>> misleading, as
>> >>>      data is no longer explicitly "flushed" to disk by this call.
>> >>>      2. Have them record nothing at all, which is equivalent to
>> removing
>> >> the
>> >>>      metrics, except that users will see the metric still exists and
>> so
>> >> assume
>> >>>      that the metric is correct, and that there's a problem with their
>> >> system
>> >>>      when there isn't.
>> >>>
>> >>> I agree that removing them is also a bad solution, and I'd like some
>> >>> guidance on the best path forward here.
>> >>>
>> >>> 5.
>> >>> Position files are updated on every write to a StateStore. Since our
>> >> writes
>> >>> are now buffered until commit(), we can't update the Position file
>> until
>> >>> commit() has been called, otherwise it would be inconsistent with the
>> >> data
>> >>> in the event of a rollback. Consequently, we need to manage these
>> offsets
>> >>> the same way we manage the checkpoint offsets, and ensure they're only
>> >>> written on commit().
>> >>>
>> >>> 6.
>> >>> Agreed, although I'm not exactly sure yet what tests to write. How
>> >> explicit
>> >>> do we need to be here in the KIP?
>> >>>
>> >>> As for upgrade/downgrade: upgrade is designed to be seamless, and we
>> >> should
>> >>> definitely add some tests around that. Downgrade, it transpires, isn't
>> >>> currently possible, as the extra column family for offset storage is
>> >>> incompatible with the pre-KIP-892 implementation: when you open a
>> RocksDB
>> >>> database, you must open all available column families or receive an
>> >> error.
>> >>> What currently happens on downgrade is that it attempts to open the
>> >> store,
>> >>> throws an error about the offsets column family not being opened,
>> which
>> >>> triggers a wipe and rebuild of the Task. Given that downgrades should
>> be
>> >>> uncommon, I think this is acceptable behaviour, as the end-state is
>> >>> consistent, even if it results in an undesirable state restore.
>> >>>
>> >>> Should I document the upgrade/downgrade behaviour explicitly in the
>> KIP?
>> >>>
>> >>> --
>> >>>
>> >>> Regards,
>> >>> Nick
>> >>>
>> >>>
>> >>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org>
>> wrote:
>> >>>
>> >>>> Hi Nick!
>> >>>>
>> >>>> Thanks for the updates!
>> >>>>
>> >>>> 1.
>> >>>> Why does StateStore#flush() default to
>> >>>> StateStore#commit(Collections.emptyMap())?
>> >>>> Since calls to flush() will not exist anymore after this KIP is
>> >>>> released, I would rather throw an unsupported operation exception by
>> >>>> default.
>> >>>>
>> >>>>
>> >>>> 2.
>> >>>> When would a state store return -1 from
>> >>>> StateStore#approximateNumUncommittedBytes() while being
>> transactional?
>> >>>>
>> >>>> Wouldn't StateStore#approximateNumUncommittedBytes() also return 0 if
>> >>>> the state store is transactional but nothing has been written to the
>> >>>> state store yet?
>> >>>>
>> >>>>
>> >>>> 3.
>> >>>> Sorry for bringing this up again. Does this KIP really need to
>> introduce
>> >>>> StateStoreContext#isolationLevel()? StateStoreContext has already
>> >>>> appConfigs() which basically exposes the same information, i.e., if
>> EOS
>> >>>> is enabled or not.
>> >>>> In one of your previous e-mails you wrote:
>> >>>>
>> >>>> "My idea was to try to keep the StateStore interface as loosely
>> coupled
>> >>>> from the Streams engine as possible, to give implementers more
>> freedom,
>> >>>> and reduce the amount of internal knowledge required."
>> >>>>
>> >>>> While I understand the intent, I doubt that it decreases the
>> coupling of
>> >>>> a StateStore interface and the Streams engine. READ_COMMITTED only
>> >>>> applies to IQ but not to reads by processors. Thus, implementers
>> need to
>> >>>> understand how Streams accesses the state stores.
>> >>>>
>> >>>> I would like to hear what others think about this.
>> >>>>
>> >>>>
>> >>>> 4.
>> >>>> Great exposing new metrics for transactional state stores! However, I
>> >>>> would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides unit
>> >>>> tests. Additionally, we probably need integration or system tests to
>> >>>> verify that upgrades and downgrades between transactional and
>> >>>> non-transactional state stores work as expected.
>> >>>>
>> >>>>
>> >>>> Best,
>> >>>> Bruno
>> >>>>
>> >>>>
>> >>>>
>> >>>>
>> >>>>
>> >>>> On 7/21/23 10:34 PM, Nick Telford wrote:
>> >>>>> One more thing: I noted John's suggestion in the KIP, under
>> "Rejected
>> >>>>> Alternatives". I still think it's an idea worth pursuing, but I
>> believe
>> >>>>> that it's out of the scope of this KIP, because it solves a
>> different
>> >> set
>> >>>>> of problems to this KIP, and the scope of this one has already grown
>> >>>> quite
>> >>>>> large!
>> >>>>>
>> >>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <ni...@gmail.com>
>> >>>> wrote:
>> >>>>>
>> >>>>>> Hi everyone,
>> >>>>>>
>> >>>>>> I've updated the KIP (
>> >>>>>>
>> >>>>
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> >>>> )
>> >>>>>> with the latest changes; mostly bringing back "Atomic
>> Checkpointing"
>> >>>> (for
>> >>>>>> what feels like the 10th time!). I think the one thing missing is
>> some
>> >>>>>> changes to metrics (notably the store "flush" metrics will need to
>> be
>> >>>>>> renamed to "commit").
>> >>>>>>
>> >>>>>> The reason I brought back Atomic Checkpointing was to decouple
>> store
>> >>>> flush
>> >>>>>> from store commit. This is important, because with Transactional
>> >>>>>> StateStores, we now need to call "flush" on *every* Task commit,
>> and
>> >> not
>> >>>>>> just when the StateStore is closing, otherwise our transaction
>> buffer
>> >>>> will
>> >>>>>> never be written and persisted, instead growing unbounded! I
>> >>>> experimented
>> >>>>>> with some simple solutions, like forcing a store flush whenever the
>> >>>>>> transaction buffer was likely to exceed its configured size, but
>> this
>> >>>> was
>> >>>>>> brittle: it prevented the transaction buffer from being configured
>> to
>> >> be
>> >>>>>> unbounded, and it still would have required explicit flushes of
>> >> RocksDB,
>> >>>>>> yielding sub-optimal performance and memory utilization.
>> >>>>>>
>> >>>>>> I deemed Atomic Checkpointing to be the "right" way to resolve this
>> >>>>>> problem. By ensuring that the changelog offsets that correspond to
>> the
>> >>>> most
>> >>>>>> recently written records are always atomically written to the
>> >> StateStore
>> >>>>>> (by writing them to the same transaction buffer), we can avoid
>> >> forcibly
>> >>>>>> flushing the RocksDB memtables to disk, letting RocksDB flush them
>> >> only
>> >>>>>> when necessary, without losing any of our consistency guarantees.
>> See
>> >>>> the
>> >>>>>> updated KIP for more info.
>> >>>>>>
>> >>>>>> I have fully implemented these changes, although I'm still not
>> >> entirely
>> >>>>>> happy with the implementation for segmented StateStores, so I plan
>> to
>> >>>>>> refactor that. Despite that, all tests pass. If you'd like to try
>> out
>> >> or
>> >>>>>> review this highly experimental and incomplete branch, it's
>> available
>> >>>> here:
>> >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note:
>> it's
>> >>>> built
>> >>>>>> against Kafka 3.5.0 so that I had a stable base to build and test
>> it
>> >> on,
>> >>>>>> and to enable easy apples-to-apples comparisons in a live
>> >> environment. I
>> >>>>>> plan to rebase it against trunk once it's nearer completion and has
>> >> been
>> >>>>>> proven on our main application.
>> >>>>>>
>> >>>>>> I would really appreciate help in reviewing and testing:
>> >>>>>> - Segmented (Versioned, Session and Window) stores
>> >>>>>> - Global stores
>> >>>>>>
>> >>>>>> As I do not currently use either of these, so my primary test
>> >>>> environment
>> >>>>>> doesn't test these areas.
>> >>>>>>
>> >>>>>> I'm going on Parental Leave starting next week for a few weeks, so
>> >> will
>> >>>>>> not have time to move this forward until late August. That said,
>> your
>> >>>>>> feedback is welcome and appreciated, I just won't be able to
>> respond
>> >> as
>> >>>>>> quickly as usual.
>> >>>>>>
>> >>>>>> Regards,
>> >>>>>> Nick
>> >>>>>>
>> >>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com>
>> >>>> wrote:
>> >>>>>>
>> >>>>>>> Hi Bruno
>> >>>>>>>
>> >>>>>>> Yes, that's correct, although the impact on IQ is not something I
>> had
>> >>>>>>> considered.
>> >>>>>>>
>> >>>>>>> What about atomically updating the state store from the
>> transaction
>> >>>>>>>> buffer every commit interval and writing the checkpoint (thus,
>> >>>> flushing
>> >>>>>>>> the memtable) every configured amount of data and/or number of
>> >> commit
>> >>>>>>>> intervals?
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>> I'm not quite sure I follow. Are you suggesting that we add an
>> >>>> additional
>> >>>>>>> config for the max number of commit intervals between checkpoints?
>> >> That
>> >>>>>>> way, we would checkpoint *either* when the transaction buffers are
>> >>>> nearly
>> >>>>>>> full, *OR* whenever a certain number of commit intervals have
>> >> elapsed,
>> >>>>>>> whichever comes first?
>> >>>>>>>
>> >>>>>>> That certainly seems reasonable, although this re-ignites an
>> earlier
>> >>>>>>> debate about whether a config should be measured in "number of
>> commit
>> >>>>>>> intervals", instead of just an absolute time.
>> >>>>>>>
>> >>>>>>> FWIW, I realised that this issue is the reason I was pursuing the
>> >>>> Atomic
>> >>>>>>> Checkpoints, as it de-couples memtable flush from checkpointing,
>> >> which
>> >>>>>>> enables us to just checkpoint on every commit without any
>> performance
>> >>>>>>> impact. Atomic Checkpointing is definitely the "best" solution,
>> but
>> >>>> I'm not
>> >>>>>>> sure if this is enough to bring it back into this KIP.
>> >>>>>>>
>> >>>>>>> I'm currently working on moving all the transactional logic
>> directly
>> >>>> into
>> >>>>>>> RocksDBStore itself, which does away with the
>> >> StateStore#newTransaction
>> >>>>>>> method, and reduces the number of new classes introduced,
>> >> significantly
>> >>>>>>> reducing the complexity. If it works, and the complexity is
>> >> drastically
>> >>>>>>> reduced, I may try bringing back Atomic Checkpoints into this KIP.
>> >>>>>>>
>> >>>>>>> Regards,
>> >>>>>>> Nick
>> >>>>>>>
>> >>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org>
>> >> wrote:
>> >>>>>>>
>> >>>>>>>> Hi Nick,
>> >>>>>>>>
>> >>>>>>>> Thanks for the insights! Very interesting!
>> >>>>>>>>
>> >>>>>>>> As far as I understand, you want to atomically update the state
>> >> store
>> >>>>>>>> from the transaction buffer, flush the memtable of a state store
>> and
>> >>>>>>>> write the checkpoint not after the commit time elapsed but after
>> the
>> >>>>>>>> transaction buffer reached a size that would lead to exceeding
>> >>>>>>>> statestore.transaction.buffer.max.bytes before the next commit
>> >>>> interval
>> >>>>>>>> ends.
>> >>>>>>>> That means, the Kafka transaction would commit every commit
>> interval
>> >>>> but
>> >>>>>>>> the state store will only be atomically updated roughly every
>> >>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ would
>> then
>> >>>> only
>> >>>>>>>> see new data roughly every
>> statestore.transaction.buffer.max.bytes.
>> >>>>>>>> After a failure the state store needs to restore up to
>> >>>>>>>> statestore.transaction.buffer.max.bytes.
>> >>>>>>>>
>> >>>>>>>> Is this correct?
>> >>>>>>>>
>> >>>>>>>> What about atomically updating the state store from the
>> transaction
>> >>>>>>>> buffer every commit interval and writing the checkpoint (thus,
>> >>>> flushing
>> >>>>>>>> the memtable) every configured amount of data and/or number of
>> >> commit
>> >>>>>>>> intervals? In such a way, we would have the same delay for
>> records
>> >>>>>>>> appearing in output topics and IQ because both would appear when
>> the
>> >>>>>>>> Kafka transaction is committed. However, after a failure the
>> state
>> >>>> store
>> >>>>>>>> still needs to restore up to
>> statestore.transaction.buffer.max.bytes
>> >>>> and
>> >>>>>>>> it might restore data that is already in the state store because
>> the
>> >>>>>>>> checkpoint lags behind the last stable offset (i.e. the last
>> >> committed
>> >>>>>>>> offset) of the changelog topics. Restoring data that is already
>> in
>> >> the
>> >>>>>>>> state store is idempotent, so eos should not violated.
>> >>>>>>>> This solution needs at least one new config to specify when a
>> >>>> checkpoint
>> >>>>>>>> should be written.
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> A small correction to your previous e-mail that does not change
>> >>>> anything
>> >>>>>>>> you said: Under alos the default commit interval is 30 seconds,
>> not
>> >>>> five
>> >>>>>>>> seconds.
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> Best,
>> >>>>>>>> Bruno
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
>> >>>>>>>>> Hi everyone,
>> >>>>>>>>>
>> >>>>>>>>> I've begun performance testing my branch on our staging
>> >> environment,
>> >>>>>>>>> putting it through its paces in our non-trivial application. I'm
>> >>>>>>>> already
>> >>>>>>>>> observing the same increased flush rate that we saw the last
>> time
>> >> we
>> >>>>>>>>> attempted to use a version of this KIP, but this time, I think I
>> >> know
>> >>>>>>>> why.
>> >>>>>>>>>
>> >>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end
>> of
>> >> the
>> >>>>>>>> Task
>> >>>>>>>>> commit process, has the following behaviour:
>> >>>>>>>>>
>> >>>>>>>>>        - Under ALOS: checkpoint the state stores. This includes
>> >>>>>>>>>        flushing memtables in RocksDB. This is acceptable
>> because the
>> >>>>>>>> default
>> >>>>>>>>>        commit.interval.ms is 5 seconds, so forcibly flushing
>> >> memtables
>> >>>>>>>> every 5
>> >>>>>>>>>        seconds is acceptable for most applications.
>> >>>>>>>>>        - Under EOS: checkpointing is not done, *unless* it's
>> being
>> >>>>>>>> forced, due
>> >>>>>>>>>        to e.g. the Task closing or being revoked. This means
>> that
>> >> under
>> >>>>>>>> normal
>> >>>>>>>>>        processing conditions, the state stores will not be
>> >>>> checkpointed,
>> >>>>>>>> and will
>> >>>>>>>>>        not have memtables flushed at all , unless RocksDB
>> decides to
>> >>>>>>>> flush them on
>> >>>>>>>>>        its own. Checkpointing stores and force-flushing their
>> >> memtables
>> >>>>>>>> is only
>> >>>>>>>>>        done when a Task is being closed.
>> >>>>>>>>>
>> >>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least *some*
>> >>>> normal
>> >>>>>>>>> Task commits, in order to write the RocksDB transaction buffers
>> to
>> >>>> the
>> >>>>>>>>> state stores, and to ensure the offsets are synced to disk to
>> >> prevent
>> >>>>>>>>> restores from getting out of hand. Consequently, my current
>> >>>>>>>> implementation
>> >>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is far too
>> >>>>>>>> frequent.
>> >>>>>>>>> This causes checkpoints every 10,000 records, which is a change
>> in
>> >>>>>>>> flush
>> >>>>>>>>> behaviour, potentially causing performance problems for some
>> >>>>>>>> applications.
>> >>>>>>>>>
>> >>>>>>>>> I'm looking into possible solutions, and I'm currently leaning
>> >>>> towards
>> >>>>>>>>> using the statestore.transaction.buffer.max.bytes configuration
>> to
>> >>>>>>>>> checkpoint Tasks once we are likely to exceed it. This would
>> >>>>>>>> complement the
>> >>>>>>>>> existing "early Task commit" functionality that this
>> configuration
>> >>>>>>>>> provides, in the following way:
>> >>>>>>>>>
>> >>>>>>>>>        - Currently, we use
>> statestore.transaction.buffer.max.bytes
>> >> to
>> >>>>>>>> force an
>> >>>>>>>>>        early Task commit if processing more records would cause
>> our
>> >>>> state
>> >>>>>>>> store
>> >>>>>>>>>        transactions to exceed the memory assigned to them.
>> >>>>>>>>>        - New functionality: when a Task *does* commit, we will
>> not
>> >>>>>>>> checkpoint
>> >>>>>>>>>        the stores (and hence flush the transaction buffers)
>> unless
>> >> we
>> >>>>>>>> expect to
>> >>>>>>>>>        cross the statestore.transaction.buffer.max.bytes
>> threshold
>> >>>> before
>> >>>>>>>> the next
>> >>>>>>>>>        commit
>> >>>>>>>>>
>> >>>>>>>>> I'm also open to suggestions.
>> >>>>>>>>>
>> >>>>>>>>> Regards,
>> >>>>>>>>> Nick
>> >>>>>>>>>
>> >>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
>> nick.telford@gmail.com
>> >>>
>> >>>>>>>> wrote:
>> >>>>>>>>>
>> >>>>>>>>>> Hi Bruno!
>> >>>>>>>>>>
>> >>>>>>>>>> 3.
>> >>>>>>>>>> By "less predictable for users", I meant in terms of
>> understanding
>> >>>> the
>> >>>>>>>>>> performance profile under various circumstances. The more
>> complex
>> >>>> the
>> >>>>>>>>>> solution, the more difficult it would be for users to
>> understand
>> >> the
>> >>>>>>>>>> performance they see. For example, spilling records to disk
>> when
>> >> the
>> >>>>>>>>>> transaction buffer reaches a threshold would, I expect, reduce
>> >> write
>> >>>>>>>>>> throughput. This reduction in write throughput could be
>> >> unexpected,
>> >>>>>>>> and
>> >>>>>>>>>> potentially difficult to diagnose/understand for users.
>> >>>>>>>>>> At the moment, I think the "early commit" concept is relatively
>> >>>>>>>>>> straightforward; it's easy to document, and conceptually fairly
>> >>>>>>>> obvious to
>> >>>>>>>>>> users. We could probably add a metric to make it easier to
>> >>>> understand
>> >>>>>>>> when
>> >>>>>>>>>> it happens though.
>> >>>>>>>>>>
>> >>>>>>>>>> 3. (the second one)
>> >>>>>>>>>> The IsolationLevel is *essentially* an indirect way of telling
>> >>>>>>>> StateStores
>> >>>>>>>>>> whether they should be transactional. READ_COMMITTED
>> essentially
>> >>>>>>>> requires
>> >>>>>>>>>> transactions, because it dictates that two threads calling
>> >>>>>>>>>> `newTransaction()` should not see writes from the other
>> >> transaction
>> >>>>>>>> until
>> >>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all bets are
>> off,
>> >>>> and
>> >>>>>>>>>> stores can allow threads to observe written records at any
>> time,
>> >>>>>>>> which is
>> >>>>>>>>>> essentially "no transactions". That said, StateStores are free
>> to
>> >>>>>>>> implement
>> >>>>>>>>>> these guarantees however they can, which is a bit more relaxed
>> >> than
>> >>>>>>>>>> dictating "you must use transactions". For example, with
>> RocksDB
>> >> we
>> >>>>>>>> would
>> >>>>>>>>>> implement these as READ_COMMITTED == WBWI-based "transactions",
>> >>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But with
>> other
>> >>>>>>>> storage
>> >>>>>>>>>> engines, it might be preferable to *always* use transactions,
>> even
>> >>>>>>>> when
>> >>>>>>>>>> unnecessary; or there may be storage engines that don't provide
>> >>>>>>>>>> transactions, but the isolation guarantees can be met using a
>> >>>>>>>> different
>> >>>>>>>>>> technique.
>> >>>>>>>>>> My idea was to try to keep the StateStore interface as loosely
>> >>>> coupled
>> >>>>>>>>>> from the Streams engine as possible, to give implementers more
>> >>>>>>>> freedom, and
>> >>>>>>>>>> reduce the amount of internal knowledge required.
>> >>>>>>>>>> That said, I understand that "IsolationLevel" might not be the
>> >> right
>> >>>>>>>>>> abstraction, and we can always make it much more explicit if
>> >>>>>>>> required, e.g.
>> >>>>>>>>>> boolean transactional()
>> >>>>>>>>>>
>> >>>>>>>>>> 7-8.
>> >>>>>>>>>> I can make these changes either later today or tomorrow.
>> >>>>>>>>>>
>> >>>>>>>>>> Small update:
>> >>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of issues
>> that
>> >>>>>>>> needed
>> >>>>>>>>>> addressing. Currently, all the tests pass, which is promising,
>> but
>> >>>> it
>> >>>>>>>> will
>> >>>>>>>>>> need to undergo some performance testing. I haven't (yet)
>> worked
>> >> on
>> >>>>>>>>>> removing the `newTransaction()` stuff, but I would expect that,
>> >>>>>>>>>> behaviourally, it should make no difference. The branch is
>> >> available
>> >>>>>>>> at
>> >>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone
>> is
>> >>>>>>>>>> interested in taking an early look.
>> >>>>>>>>>>
>> >>>>>>>>>> Regards,
>> >>>>>>>>>> Nick
>> >>>>>>>>>>
>> >>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
>> cadonna@apache.org>
>> >>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>> Hi Nick,
>> >>>>>>>>>>>
>> >>>>>>>>>>> 1.
>> >>>>>>>>>>> Yeah, I agree with you. That was actually also my point. I
>> >>>> understood
>> >>>>>>>>>>> that John was proposing the ingestion path as a way to avoid
>> the
>> >>>>>>>> early
>> >>>>>>>>>>> commits. Probably, I misinterpreted the intent.
>> >>>>>>>>>>>
>> >>>>>>>>>>> 2.
>> >>>>>>>>>>> I agree with John here, that actually it is public API. My
>> >> question
>> >>>>>>>> is
>> >>>>>>>>>>> how this usage pattern affects normal processing.
>> >>>>>>>>>>>
>> >>>>>>>>>>> 3.
>> >>>>>>>>>>> My concern is that checking for the size of the transaction
>> >> buffer
>> >>>>>>>> and
>> >>>>>>>>>>> maybe triggering an early commit affects the whole processing
>> of
>> >>>>>>>> Kafka
>> >>>>>>>>>>> Streams. The transactionality of a state store is not
>> confined to
>> >>>> the
>> >>>>>>>>>>> state store itself, but spills over and changes the behavior
>> of
>> >>>> other
>> >>>>>>>>>>> parts of the system. I agree with you that it is a decent
>> >>>>>>>> compromise. I
>> >>>>>>>>>>> just wanted to analyse the downsides and list the options to
>> >>>> overcome
>> >>>>>>>>>>> them. I also agree with you that all options seem quite heavy
>> >>>>>>>> compared
>> >>>>>>>>>>> with your KIP. I do not understand what you mean with "less
>> >>>>>>>> predictable
>> >>>>>>>>>>> for users", though.
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> I found the discussions about the alternatives really
>> >> interesting.
>> >>>>>>>> But I
>> >>>>>>>>>>> also think that your plan sounds good and we should continue
>> with
>> >>>> it!
>> >>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
>> >>>>>>>>>>>
>> >>>>>>>>>>> 3.
>> >>>>>>>>>>> Ah, now, I understand the reasoning behind putting isolation
>> >> level
>> >>>> in
>> >>>>>>>>>>> the state store context. Thanks! Should that also be a way to
>> >> give
>> >>>>>>>> the
>> >>>>>>>>>>> the state store the opportunity to decide whether to turn on
>> >>>>>>>>>>> transactions or not?
>> >>>>>>>>>>> With my comment, I was more concerned about how do you know
>> if a
>> >>>>>>>>>>> checkpoint file needs to be written under EOS, if you do not
>> >> have a
>> >>>>>>>> way
>> >>>>>>>>>>> to know if the state store is transactional or not. If a state
>> >>>> store
>> >>>>>>>> is
>> >>>>>>>>>>> transactional, the checkpoint file can be written during
>> normal
>> >>>>>>>>>>> processing under EOS. If the state store is not transactional,
>> >> the
>> >>>>>>>>>>> checkpoint file must not be written under EOS.
>> >>>>>>>>>>>
>> >>>>>>>>>>> 7.
>> >>>>>>>>>>> My point was about not only considering the bytes in memory in
>> >>>> config
>> >>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that might be
>> >>>>>>>> spilled
>> >>>>>>>>>>> on disk. Basically, I was wondering whether you should remove
>> the
>> >>>>>>>>>>> "memory" in "Maximum number of memory bytes to be used to
>> >>>>>>>>>>> buffer uncommitted state-store records." My thinking was that
>> >> even
>> >>>>>>>> if a
>> >>>>>>>>>>> state store spills uncommitted bytes to disk, limiting the
>> >> overall
>> >>>>>>>> bytes
>> >>>>>>>>>>> might make sense. Thinking about it again and considering the
>> >>>> recent
>> >>>>>>>>>>> discussions, it does not make too much sense anymore.
>> >>>>>>>>>>> I like the name statestore.transaction.buffer.max.bytes that
>> you
>> >>>>>>>> proposed.
>> >>>>>>>>>>>
>> >>>>>>>>>>> 8.
>> >>>>>>>>>>> A high-level description (without implementation details) of
>> how
>> >>>>>>>> Kafka
>> >>>>>>>>>>> Streams will manage the commit of changelog transactions,
>> state
>> >>>> store
>> >>>>>>>>>>> transactions and checkpointing would be great. Would be great
>> if
>> >>>> you
>> >>>>>>>>>>> could also add some sentences about the behavior in case of a
>> >>>>>>>> failure.
>> >>>>>>>>>>> For instance how does a transactional state store recover
>> after a
>> >>>>>>>>>>> failure or what happens with the transaction buffer, etc.
>> (that
>> >> is
>> >>>>>>>> what
>> >>>>>>>>>>> I meant by "fail-over" in point 9.)
>> >>>>>>>>>>>
>> >>>>>>>>>>> Best,
>> >>>>>>>>>>> Bruno
>> >>>>>>>>>>>
>> >>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
>> >>>>>>>>>>>> Hi Bruno,
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> 1.
>> >>>>>>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
>> >>>>>>>> transactions
>> >>>>>>>>>>>> have, whereby exceeding (or likely to exceed) configured
>> memory
>> >>>>>>>> needs to
>> >>>>>>>>>>>> trigger an early commit?
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> 2.
>> >>>>>>>>>>>> This is one of my big concerns. Ultimately, any approach
>> based
>> >> on
>> >>>>>>>>>>> cracking
>> >>>>>>>>>>>> open RocksDB internals and using it in ways it's not really
>> >>>> designed
>> >>>>>>>>>>> for is
>> >>>>>>>>>>>> likely to have some unforseen performance or consistency
>> issues.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> 3.
>> >>>>>>>>>>>> What's your motivation for removing these early commits?
>> While
>> >> not
>> >>>>>>>>>>> ideal, I
>> >>>>>>>>>>>> think they're a decent compromise to ensure consistency
>> whilst
>> >>>>>>>>>>> maintaining
>> >>>>>>>>>>>> good and predictable performance.
>> >>>>>>>>>>>> All 3 of your suggested ideas seem *very* complicated, and
>> might
>> >>>>>>>>>>> actually
>> >>>>>>>>>>>> make behaviour less predictable for users as a consequence.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is growing a
>> bit
>> >>>> out
>> >>>>>>>> of
>> >>>>>>>>>>>> control. While it's good to discuss ideas for future
>> >>>> improvements, I
>> >>>>>>>>>>> think
>> >>>>>>>>>>>> it's important to narrow the scope down to a design that
>> >> achieves
>> >>>>>>>> the
>> >>>>>>>>>>> most
>> >>>>>>>>>>>> pressing objectives (constant sized restorations during dirty
>> >>>>>>>>>>>> close/unexpected errors). Any design that this KIP produces
>> can
>> >>>>>>>>>>> ultimately
>> >>>>>>>>>>>> be changed in the future, especially if the bulk of it is
>> >> internal
>> >>>>>>>>>>>> behaviour.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> I'm going to spend some time next week trying to re-work the
>> >>>>>>>> original
>> >>>>>>>>>>>> WriteBatchWithIndex design to remove the newTransaction()
>> >> method,
>> >>>>>>>> such
>> >>>>>>>>>>> that
>> >>>>>>>>>>>> it's just an implementation detail of RocksDBStore. That
>> way, if
>> >>>> we
>> >>>>>>>>>>> want to
>> >>>>>>>>>>>> replace WBWI with something in the future, like the SST file
>> >>>>>>>> management
>> >>>>>>>>>>>> outlined by John, then we can do so with little/no API
>> changes.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Regards,
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Nick
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>
>> >>>>
>> >>>
>> >>
>> >
>>
>

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

Posted by Nick Telford <ni...@gmail.com>.
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.
2. Making IsolationLevel a query-time constraint, rather than linking it to
the processing.guarantee.

There's a wrinkle that makes this a challenge: Interactive Queries that
open an Iterator, when using transactions and READ_UNCOMMITTED.
The problem is that under READ_UNCOMMITTED, queries need to be able to read
records from the currently uncommitted transaction buffer (WriteBatch).
This includes for Iterators, which should iterate both the transaction
buffer and underlying database (using WriteBatch#iteratorWithBase()).

The issue is that when the StreamThread commits, it writes the current
WriteBatch to RocksDB *and then clears the WriteBatch*. Clearing the
WriteBatch while an Interactive Query holds an open Iterator on it will
invalidate the Iterator. Worse, it turns out that Iterators over a
WriteBatch become invalidated not just when the WriteBatch is cleared, but
also when the Iterators' current key receives a new write.

Now that I'm writing this, I remember that this is the major reason that I
switched the original design from having a query-time IsolationLevel to
having the IsolationLevel linked to the transactionality of the stores
themselves.

It *might* be possible to resolve this, by having a "chain" of
WriteBatches, with the StreamThread switching to a new WriteBatch whenever
a new Interactive Query attempts to read from the database, but that could
cause some performance problems/memory pressure when subjected to a high
Interactive Query load. It would also reduce the efficiency of WriteBatches
on-commit, as we'd have to write N WriteBatches, where N is the number of
Interactive Queries since the last commit.

I realise this is getting into the weeds of the implementation, and you'd
rather we focus on the API for now, but I think it's important to consider
how to implement the desired API, in case we come up with an API that
cannot be implemented efficiently, or even at all!

Thoughts?
--
Nick

On Wed, 13 Sept 2023 at 13:03, Bruno Cadonna <ca...@apache.org> wrote:

> Hi Nick,
>
> 6.
> Of course, you are right! My bad!
> Wiping out the state in the downgrading case is fine.
>
>
> 3a.
> Focus on the public facing changes for the KIP. We will manage 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.
>
>
> Best,
> Bruno
>
> On 9/13/23 11:57 AM, Nick Telford wrote:
> > Hi Bruno,
> >
> > Thanks for getting back to me!
> >
> > 2.
> > The fact that implementations can always track estimated memory usage in
> > the wrapper is a good point. I can remove -1 as an option, and I'll
> clarify
> > the JavaDoc that 0 is not just for non-transactional stores, which is
> > currently misleading.
> >
> > 6.
> > The problem with catching the exception in the downgrade process is that
> > would require new code in the Kafka version being downgraded to. Since
> > users could conceivably downgrade to almost *any* older version of Kafka
> > Streams, I'm not sure how we could add that code?
> > The only way I can think of doing it would be to provide a dedicated
> > downgrade tool, that goes through every local store and removes the
> > offsets column families. But that seems like an unnecessary amount of
> extra
> > code to maintain just to handle a somewhat niche situation, when the
> > alternative (automatically wipe and restore stores) should be acceptable.
> >
> > 1, 4, 5: Agreed. I'll make the changes you've requested.
> >
> > 3a.
> > I agree that IsolationLevel makes more sense at query-time, and I
> actually
> > initially attempted to place the IsolationLevel at query-time, but I ran
> > into some problems:
> > - The key issue is that, under ALOS we're not staging writes in
> > transactions, so can't perform writes at the READ_COMMITTED isolation
> > level. However, this may be addressed if we decide to *always* use
> > transactions as discussed under 3b.
> > - IQv1 and IQv2 have quite different implementations. I remember having
> > some difficulty understanding the IQv1 internals, which made it difficult
> > to determine what needed to be changed. However, I *think* this can be
> > addressed for both implementations by wrapping the RocksDBStore in an
> > IsolationLevel-dependent wrapper, that overrides read methods (get, etc.)
> > to either read directly from the database or from the ongoing
> transaction.
> > But IQv1 might still be difficult.
> > - If IsolationLevel becomes a query constraint, then all other
> StateStores
> > will need to respect it, including the in-memory stores. This would
> require
> > us to adapt in-memory stores to stage their writes so they can be
> isolated
> > from READ_COMMITTTED queries. It would also become an important
> > consideration for third-party stores on upgrade, as without changes, they
> > would not support READ_COMMITTED queries correctly.
> >
> > Ultimately, I may need some help making the necessary change to IQv1 to
> > support this, but I don't think it's fundamentally impossible, if we want
> > to pursue this route.
> >
> > 3b.
> > The main reason I chose to keep ALOS un-transactional was to minimize
> > behavioural change for most users (I believe most Streams users use the
> > default configuration, which is ALOS). That said, it's clear that if ALOS
> > also used transactional stores, the only change in behaviour would be
> that
> > it would become *more correct*, which could be considered a "bug fix" by
> > users, rather than a change they need to handle.
> >
> > I believe that performance using transactions (aka. RocksDB WriteBatches)
> > should actually be *better* than the un-batched write-path that is
> > currently used[1]. The only "performance" consideration will be the
> > increased memory usage that transactions require. Given the mitigations
> for
> > this memory that we have in place, I would expect that this is not a
> > problem for most users.
> >
> > If we're happy to do so, we can make ALOS also use transactions.
> >
> > Regards,
> > Nick
> >
> > Link 1:
> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >
> > On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <ca...@apache.org> wrote:
> >
> >> Hi Nick,
> >>
> >> Thanks for the updates and sorry for the delay on my side!
> >>
> >>
> >> 1.
> >> Making the default implementation for flush() a no-op sounds good to me.
> >>
> >>
> >> 2.
> >> I think what was bugging me here is that a third-party state store needs
> >> to implement the state store interface. That means they need to
> >> implement a wrapper around the actual state store as we do for RocksDB
> >> with RocksDBStore. So, a third-party state store can always estimate the
> >> uncommitted bytes, if it wants, because the wrapper can record the added
> >> bytes.
> >> One case I can think of where returning -1 makes sense is when Streams
> >> does not need to estimate the size of the write batch and trigger
> >> extraordinary commits, because the third-party state store takes care of
> >> memory. But in that case the method could also just return 0. Even that
> >> case would be better solved with a method that returns whether the state
> >> store manages itself the memory used for uncommitted bytes or not.
> >> Said that, I am fine with keeping the -1 return value, I was just
> >> wondering when and if it will be used.
> >>
> >> Regarding returning 0 for transactional state stores when the batch is
> >> empty, I was just wondering because you explicitly stated
> >>
> >> "or {@code 0} if this StateStore does not support transactions."
> >>
> >> So it seemed to me returning 0 could only happen for non-transactional
> >> state stores.
> >>
> >>
> >> 3.
> >>
> >> a) What do you think if we move the isolation level to IQ (v1 and v2)?
> >> In the end this is the only component that really needs to specify the
> >> isolation level. It is similar to the Kafka consumer that can choose
> >> with what isolation level to read the input topic.
> >> For IQv1 the isolation level should go into StoreQueryParameters. For
> >> IQv2, I would add it to the Query interface.
> >>
> >> b) Point a) raises the question what should happen during at-least-once
> >> processing when the state store does not use transactions? John in the
> >> past proposed to also use transactions on state stores for
> >> at-least-once. I like that idea, because it avoids aggregating the same
> >> records over and over again in the case of a failure. We had a case in
> >> the past where a Streams applications in at-least-once mode was failing
> >> continuously for some reasons I do not remember before committing the
> >> offsets. After each failover, the app aggregated again and again the
> >> same records. Of course the aggregate increased to very wrong values
> >> just because of the failover. With transactions on the state stores we
> >> could have avoided this. The app would have output the same aggregate
> >> multiple times (i.e., after each failover) but at least the value of the
> >> aggregate would not depend on the number of failovers. Outputting the
> >> same aggregate multiple times would be incorrect under exactly-once but
> >> it is OK for at-least-once.
> >> If it makes sense to add a config to turn on and off transactions on
> >> state stores under at-least-once or just use transactions in any case is
> >> a question we should also discuss in this KIP. It depends a bit on the
> >> performance trade-off. Maybe to be safe, I would add a config.
> >>
> >>
> >> 4.
> >> Your points are all valid. I tend to say to keep the metrics around
> >> flush() until we remove flush() completely from the interface. Calls to
> >> flush() might still exist since existing processors might still call
> >> flush() explicitly as you mentioned in 1). For sure, we need to document
> >> how the metrics change due to the transactions in the upgrade notes.
> >>
> >>
> >> 5.
> >> I see. Then you should describe how the .position files are handled  in
> >> a dedicated section of the KIP or incorporate the description in the
> >> "Atomic Checkpointing" section instead of only mentioning it in the
> >> "Compatibility, 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:
> >>
> >>
> >>
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
> >>
> >> It is a bit brittle, but it works.
> >>
> >>
> >> Best,
> >> Bruno
> >>
> >>
> >> On 8/24/23 12:18 PM, Nick Telford wrote:
> >>> Hi Bruno,
> >>>
> >>> Thanks for taking the time to review the KIP. I'm back from leave now
> and
> >>> intend to move this forwards as quickly as I can.
> >>>
> >>> Addressing your points:
> >>>
> >>> 1.
> >>> Because flush() is part of the StateStore API, it's exposed to custom
> >>> Processors, which might be making calls to flush(). This was actually
> the
> >>> case in a few integration tests.
> >>> To maintain as much compatibility as possible, I'd prefer not to make
> >> this
> >>> an UnsupportedOperationException, as it will cause previously working
> >>> Processors to start throwing exceptions at runtime.
> >>> I agree that it doesn't make sense for it to proxy commit(), though, as
> >>> that would cause it to violate the "StateStores commit only when the
> Task
> >>> commits" rule.
> >>> Instead, I think we should make this a no-op. That way, existing user
> >>> Processors will continue to work as-before, without violation of store
> >>> consistency that would be caused by premature flush/commit of
> StateStore
> >>> data to disk.
> >>> What do you think?
> >>>
> >>> 2.
> >>> As stated in the JavaDoc, when a StateStore implementation is
> >>> transactional, but is unable to estimate the uncommitted memory usage,
> >> the
> >>> method will return -1.
> >>> The intention here is to permit third-party implementations that may
> not
> >> be
> >>> able to estimate memory usage.
> >>>
> >>> Yes, it will be 0 when nothing has been written to the store yet. I
> >> thought
> >>> that was implied by "This method will return an approximation of the
> >> memory
> >>> would be freed by the next call to {@link #commit(Map)}" and "@return
> The
> >>> approximate size of all records awaiting {@link #commit(Map)}",
> however,
> >> I
> >>> can add it explicitly to the JavaDoc if you think this is unclear?
> >>>
> >>> 3.
> >>> I realise this is probably the most contentious point in my design, and
> >> I'm
> >>> open to changing it if I'm unable to convince you of the benefits.
> >>> Nevertheless, here's my argument:
> >>> The Interactive Query (IQ) API(s) are directly provided StateStores to
> >>> query, and it may be important for users to programmatically know which
> >>> mode the StateStore is operating under. If we simply provide an
> >>> "eosEnabled" boolean (as used throughout the internal streams engine),
> or
> >>> similar, then users will need to understand the operation and
> >> consequences
> >>> of each available processing mode and how it pertains to their
> >> StateStore.
> >>>
> >>> Interactive Query users aren't the only people that care about the
> >>> processing.mode/IsolationLevel of a StateStore: implementers of custom
> >>> StateStores also need to understand the behaviour expected of their
> >>> implementation. KIP-892 introduces some assumptions into the Streams
> >> Engine
> >>> about how StateStores operate under each processing mode, and it's
> >>> important that custom implementations adhere to those assumptions in
> >> order
> >>> to maintain the consistency guarantees.
> >>>
> >>> IsolationLevels provide a high-level contract on the behaviour of the
> >>> StateStore: a user knows that under READ_COMMITTED, they will see
> writes
> >>> only after the Task has committed, and under READ_UNCOMMITTED they will
> >> see
> >>> writes immediately. No understanding of the details of each
> >> processing.mode
> >>> is required, either for IQ users or StateStore implementers.
> >>>
> >>> An argument can be made that these contractual guarantees can simply be
> >>> documented for the processing.mode (i.e. that exactly-once and
> >>> exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves
> like
> >>> READ_UNCOMMITTED), but there are several small issues with this I'd
> >> prefer
> >>> to avoid:
> >>>
> >>>      - Where would we document these contracts, in a way that is
> difficult
> >>>      for users/implementers to miss/ignore?
> >>>      - It's not clear to users that the processing mode is
> communicating
> >>>      an expectation of read isolation, unless they read the
> >> documentation. Users
> >>>      rarely consult documentation unless they feel they need to, so
> it's
> >> likely
> >>>      this detail would get missed by many users.
> >>>      - It tightly couples processing modes to read isolation. Adding
> new
> >>>      processing modes, or changing the read isolation of existing
> >> processing
> >>>      modes would be difficult/impossible.
> >>>
> >>> Ultimately, the cost of introducing IsolationLevels is just a single
> >>> method, since we re-use the existing IsolationLevel enum from Kafka.
> This
> >>> gives us a clear place to document the contractual guarantees expected
> >>> of/provided by StateStores, that is accessible both by the StateStore
> >>> itself, and by IQ users.
> >>>
> >>> (Writing this I've just realised that the StateStore and IQ APIs
> actually
> >>> don't provide access to StateStoreContext that IQ users would have
> direct
> >>> access to... Perhaps StateStore should expose isolationLevel() itself
> >> too?)
> >>>
> >>> 4.
> >>> Yeah, I'm not comfortable renaming the metrics in-place either, as
> it's a
> >>> backwards incompatible change. My concern is that, if we leave the
> >> existing
> >>> "flush" metrics in place, they will be confusing to users. Right now,
> >>> "flush" metrics record explicit flushes to disk, but under KIP-892,
> even
> >> a
> >>> commit() will not explicitly flush data to disk - RocksDB will decide
> on
> >>> when to flush memtables to disk itself.
> >>>
> >>> If we keep the existing "flush" metrics, we'd have two options, which
> >> both
> >>> seem pretty bad to me:
> >>>
> >>>      1. Have them record calls to commit(), which would be misleading,
> as
> >>>      data is no longer explicitly "flushed" to disk by this call.
> >>>      2. Have them record nothing at all, which is equivalent to
> removing
> >> the
> >>>      metrics, except that users will see the metric still exists and so
> >> assume
> >>>      that the metric is correct, and that there's a problem with their
> >> system
> >>>      when there isn't.
> >>>
> >>> I agree that removing them is also a bad solution, and I'd like some
> >>> guidance on the best path forward here.
> >>>
> >>> 5.
> >>> Position files are updated on every write to a StateStore. Since our
> >> writes
> >>> are now buffered until commit(), we can't update the Position file
> until
> >>> commit() has been called, otherwise it would be inconsistent with the
> >> data
> >>> in the event of a rollback. Consequently, we need to manage these
> offsets
> >>> the same way we manage the checkpoint offsets, and ensure they're only
> >>> written on commit().
> >>>
> >>> 6.
> >>> Agreed, although I'm not exactly sure yet what tests to write. How
> >> explicit
> >>> do we need to be here in the KIP?
> >>>
> >>> As for upgrade/downgrade: upgrade is designed to be seamless, and we
> >> should
> >>> definitely add some tests around that. Downgrade, it transpires, isn't
> >>> currently possible, as the extra column family for offset storage is
> >>> incompatible with the pre-KIP-892 implementation: when you open a
> RocksDB
> >>> database, you must open all available column families or receive an
> >> error.
> >>> What currently happens on downgrade is that it attempts to open the
> >> store,
> >>> throws an error about the offsets column family not being opened, which
> >>> triggers a wipe and rebuild of the Task. Given that downgrades should
> be
> >>> uncommon, I think this is acceptable behaviour, as the end-state is
> >>> consistent, even if it results in an undesirable state restore.
> >>>
> >>> Should I document the upgrade/downgrade behaviour explicitly in the
> KIP?
> >>>
> >>> --
> >>>
> >>> Regards,
> >>> Nick
> >>>
> >>>
> >>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org>
> wrote:
> >>>
> >>>> Hi Nick!
> >>>>
> >>>> Thanks for the updates!
> >>>>
> >>>> 1.
> >>>> Why does StateStore#flush() default to
> >>>> StateStore#commit(Collections.emptyMap())?
> >>>> Since calls to flush() will not exist anymore after this KIP is
> >>>> released, I would rather throw an unsupported operation exception by
> >>>> default.
> >>>>
> >>>>
> >>>> 2.
> >>>> When would a state store return -1 from
> >>>> StateStore#approximateNumUncommittedBytes() while being transactional?
> >>>>
> >>>> Wouldn't StateStore#approximateNumUncommittedBytes() also return 0 if
> >>>> the state store is transactional but nothing has been written to the
> >>>> state store yet?
> >>>>
> >>>>
> >>>> 3.
> >>>> Sorry for bringing this up again. Does this KIP really need to
> introduce
> >>>> StateStoreContext#isolationLevel()? StateStoreContext has already
> >>>> appConfigs() which basically exposes the same information, i.e., if
> EOS
> >>>> is enabled or not.
> >>>> In one of your previous e-mails you wrote:
> >>>>
> >>>> "My idea was to try to keep the StateStore interface as loosely
> coupled
> >>>> from the Streams engine as possible, to give implementers more
> freedom,
> >>>> and reduce the amount of internal knowledge required."
> >>>>
> >>>> While I understand the intent, I doubt that it decreases the coupling
> of
> >>>> a StateStore interface and the Streams engine. READ_COMMITTED only
> >>>> applies to IQ but not to reads by processors. Thus, implementers need
> to
> >>>> understand how Streams accesses the state stores.
> >>>>
> >>>> I would like to hear what others think about this.
> >>>>
> >>>>
> >>>> 4.
> >>>> Great exposing new metrics for transactional state stores! However, I
> >>>> would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides unit
> >>>> tests. Additionally, we probably need integration or system tests to
> >>>> verify that upgrades and downgrades between transactional and
> >>>> non-transactional state stores work as expected.
> >>>>
> >>>>
> >>>> Best,
> >>>> Bruno
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> On 7/21/23 10:34 PM, Nick Telford wrote:
> >>>>> One more thing: I noted John's suggestion in the KIP, under "Rejected
> >>>>> Alternatives". I still think it's an idea worth pursuing, but I
> believe
> >>>>> that it's out of the scope of this KIP, because it solves a different
> >> set
> >>>>> of problems to this KIP, and the scope of this one has already grown
> >>>> quite
> >>>>> large!
> >>>>>
> >>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <ni...@gmail.com>
> >>>> wrote:
> >>>>>
> >>>>>> Hi everyone,
> >>>>>>
> >>>>>> I've updated the KIP (
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>> )
> >>>>>> with the latest changes; mostly bringing back "Atomic Checkpointing"
> >>>> (for
> >>>>>> what feels like the 10th time!). I think the one thing missing is
> some
> >>>>>> changes to metrics (notably the store "flush" metrics will need to
> be
> >>>>>> renamed to "commit").
> >>>>>>
> >>>>>> The reason I brought back Atomic Checkpointing was to decouple store
> >>>> flush
> >>>>>> from store commit. This is important, because with Transactional
> >>>>>> StateStores, we now need to call "flush" on *every* Task commit, and
> >> not
> >>>>>> just when the StateStore is closing, otherwise our transaction
> buffer
> >>>> will
> >>>>>> never be written and persisted, instead growing unbounded! I
> >>>> experimented
> >>>>>> with some simple solutions, like forcing a store flush whenever the
> >>>>>> transaction buffer was likely to exceed its configured size, but
> this
> >>>> was
> >>>>>> brittle: it prevented the transaction buffer from being configured
> to
> >> be
> >>>>>> unbounded, and it still would have required explicit flushes of
> >> RocksDB,
> >>>>>> yielding sub-optimal performance and memory utilization.
> >>>>>>
> >>>>>> I deemed Atomic Checkpointing to be the "right" way to resolve this
> >>>>>> problem. By ensuring that the changelog offsets that correspond to
> the
> >>>> most
> >>>>>> recently written records are always atomically written to the
> >> StateStore
> >>>>>> (by writing them to the same transaction buffer), we can avoid
> >> forcibly
> >>>>>> flushing the RocksDB memtables to disk, letting RocksDB flush them
> >> only
> >>>>>> when necessary, without losing any of our consistency guarantees.
> See
> >>>> the
> >>>>>> updated KIP for more info.
> >>>>>>
> >>>>>> I have fully implemented these changes, although I'm still not
> >> entirely
> >>>>>> happy with the implementation for segmented StateStores, so I plan
> to
> >>>>>> refactor that. Despite that, all tests pass. If you'd like to try
> out
> >> or
> >>>>>> review this highly experimental and incomplete branch, it's
> available
> >>>> here:
> >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note: it's
> >>>> built
> >>>>>> against Kafka 3.5.0 so that I had a stable base to build and test it
> >> on,
> >>>>>> and to enable easy apples-to-apples comparisons in a live
> >> environment. I
> >>>>>> plan to rebase it against trunk once it's nearer completion and has
> >> been
> >>>>>> proven on our main application.
> >>>>>>
> >>>>>> I would really appreciate help in reviewing and testing:
> >>>>>> - Segmented (Versioned, Session and Window) stores
> >>>>>> - Global stores
> >>>>>>
> >>>>>> As I do not currently use either of these, so my primary test
> >>>> environment
> >>>>>> doesn't test these areas.
> >>>>>>
> >>>>>> I'm going on Parental Leave starting next week for a few weeks, so
> >> will
> >>>>>> not have time to move this forward until late August. That said,
> your
> >>>>>> feedback is welcome and appreciated, I just won't be able to respond
> >> as
> >>>>>> quickly as usual.
> >>>>>>
> >>>>>> Regards,
> >>>>>> Nick
> >>>>>>
> >>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com>
> >>>> wrote:
> >>>>>>
> >>>>>>> Hi Bruno
> >>>>>>>
> >>>>>>> Yes, that's correct, although the impact on IQ is not something I
> had
> >>>>>>> considered.
> >>>>>>>
> >>>>>>> What about atomically updating the state store from the transaction
> >>>>>>>> buffer every commit interval and writing the checkpoint (thus,
> >>>> flushing
> >>>>>>>> the memtable) every configured amount of data and/or number of
> >> commit
> >>>>>>>> intervals?
> >>>>>>>>
> >>>>>>>
> >>>>>>> I'm not quite sure I follow. Are you suggesting that we add an
> >>>> additional
> >>>>>>> config for the max number of commit intervals between checkpoints?
> >> That
> >>>>>>> way, we would checkpoint *either* when the transaction buffers are
> >>>> nearly
> >>>>>>> full, *OR* whenever a certain number of commit intervals have
> >> elapsed,
> >>>>>>> whichever comes first?
> >>>>>>>
> >>>>>>> That certainly seems reasonable, although this re-ignites an
> earlier
> >>>>>>> debate about whether a config should be measured in "number of
> commit
> >>>>>>> intervals", instead of just an absolute time.
> >>>>>>>
> >>>>>>> FWIW, I realised that this issue is the reason I was pursuing the
> >>>> Atomic
> >>>>>>> Checkpoints, as it de-couples memtable flush from checkpointing,
> >> which
> >>>>>>> enables us to just checkpoint on every commit without any
> performance
> >>>>>>> impact. Atomic Checkpointing is definitely the "best" solution, but
> >>>> I'm not
> >>>>>>> sure if this is enough to bring it back into this KIP.
> >>>>>>>
> >>>>>>> I'm currently working on moving all the transactional logic
> directly
> >>>> into
> >>>>>>> RocksDBStore itself, which does away with the
> >> StateStore#newTransaction
> >>>>>>> method, and reduces the number of new classes introduced,
> >> significantly
> >>>>>>> reducing the complexity. If it works, and the complexity is
> >> drastically
> >>>>>>> reduced, I may try bringing back Atomic Checkpoints into this KIP.
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>> Nick
> >>>>>>>
> >>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org>
> >> wrote:
> >>>>>>>
> >>>>>>>> Hi Nick,
> >>>>>>>>
> >>>>>>>> Thanks for the insights! Very interesting!
> >>>>>>>>
> >>>>>>>> As far as I understand, you want to atomically update the state
> >> store
> >>>>>>>> from the transaction buffer, flush the memtable of a state store
> and
> >>>>>>>> write the checkpoint not after the commit time elapsed but after
> the
> >>>>>>>> transaction buffer reached a size that would lead to exceeding
> >>>>>>>> statestore.transaction.buffer.max.bytes before the next commit
> >>>> interval
> >>>>>>>> ends.
> >>>>>>>> That means, the Kafka transaction would commit every commit
> interval
> >>>> but
> >>>>>>>> the state store will only be atomically updated roughly every
> >>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ would
> then
> >>>> only
> >>>>>>>> see new data roughly every
> statestore.transaction.buffer.max.bytes.
> >>>>>>>> After a failure the state store needs to restore up to
> >>>>>>>> statestore.transaction.buffer.max.bytes.
> >>>>>>>>
> >>>>>>>> Is this correct?
> >>>>>>>>
> >>>>>>>> What about atomically updating the state store from the
> transaction
> >>>>>>>> buffer every commit interval and writing the checkpoint (thus,
> >>>> flushing
> >>>>>>>> the memtable) every configured amount of data and/or number of
> >> commit
> >>>>>>>> intervals? In such a way, we would have the same delay for records
> >>>>>>>> appearing in output topics and IQ because both would appear when
> the
> >>>>>>>> Kafka transaction is committed. However, after a failure the state
> >>>> store
> >>>>>>>> still needs to restore up to
> statestore.transaction.buffer.max.bytes
> >>>> and
> >>>>>>>> it might restore data that is already in the state store because
> the
> >>>>>>>> checkpoint lags behind the last stable offset (i.e. the last
> >> committed
> >>>>>>>> offset) of the changelog topics. Restoring data that is already in
> >> the
> >>>>>>>> state store is idempotent, so eos should not violated.
> >>>>>>>> This solution needs at least one new config to specify when a
> >>>> checkpoint
> >>>>>>>> should be written.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> A small correction to your previous e-mail that does not change
> >>>> anything
> >>>>>>>> you said: Under alos the default commit interval is 30 seconds,
> not
> >>>> five
> >>>>>>>> seconds.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Best,
> >>>>>>>> Bruno
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
> >>>>>>>>> Hi everyone,
> >>>>>>>>>
> >>>>>>>>> I've begun performance testing my branch on our staging
> >> environment,
> >>>>>>>>> putting it through its paces in our non-trivial application. I'm
> >>>>>>>> already
> >>>>>>>>> observing the same increased flush rate that we saw the last time
> >> we
> >>>>>>>>> attempted to use a version of this KIP, but this time, I think I
> >> know
> >>>>>>>> why.
> >>>>>>>>>
> >>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end of
> >> the
> >>>>>>>> Task
> >>>>>>>>> commit process, has the following behaviour:
> >>>>>>>>>
> >>>>>>>>>        - Under ALOS: checkpoint the state stores. This includes
> >>>>>>>>>        flushing memtables in RocksDB. This is acceptable because
> the
> >>>>>>>> default
> >>>>>>>>>        commit.interval.ms is 5 seconds, so forcibly flushing
> >> memtables
> >>>>>>>> every 5
> >>>>>>>>>        seconds is acceptable for most applications.
> >>>>>>>>>        - Under EOS: checkpointing is not done, *unless* it's
> being
> >>>>>>>> forced, due
> >>>>>>>>>        to e.g. the Task closing or being revoked. This means that
> >> under
> >>>>>>>> normal
> >>>>>>>>>        processing conditions, the state stores will not be
> >>>> checkpointed,
> >>>>>>>> and will
> >>>>>>>>>        not have memtables flushed at all , unless RocksDB
> decides to
> >>>>>>>> flush them on
> >>>>>>>>>        its own. Checkpointing stores and force-flushing their
> >> memtables
> >>>>>>>> is only
> >>>>>>>>>        done when a Task is being closed.
> >>>>>>>>>
> >>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least *some*
> >>>> normal
> >>>>>>>>> Task commits, in order to write the RocksDB transaction buffers
> to
> >>>> the
> >>>>>>>>> state stores, and to ensure the offsets are synced to disk to
> >> prevent
> >>>>>>>>> restores from getting out of hand. Consequently, my current
> >>>>>>>> implementation
> >>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is far too
> >>>>>>>> frequent.
> >>>>>>>>> This causes checkpoints every 10,000 records, which is a change
> in
> >>>>>>>> flush
> >>>>>>>>> behaviour, potentially causing performance problems for some
> >>>>>>>> applications.
> >>>>>>>>>
> >>>>>>>>> I'm looking into possible solutions, and I'm currently leaning
> >>>> towards
> >>>>>>>>> using the statestore.transaction.buffer.max.bytes configuration
> to
> >>>>>>>>> checkpoint Tasks once we are likely to exceed it. This would
> >>>>>>>> complement the
> >>>>>>>>> existing "early Task commit" functionality that this
> configuration
> >>>>>>>>> provides, in the following way:
> >>>>>>>>>
> >>>>>>>>>        - Currently, we use
> statestore.transaction.buffer.max.bytes
> >> to
> >>>>>>>> force an
> >>>>>>>>>        early Task commit if processing more records would cause
> our
> >>>> state
> >>>>>>>> store
> >>>>>>>>>        transactions to exceed the memory assigned to them.
> >>>>>>>>>        - New functionality: when a Task *does* commit, we will
> not
> >>>>>>>> checkpoint
> >>>>>>>>>        the stores (and hence flush the transaction buffers)
> unless
> >> we
> >>>>>>>> expect to
> >>>>>>>>>        cross the statestore.transaction.buffer.max.bytes
> threshold
> >>>> before
> >>>>>>>> the next
> >>>>>>>>>        commit
> >>>>>>>>>
> >>>>>>>>> I'm also open to suggestions.
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>> Nick
> >>>>>>>>>
> >>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> nick.telford@gmail.com
> >>>
> >>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi Bruno!
> >>>>>>>>>>
> >>>>>>>>>> 3.
> >>>>>>>>>> By "less predictable for users", I meant in terms of
> understanding
> >>>> the
> >>>>>>>>>> performance profile under various circumstances. The more
> complex
> >>>> the
> >>>>>>>>>> solution, the more difficult it would be for users to understand
> >> the
> >>>>>>>>>> performance they see. For example, spilling records to disk when
> >> the
> >>>>>>>>>> transaction buffer reaches a threshold would, I expect, reduce
> >> write
> >>>>>>>>>> throughput. This reduction in write throughput could be
> >> unexpected,
> >>>>>>>> and
> >>>>>>>>>> potentially difficult to diagnose/understand for users.
> >>>>>>>>>> At the moment, I think the "early commit" concept is relatively
> >>>>>>>>>> straightforward; it's easy to document, and conceptually fairly
> >>>>>>>> obvious to
> >>>>>>>>>> users. We could probably add a metric to make it easier to
> >>>> understand
> >>>>>>>> when
> >>>>>>>>>> it happens though.
> >>>>>>>>>>
> >>>>>>>>>> 3. (the second one)
> >>>>>>>>>> The IsolationLevel is *essentially* an indirect way of telling
> >>>>>>>> StateStores
> >>>>>>>>>> whether they should be transactional. READ_COMMITTED essentially
> >>>>>>>> requires
> >>>>>>>>>> transactions, because it dictates that two threads calling
> >>>>>>>>>> `newTransaction()` should not see writes from the other
> >> transaction
> >>>>>>>> until
> >>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all bets are
> off,
> >>>> and
> >>>>>>>>>> stores can allow threads to observe written records at any time,
> >>>>>>>> which is
> >>>>>>>>>> essentially "no transactions". That said, StateStores are free
> to
> >>>>>>>> implement
> >>>>>>>>>> these guarantees however they can, which is a bit more relaxed
> >> than
> >>>>>>>>>> dictating "you must use transactions". For example, with RocksDB
> >> we
> >>>>>>>> would
> >>>>>>>>>> implement these as READ_COMMITTED == WBWI-based "transactions",
> >>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But with
> other
> >>>>>>>> storage
> >>>>>>>>>> engines, it might be preferable to *always* use transactions,
> even
> >>>>>>>> when
> >>>>>>>>>> unnecessary; or there may be storage engines that don't provide
> >>>>>>>>>> transactions, but the isolation guarantees can be met using a
> >>>>>>>> different
> >>>>>>>>>> technique.
> >>>>>>>>>> My idea was to try to keep the StateStore interface as loosely
> >>>> coupled
> >>>>>>>>>> from the Streams engine as possible, to give implementers more
> >>>>>>>> freedom, and
> >>>>>>>>>> reduce the amount of internal knowledge required.
> >>>>>>>>>> That said, I understand that "IsolationLevel" might not be the
> >> right
> >>>>>>>>>> abstraction, and we can always make it much more explicit if
> >>>>>>>> required, e.g.
> >>>>>>>>>> boolean transactional()
> >>>>>>>>>>
> >>>>>>>>>> 7-8.
> >>>>>>>>>> I can make these changes either later today or tomorrow.
> >>>>>>>>>>
> >>>>>>>>>> Small update:
> >>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of issues that
> >>>>>>>> needed
> >>>>>>>>>> addressing. Currently, all the tests pass, which is promising,
> but
> >>>> it
> >>>>>>>> will
> >>>>>>>>>> need to undergo some performance testing. I haven't (yet) worked
> >> on
> >>>>>>>>>> removing the `newTransaction()` stuff, but I would expect that,
> >>>>>>>>>> behaviourally, it should make no difference. The branch is
> >> available
> >>>>>>>> at
> >>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone
> is
> >>>>>>>>>> interested in taking an early look.
> >>>>>>>>>>
> >>>>>>>>>> Regards,
> >>>>>>>>>> Nick
> >>>>>>>>>>
> >>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <cadonna@apache.org
> >
> >>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>
> >>>>>>>>>>> 1.
> >>>>>>>>>>> Yeah, I agree with you. That was actually also my point. I
> >>>> understood
> >>>>>>>>>>> that John was proposing the ingestion path as a way to avoid
> the
> >>>>>>>> early
> >>>>>>>>>>> commits. Probably, I misinterpreted the intent.
> >>>>>>>>>>>
> >>>>>>>>>>> 2.
> >>>>>>>>>>> I agree with John here, that actually it is public API. My
> >> question
> >>>>>>>> is
> >>>>>>>>>>> how this usage pattern affects normal processing.
> >>>>>>>>>>>
> >>>>>>>>>>> 3.
> >>>>>>>>>>> My concern is that checking for the size of the transaction
> >> buffer
> >>>>>>>> and
> >>>>>>>>>>> maybe triggering an early commit affects the whole processing
> of
> >>>>>>>> Kafka
> >>>>>>>>>>> Streams. The transactionality of a state store is not confined
> to
> >>>> the
> >>>>>>>>>>> state store itself, but spills over and changes the behavior of
> >>>> other
> >>>>>>>>>>> parts of the system. I agree with you that it is a decent
> >>>>>>>> compromise. I
> >>>>>>>>>>> just wanted to analyse the downsides and list the options to
> >>>> overcome
> >>>>>>>>>>> them. I also agree with you that all options seem quite heavy
> >>>>>>>> compared
> >>>>>>>>>>> with your KIP. I do not understand what you mean with "less
> >>>>>>>> predictable
> >>>>>>>>>>> for users", though.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> I found the discussions about the alternatives really
> >> interesting.
> >>>>>>>> But I
> >>>>>>>>>>> also think that your plan sounds good and we should continue
> with
> >>>> it!
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
> >>>>>>>>>>>
> >>>>>>>>>>> 3.
> >>>>>>>>>>> Ah, now, I understand the reasoning behind putting isolation
> >> level
> >>>> in
> >>>>>>>>>>> the state store context. Thanks! Should that also be a way to
> >> give
> >>>>>>>> the
> >>>>>>>>>>> the state store the opportunity to decide whether to turn on
> >>>>>>>>>>> transactions or not?
> >>>>>>>>>>> With my comment, I was more concerned about how do you know if
> a
> >>>>>>>>>>> checkpoint file needs to be written under EOS, if you do not
> >> have a
> >>>>>>>> way
> >>>>>>>>>>> to know if the state store is transactional or not. If a state
> >>>> store
> >>>>>>>> is
> >>>>>>>>>>> transactional, the checkpoint file can be written during normal
> >>>>>>>>>>> processing under EOS. If the state store is not transactional,
> >> the
> >>>>>>>>>>> checkpoint file must not be written under EOS.
> >>>>>>>>>>>
> >>>>>>>>>>> 7.
> >>>>>>>>>>> My point was about not only considering the bytes in memory in
> >>>> config
> >>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that might be
> >>>>>>>> spilled
> >>>>>>>>>>> on disk. Basically, I was wondering whether you should remove
> the
> >>>>>>>>>>> "memory" in "Maximum number of memory bytes to be used to
> >>>>>>>>>>> buffer uncommitted state-store records." My thinking was that
> >> even
> >>>>>>>> if a
> >>>>>>>>>>> state store spills uncommitted bytes to disk, limiting the
> >> overall
> >>>>>>>> bytes
> >>>>>>>>>>> might make sense. Thinking about it again and considering the
> >>>> recent
> >>>>>>>>>>> discussions, it does not make too much sense anymore.
> >>>>>>>>>>> I like the name statestore.transaction.buffer.max.bytes that
> you
> >>>>>>>> proposed.
> >>>>>>>>>>>
> >>>>>>>>>>> 8.
> >>>>>>>>>>> A high-level description (without implementation details) of
> how
> >>>>>>>> Kafka
> >>>>>>>>>>> Streams will manage the commit of changelog transactions, state
> >>>> store
> >>>>>>>>>>> transactions and checkpointing would be great. Would be great
> if
> >>>> you
> >>>>>>>>>>> could also add some sentences about the behavior in case of a
> >>>>>>>> failure.
> >>>>>>>>>>> For instance how does a transactional state store recover
> after a
> >>>>>>>>>>> failure or what happens with the transaction buffer, etc. (that
> >> is
> >>>>>>>> what
> >>>>>>>>>>> I meant by "fail-over" in point 9.)
> >>>>>>>>>>>
> >>>>>>>>>>> Best,
> >>>>>>>>>>> Bruno
> >>>>>>>>>>>
> >>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> >>>>>>>>>>>> Hi Bruno,
> >>>>>>>>>>>>
> >>>>>>>>>>>> 1.
> >>>>>>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
> >>>>>>>> transactions
> >>>>>>>>>>>> have, whereby exceeding (or likely to exceed) configured
> memory
> >>>>>>>> needs to
> >>>>>>>>>>>> trigger an early commit?
> >>>>>>>>>>>>
> >>>>>>>>>>>> 2.
> >>>>>>>>>>>> This is one of my big concerns. Ultimately, any approach based
> >> on
> >>>>>>>>>>> cracking
> >>>>>>>>>>>> open RocksDB internals and using it in ways it's not really
> >>>> designed
> >>>>>>>>>>> for is
> >>>>>>>>>>>> likely to have some unforseen performance or consistency
> issues.
> >>>>>>>>>>>>
> >>>>>>>>>>>> 3.
> >>>>>>>>>>>> What's your motivation for removing these early commits? While
> >> not
> >>>>>>>>>>> ideal, I
> >>>>>>>>>>>> think they're a decent compromise to ensure consistency whilst
> >>>>>>>>>>> maintaining
> >>>>>>>>>>>> good and predictable performance.
> >>>>>>>>>>>> All 3 of your suggested ideas seem *very* complicated, and
> might
> >>>>>>>>>>> actually
> >>>>>>>>>>>> make behaviour less predictable for users as a consequence.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is growing a
> bit
> >>>> out
> >>>>>>>> of
> >>>>>>>>>>>> control. While it's good to discuss ideas for future
> >>>> improvements, I
> >>>>>>>>>>> think
> >>>>>>>>>>>> it's important to narrow the scope down to a design that
> >> achieves
> >>>>>>>> the
> >>>>>>>>>>> most
> >>>>>>>>>>>> pressing objectives (constant sized restorations during dirty
> >>>>>>>>>>>> close/unexpected errors). Any design that this KIP produces
> can
> >>>>>>>>>>> ultimately
> >>>>>>>>>>>> be changed in the future, especially if the bulk of it is
> >> internal
> >>>>>>>>>>>> behaviour.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I'm going to spend some time next week trying to re-work the
> >>>>>>>> original
> >>>>>>>>>>>> WriteBatchWithIndex design to remove the newTransaction()
> >> method,
> >>>>>>>> such
> >>>>>>>>>>> that
> >>>>>>>>>>>> it's just an implementation detail of RocksDBStore. That way,
> if
> >>>> we
> >>>>>>>>>>> want to
> >>>>>>>>>>>> replace WBWI with something in the future, like the SST file
> >>>>>>>> management
> >>>>>>>>>>>> outlined by John, then we can do so with little/no API
> changes.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Regards,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Nick
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>

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

Posted by Bruno Cadonna <ca...@apache.org>.
Hi Nick,

6.
Of course, you are right! My bad!
Wiping out the state in the downgrading case is fine.


3a.
Focus on the public facing changes for the KIP. We will manage 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.


Best,
Bruno

On 9/13/23 11:57 AM, Nick Telford wrote:
> Hi Bruno,
> 
> Thanks for getting back to me!
> 
> 2.
> The fact that implementations can always track estimated memory usage in
> the wrapper is a good point. I can remove -1 as an option, and I'll clarify
> the JavaDoc that 0 is not just for non-transactional stores, which is
> currently misleading.
> 
> 6.
> The problem with catching the exception in the downgrade process is that
> would require new code in the Kafka version being downgraded to. Since
> users could conceivably downgrade to almost *any* older version of Kafka
> Streams, I'm not sure how we could add that code?
> The only way I can think of doing it would be to provide a dedicated
> downgrade tool, that goes through every local store and removes the
> offsets column families. But that seems like an unnecessary amount of extra
> code to maintain just to handle a somewhat niche situation, when the
> alternative (automatically wipe and restore stores) should be acceptable.
> 
> 1, 4, 5: Agreed. I'll make the changes you've requested.
> 
> 3a.
> I agree that IsolationLevel makes more sense at query-time, and I actually
> initially attempted to place the IsolationLevel at query-time, but I ran
> into some problems:
> - The key issue is that, under ALOS we're not staging writes in
> transactions, so can't perform writes at the READ_COMMITTED isolation
> level. However, this may be addressed if we decide to *always* use
> transactions as discussed under 3b.
> - IQv1 and IQv2 have quite different implementations. I remember having
> some difficulty understanding the IQv1 internals, which made it difficult
> to determine what needed to be changed. However, I *think* this can be
> addressed for both implementations by wrapping the RocksDBStore in an
> IsolationLevel-dependent wrapper, that overrides read methods (get, etc.)
> to either read directly from the database or from the ongoing transaction.
> But IQv1 might still be difficult.
> - If IsolationLevel becomes a query constraint, then all other StateStores
> will need to respect it, including the in-memory stores. This would require
> us to adapt in-memory stores to stage their writes so they can be isolated
> from READ_COMMITTTED queries. It would also become an important
> consideration for third-party stores on upgrade, as without changes, they
> would not support READ_COMMITTED queries correctly.
> 
> Ultimately, I may need some help making the necessary change to IQv1 to
> support this, but I don't think it's fundamentally impossible, if we want
> to pursue this route.
> 
> 3b.
> The main reason I chose to keep ALOS un-transactional was to minimize
> behavioural change for most users (I believe most Streams users use the
> default configuration, which is ALOS). That said, it's clear that if ALOS
> also used transactional stores, the only change in behaviour would be that
> it would become *more correct*, which could be considered a "bug fix" by
> users, rather than a change they need to handle.
> 
> I believe that performance using transactions (aka. RocksDB WriteBatches)
> should actually be *better* than the un-batched write-path that is
> currently used[1]. The only "performance" consideration will be the
> increased memory usage that transactions require. Given the mitigations for
> this memory that we have in place, I would expect that this is not a
> problem for most users.
> 
> If we're happy to do so, we can make ALOS also use transactions.
> 
> Regards,
> Nick
> 
> Link 1:
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> 
> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <ca...@apache.org> wrote:
> 
>> Hi Nick,
>>
>> Thanks for the updates and sorry for the delay on my side!
>>
>>
>> 1.
>> Making the default implementation for flush() a no-op sounds good to me.
>>
>>
>> 2.
>> I think what was bugging me here is that a third-party state store needs
>> to implement the state store interface. That means they need to
>> implement a wrapper around the actual state store as we do for RocksDB
>> with RocksDBStore. So, a third-party state store can always estimate the
>> uncommitted bytes, if it wants, because the wrapper can record the added
>> bytes.
>> One case I can think of where returning -1 makes sense is when Streams
>> does not need to estimate the size of the write batch and trigger
>> extraordinary commits, because the third-party state store takes care of
>> memory. But in that case the method could also just return 0. Even that
>> case would be better solved with a method that returns whether the state
>> store manages itself the memory used for uncommitted bytes or not.
>> Said that, I am fine with keeping the -1 return value, I was just
>> wondering when and if it will be used.
>>
>> Regarding returning 0 for transactional state stores when the batch is
>> empty, I was just wondering because you explicitly stated
>>
>> "or {@code 0} if this StateStore does not support transactions."
>>
>> So it seemed to me returning 0 could only happen for non-transactional
>> state stores.
>>
>>
>> 3.
>>
>> a) What do you think if we move the isolation level to IQ (v1 and v2)?
>> In the end this is the only component that really needs to specify the
>> isolation level. It is similar to the Kafka consumer that can choose
>> with what isolation level to read the input topic.
>> For IQv1 the isolation level should go into StoreQueryParameters. For
>> IQv2, I would add it to the Query interface.
>>
>> b) Point a) raises the question what should happen during at-least-once
>> processing when the state store does not use transactions? John in the
>> past proposed to also use transactions on state stores for
>> at-least-once. I like that idea, because it avoids aggregating the same
>> records over and over again in the case of a failure. We had a case in
>> the past where a Streams applications in at-least-once mode was failing
>> continuously for some reasons I do not remember before committing the
>> offsets. After each failover, the app aggregated again and again the
>> same records. Of course the aggregate increased to very wrong values
>> just because of the failover. With transactions on the state stores we
>> could have avoided this. The app would have output the same aggregate
>> multiple times (i.e., after each failover) but at least the value of the
>> aggregate would not depend on the number of failovers. Outputting the
>> same aggregate multiple times would be incorrect under exactly-once but
>> it is OK for at-least-once.
>> If it makes sense to add a config to turn on and off transactions on
>> state stores under at-least-once or just use transactions in any case is
>> a question we should also discuss in this KIP. It depends a bit on the
>> performance trade-off. Maybe to be safe, I would add a config.
>>
>>
>> 4.
>> Your points are all valid. I tend to say to keep the metrics around
>> flush() until we remove flush() completely from the interface. Calls to
>> flush() might still exist since existing processors might still call
>> flush() explicitly as you mentioned in 1). For sure, we need to document
>> how the metrics change due to the transactions in the upgrade notes.
>>
>>
>> 5.
>> I see. Then you should describe how the .position files are handled  in
>> a dedicated section of the KIP or incorporate the description in the
>> "Atomic Checkpointing" section instead of only mentioning it in the
>> "Compatibility, 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:
>>
>>
>> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
>>
>> It is a bit brittle, but it works.
>>
>>
>> Best,
>> Bruno
>>
>>
>> On 8/24/23 12:18 PM, Nick Telford wrote:
>>> Hi Bruno,
>>>
>>> Thanks for taking the time to review the KIP. I'm back from leave now and
>>> intend to move this forwards as quickly as I can.
>>>
>>> Addressing your points:
>>>
>>> 1.
>>> Because flush() is part of the StateStore API, it's exposed to custom
>>> Processors, which might be making calls to flush(). This was actually the
>>> case in a few integration tests.
>>> To maintain as much compatibility as possible, I'd prefer not to make
>> this
>>> an UnsupportedOperationException, as it will cause previously working
>>> Processors to start throwing exceptions at runtime.
>>> I agree that it doesn't make sense for it to proxy commit(), though, as
>>> that would cause it to violate the "StateStores commit only when the Task
>>> commits" rule.
>>> Instead, I think we should make this a no-op. That way, existing user
>>> Processors will continue to work as-before, without violation of store
>>> consistency that would be caused by premature flush/commit of StateStore
>>> data to disk.
>>> What do you think?
>>>
>>> 2.
>>> As stated in the JavaDoc, when a StateStore implementation is
>>> transactional, but is unable to estimate the uncommitted memory usage,
>> the
>>> method will return -1.
>>> The intention here is to permit third-party implementations that may not
>> be
>>> able to estimate memory usage.
>>>
>>> Yes, it will be 0 when nothing has been written to the store yet. I
>> thought
>>> that was implied by "This method will return an approximation of the
>> memory
>>> would be freed by the next call to {@link #commit(Map)}" and "@return The
>>> approximate size of all records awaiting {@link #commit(Map)}", however,
>> I
>>> can add it explicitly to the JavaDoc if you think this is unclear?
>>>
>>> 3.
>>> I realise this is probably the most contentious point in my design, and
>> I'm
>>> open to changing it if I'm unable to convince you of the benefits.
>>> Nevertheless, here's my argument:
>>> The Interactive Query (IQ) API(s) are directly provided StateStores to
>>> query, and it may be important for users to programmatically know which
>>> mode the StateStore is operating under. If we simply provide an
>>> "eosEnabled" boolean (as used throughout the internal streams engine), or
>>> similar, then users will need to understand the operation and
>> consequences
>>> of each available processing mode and how it pertains to their
>> StateStore.
>>>
>>> Interactive Query users aren't the only people that care about the
>>> processing.mode/IsolationLevel of a StateStore: implementers of custom
>>> StateStores also need to understand the behaviour expected of their
>>> implementation. KIP-892 introduces some assumptions into the Streams
>> Engine
>>> about how StateStores operate under each processing mode, and it's
>>> important that custom implementations adhere to those assumptions in
>> order
>>> to maintain the consistency guarantees.
>>>
>>> IsolationLevels provide a high-level contract on the behaviour of the
>>> StateStore: a user knows that under READ_COMMITTED, they will see writes
>>> only after the Task has committed, and under READ_UNCOMMITTED they will
>> see
>>> writes immediately. No understanding of the details of each
>> processing.mode
>>> is required, either for IQ users or StateStore implementers.
>>>
>>> An argument can be made that these contractual guarantees can simply be
>>> documented for the processing.mode (i.e. that exactly-once and
>>> exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves like
>>> READ_UNCOMMITTED), but there are several small issues with this I'd
>> prefer
>>> to avoid:
>>>
>>>      - Where would we document these contracts, in a way that is difficult
>>>      for users/implementers to miss/ignore?
>>>      - It's not clear to users that the processing mode is communicating
>>>      an expectation of read isolation, unless they read the
>> documentation. Users
>>>      rarely consult documentation unless they feel they need to, so it's
>> likely
>>>      this detail would get missed by many users.
>>>      - It tightly couples processing modes to read isolation. Adding new
>>>      processing modes, or changing the read isolation of existing
>> processing
>>>      modes would be difficult/impossible.
>>>
>>> Ultimately, the cost of introducing IsolationLevels is just a single
>>> method, since we re-use the existing IsolationLevel enum from Kafka. This
>>> gives us a clear place to document the contractual guarantees expected
>>> of/provided by StateStores, that is accessible both by the StateStore
>>> itself, and by IQ users.
>>>
>>> (Writing this I've just realised that the StateStore and IQ APIs actually
>>> don't provide access to StateStoreContext that IQ users would have direct
>>> access to... Perhaps StateStore should expose isolationLevel() itself
>> too?)
>>>
>>> 4.
>>> Yeah, I'm not comfortable renaming the metrics in-place either, as it's a
>>> backwards incompatible change. My concern is that, if we leave the
>> existing
>>> "flush" metrics in place, they will be confusing to users. Right now,
>>> "flush" metrics record explicit flushes to disk, but under KIP-892, even
>> a
>>> commit() will not explicitly flush data to disk - RocksDB will decide on
>>> when to flush memtables to disk itself.
>>>
>>> If we keep the existing "flush" metrics, we'd have two options, which
>> both
>>> seem pretty bad to me:
>>>
>>>      1. Have them record calls to commit(), which would be misleading, as
>>>      data is no longer explicitly "flushed" to disk by this call.
>>>      2. Have them record nothing at all, which is equivalent to removing
>> the
>>>      metrics, except that users will see the metric still exists and so
>> assume
>>>      that the metric is correct, and that there's a problem with their
>> system
>>>      when there isn't.
>>>
>>> I agree that removing them is also a bad solution, and I'd like some
>>> guidance on the best path forward here.
>>>
>>> 5.
>>> Position files are updated on every write to a StateStore. Since our
>> writes
>>> are now buffered until commit(), we can't update the Position file until
>>> commit() has been called, otherwise it would be inconsistent with the
>> data
>>> in the event of a rollback. Consequently, we need to manage these offsets
>>> the same way we manage the checkpoint offsets, and ensure they're only
>>> written on commit().
>>>
>>> 6.
>>> Agreed, although I'm not exactly sure yet what tests to write. How
>> explicit
>>> do we need to be here in the KIP?
>>>
>>> As for upgrade/downgrade: upgrade is designed to be seamless, and we
>> should
>>> definitely add some tests around that. Downgrade, it transpires, isn't
>>> currently possible, as the extra column family for offset storage is
>>> incompatible with the pre-KIP-892 implementation: when you open a RocksDB
>>> database, you must open all available column families or receive an
>> error.
>>> What currently happens on downgrade is that it attempts to open the
>> store,
>>> throws an error about the offsets column family not being opened, which
>>> triggers a wipe and rebuild of the Task. Given that downgrades should be
>>> uncommon, I think this is acceptable behaviour, as the end-state is
>>> consistent, even if it results in an undesirable state restore.
>>>
>>> Should I document the upgrade/downgrade behaviour explicitly in the KIP?
>>>
>>> --
>>>
>>> Regards,
>>> Nick
>>>
>>>
>>> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org> wrote:
>>>
>>>> Hi Nick!
>>>>
>>>> Thanks for the updates!
>>>>
>>>> 1.
>>>> Why does StateStore#flush() default to
>>>> StateStore#commit(Collections.emptyMap())?
>>>> Since calls to flush() will not exist anymore after this KIP is
>>>> released, I would rather throw an unsupported operation exception by
>>>> default.
>>>>
>>>>
>>>> 2.
>>>> When would a state store return -1 from
>>>> StateStore#approximateNumUncommittedBytes() while being transactional?
>>>>
>>>> Wouldn't StateStore#approximateNumUncommittedBytes() also return 0 if
>>>> the state store is transactional but nothing has been written to the
>>>> state store yet?
>>>>
>>>>
>>>> 3.
>>>> Sorry for bringing this up again. Does this KIP really need to introduce
>>>> StateStoreContext#isolationLevel()? StateStoreContext has already
>>>> appConfigs() which basically exposes the same information, i.e., if EOS
>>>> is enabled or not.
>>>> In one of your previous e-mails you wrote:
>>>>
>>>> "My idea was to try to keep the StateStore interface as loosely coupled
>>>> from the Streams engine as possible, to give implementers more freedom,
>>>> and reduce the amount of internal knowledge required."
>>>>
>>>> While I understand the intent, I doubt that it decreases the coupling of
>>>> a StateStore interface and the Streams engine. READ_COMMITTED only
>>>> applies to IQ but not to reads by processors. Thus, implementers need to
>>>> understand how Streams accesses the state stores.
>>>>
>>>> I would like to hear what others think about this.
>>>>
>>>>
>>>> 4.
>>>> Great exposing new metrics for transactional state stores! However, I
>>>> would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides unit
>>>> tests. Additionally, we probably need integration or system tests to
>>>> verify that upgrades and downgrades between transactional and
>>>> non-transactional state stores work as expected.
>>>>
>>>>
>>>> Best,
>>>> Bruno
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On 7/21/23 10:34 PM, Nick Telford wrote:
>>>>> One more thing: I noted John's suggestion in the KIP, under "Rejected
>>>>> Alternatives". I still think it's an idea worth pursuing, but I believe
>>>>> that it's out of the scope of this KIP, because it solves a different
>> set
>>>>> of problems to this KIP, and the scope of this one has already grown
>>>> quite
>>>>> large!
>>>>>
>>>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <ni...@gmail.com>
>>>> wrote:
>>>>>
>>>>>> Hi everyone,
>>>>>>
>>>>>> I've updated the KIP (
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>> )
>>>>>> with the latest changes; mostly bringing back "Atomic Checkpointing"
>>>> (for
>>>>>> what feels like the 10th time!). I think the one thing missing is some
>>>>>> changes to metrics (notably the store "flush" metrics will need to be
>>>>>> renamed to "commit").
>>>>>>
>>>>>> The reason I brought back Atomic Checkpointing was to decouple store
>>>> flush
>>>>>> from store commit. This is important, because with Transactional
>>>>>> StateStores, we now need to call "flush" on *every* Task commit, and
>> not
>>>>>> just when the StateStore is closing, otherwise our transaction buffer
>>>> will
>>>>>> never be written and persisted, instead growing unbounded! I
>>>> experimented
>>>>>> with some simple solutions, like forcing a store flush whenever the
>>>>>> transaction buffer was likely to exceed its configured size, but this
>>>> was
>>>>>> brittle: it prevented the transaction buffer from being configured to
>> be
>>>>>> unbounded, and it still would have required explicit flushes of
>> RocksDB,
>>>>>> yielding sub-optimal performance and memory utilization.
>>>>>>
>>>>>> I deemed Atomic Checkpointing to be the "right" way to resolve this
>>>>>> problem. By ensuring that the changelog offsets that correspond to the
>>>> most
>>>>>> recently written records are always atomically written to the
>> StateStore
>>>>>> (by writing them to the same transaction buffer), we can avoid
>> forcibly
>>>>>> flushing the RocksDB memtables to disk, letting RocksDB flush them
>> only
>>>>>> when necessary, without losing any of our consistency guarantees. See
>>>> the
>>>>>> updated KIP for more info.
>>>>>>
>>>>>> I have fully implemented these changes, although I'm still not
>> entirely
>>>>>> happy with the implementation for segmented StateStores, so I plan to
>>>>>> refactor that. Despite that, all tests pass. If you'd like to try out
>> or
>>>>>> review this highly experimental and incomplete branch, it's available
>>>> here:
>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note: it's
>>>> built
>>>>>> against Kafka 3.5.0 so that I had a stable base to build and test it
>> on,
>>>>>> and to enable easy apples-to-apples comparisons in a live
>> environment. I
>>>>>> plan to rebase it against trunk once it's nearer completion and has
>> been
>>>>>> proven on our main application.
>>>>>>
>>>>>> I would really appreciate help in reviewing and testing:
>>>>>> - Segmented (Versioned, Session and Window) stores
>>>>>> - Global stores
>>>>>>
>>>>>> As I do not currently use either of these, so my primary test
>>>> environment
>>>>>> doesn't test these areas.
>>>>>>
>>>>>> I'm going on Parental Leave starting next week for a few weeks, so
>> will
>>>>>> not have time to move this forward until late August. That said, your
>>>>>> feedback is welcome and appreciated, I just won't be able to respond
>> as
>>>>>> quickly as usual.
>>>>>>
>>>>>> Regards,
>>>>>> Nick
>>>>>>
>>>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com>
>>>> wrote:
>>>>>>
>>>>>>> Hi Bruno
>>>>>>>
>>>>>>> Yes, that's correct, although the impact on IQ is not something I had
>>>>>>> considered.
>>>>>>>
>>>>>>> What about atomically updating the state store from the transaction
>>>>>>>> buffer every commit interval and writing the checkpoint (thus,
>>>> flushing
>>>>>>>> the memtable) every configured amount of data and/or number of
>> commit
>>>>>>>> intervals?
>>>>>>>>
>>>>>>>
>>>>>>> I'm not quite sure I follow. Are you suggesting that we add an
>>>> additional
>>>>>>> config for the max number of commit intervals between checkpoints?
>> That
>>>>>>> way, we would checkpoint *either* when the transaction buffers are
>>>> nearly
>>>>>>> full, *OR* whenever a certain number of commit intervals have
>> elapsed,
>>>>>>> whichever comes first?
>>>>>>>
>>>>>>> That certainly seems reasonable, although this re-ignites an earlier
>>>>>>> debate about whether a config should be measured in "number of commit
>>>>>>> intervals", instead of just an absolute time.
>>>>>>>
>>>>>>> FWIW, I realised that this issue is the reason I was pursuing the
>>>> Atomic
>>>>>>> Checkpoints, as it de-couples memtable flush from checkpointing,
>> which
>>>>>>> enables us to just checkpoint on every commit without any performance
>>>>>>> impact. Atomic Checkpointing is definitely the "best" solution, but
>>>> I'm not
>>>>>>> sure if this is enough to bring it back into this KIP.
>>>>>>>
>>>>>>> I'm currently working on moving all the transactional logic directly
>>>> into
>>>>>>> RocksDBStore itself, which does away with the
>> StateStore#newTransaction
>>>>>>> method, and reduces the number of new classes introduced,
>> significantly
>>>>>>> reducing the complexity. If it works, and the complexity is
>> drastically
>>>>>>> reduced, I may try bringing back Atomic Checkpoints into this KIP.
>>>>>>>
>>>>>>> Regards,
>>>>>>> Nick
>>>>>>>
>>>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org>
>> wrote:
>>>>>>>
>>>>>>>> Hi Nick,
>>>>>>>>
>>>>>>>> Thanks for the insights! Very interesting!
>>>>>>>>
>>>>>>>> As far as I understand, you want to atomically update the state
>> store
>>>>>>>> from the transaction buffer, flush the memtable of a state store and
>>>>>>>> write the checkpoint not after the commit time elapsed but after the
>>>>>>>> transaction buffer reached a size that would lead to exceeding
>>>>>>>> statestore.transaction.buffer.max.bytes before the next commit
>>>> interval
>>>>>>>> ends.
>>>>>>>> That means, the Kafka transaction would commit every commit interval
>>>> but
>>>>>>>> the state store will only be atomically updated roughly every
>>>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ would then
>>>> only
>>>>>>>> see new data roughly every statestore.transaction.buffer.max.bytes.
>>>>>>>> After a failure the state store needs to restore up to
>>>>>>>> statestore.transaction.buffer.max.bytes.
>>>>>>>>
>>>>>>>> Is this correct?
>>>>>>>>
>>>>>>>> What about atomically updating the state store from the transaction
>>>>>>>> buffer every commit interval and writing the checkpoint (thus,
>>>> flushing
>>>>>>>> the memtable) every configured amount of data and/or number of
>> commit
>>>>>>>> intervals? In such a way, we would have the same delay for records
>>>>>>>> appearing in output topics and IQ because both would appear when the
>>>>>>>> Kafka transaction is committed. However, after a failure the state
>>>> store
>>>>>>>> still needs to restore up to statestore.transaction.buffer.max.bytes
>>>> and
>>>>>>>> it might restore data that is already in the state store because the
>>>>>>>> checkpoint lags behind the last stable offset (i.e. the last
>> committed
>>>>>>>> offset) of the changelog topics. Restoring data that is already in
>> the
>>>>>>>> state store is idempotent, so eos should not violated.
>>>>>>>> This solution needs at least one new config to specify when a
>>>> checkpoint
>>>>>>>> should be written.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> A small correction to your previous e-mail that does not change
>>>> anything
>>>>>>>> you said: Under alos the default commit interval is 30 seconds, not
>>>> five
>>>>>>>> seconds.
>>>>>>>>
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Bruno
>>>>>>>>
>>>>>>>>
>>>>>>>> On 01.07.23 12:37, Nick Telford wrote:
>>>>>>>>> Hi everyone,
>>>>>>>>>
>>>>>>>>> I've begun performance testing my branch on our staging
>> environment,
>>>>>>>>> putting it through its paces in our non-trivial application. I'm
>>>>>>>> already
>>>>>>>>> observing the same increased flush rate that we saw the last time
>> we
>>>>>>>>> attempted to use a version of this KIP, but this time, I think I
>> know
>>>>>>>> why.
>>>>>>>>>
>>>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end of
>> the
>>>>>>>> Task
>>>>>>>>> commit process, has the following behaviour:
>>>>>>>>>
>>>>>>>>>        - Under ALOS: checkpoint the state stores. This includes
>>>>>>>>>        flushing memtables in RocksDB. This is acceptable because the
>>>>>>>> default
>>>>>>>>>        commit.interval.ms is 5 seconds, so forcibly flushing
>> memtables
>>>>>>>> every 5
>>>>>>>>>        seconds is acceptable for most applications.
>>>>>>>>>        - Under EOS: checkpointing is not done, *unless* it's being
>>>>>>>> forced, due
>>>>>>>>>        to e.g. the Task closing or being revoked. This means that
>> under
>>>>>>>> normal
>>>>>>>>>        processing conditions, the state stores will not be
>>>> checkpointed,
>>>>>>>> and will
>>>>>>>>>        not have memtables flushed at all , unless RocksDB decides to
>>>>>>>> flush them on
>>>>>>>>>        its own. Checkpointing stores and force-flushing their
>> memtables
>>>>>>>> is only
>>>>>>>>>        done when a Task is being closed.
>>>>>>>>>
>>>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least *some*
>>>> normal
>>>>>>>>> Task commits, in order to write the RocksDB transaction buffers to
>>>> the
>>>>>>>>> state stores, and to ensure the offsets are synced to disk to
>> prevent
>>>>>>>>> restores from getting out of hand. Consequently, my current
>>>>>>>> implementation
>>>>>>>>> calls maybeCheckpoint on *every* Task commit, which is far too
>>>>>>>> frequent.
>>>>>>>>> This causes checkpoints every 10,000 records, which is a change in
>>>>>>>> flush
>>>>>>>>> behaviour, potentially causing performance problems for some
>>>>>>>> applications.
>>>>>>>>>
>>>>>>>>> I'm looking into possible solutions, and I'm currently leaning
>>>> towards
>>>>>>>>> using the statestore.transaction.buffer.max.bytes configuration to
>>>>>>>>> checkpoint Tasks once we are likely to exceed it. This would
>>>>>>>> complement the
>>>>>>>>> existing "early Task commit" functionality that this configuration
>>>>>>>>> provides, in the following way:
>>>>>>>>>
>>>>>>>>>        - Currently, we use statestore.transaction.buffer.max.bytes
>> to
>>>>>>>> force an
>>>>>>>>>        early Task commit if processing more records would cause our
>>>> state
>>>>>>>> store
>>>>>>>>>        transactions to exceed the memory assigned to them.
>>>>>>>>>        - New functionality: when a Task *does* commit, we will not
>>>>>>>> checkpoint
>>>>>>>>>        the stores (and hence flush the transaction buffers) unless
>> we
>>>>>>>> expect to
>>>>>>>>>        cross the statestore.transaction.buffer.max.bytes threshold
>>>> before
>>>>>>>> the next
>>>>>>>>>        commit
>>>>>>>>>
>>>>>>>>> I'm also open to suggestions.
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>> Nick
>>>>>>>>>
>>>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <nick.telford@gmail.com
>>>
>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Bruno!
>>>>>>>>>>
>>>>>>>>>> 3.
>>>>>>>>>> By "less predictable for users", I meant in terms of understanding
>>>> the
>>>>>>>>>> performance profile under various circumstances. The more complex
>>>> the
>>>>>>>>>> solution, the more difficult it would be for users to understand
>> the
>>>>>>>>>> performance they see. For example, spilling records to disk when
>> the
>>>>>>>>>> transaction buffer reaches a threshold would, I expect, reduce
>> write
>>>>>>>>>> throughput. This reduction in write throughput could be
>> unexpected,
>>>>>>>> and
>>>>>>>>>> potentially difficult to diagnose/understand for users.
>>>>>>>>>> At the moment, I think the "early commit" concept is relatively
>>>>>>>>>> straightforward; it's easy to document, and conceptually fairly
>>>>>>>> obvious to
>>>>>>>>>> users. We could probably add a metric to make it easier to
>>>> understand
>>>>>>>> when
>>>>>>>>>> it happens though.
>>>>>>>>>>
>>>>>>>>>> 3. (the second one)
>>>>>>>>>> The IsolationLevel is *essentially* an indirect way of telling
>>>>>>>> StateStores
>>>>>>>>>> whether they should be transactional. READ_COMMITTED essentially
>>>>>>>> requires
>>>>>>>>>> transactions, because it dictates that two threads calling
>>>>>>>>>> `newTransaction()` should not see writes from the other
>> transaction
>>>>>>>> until
>>>>>>>>>> they have been committed. With READ_UNCOMMITTED, all bets are off,
>>>> and
>>>>>>>>>> stores can allow threads to observe written records at any time,
>>>>>>>> which is
>>>>>>>>>> essentially "no transactions". That said, StateStores are free to
>>>>>>>> implement
>>>>>>>>>> these guarantees however they can, which is a bit more relaxed
>> than
>>>>>>>>>> dictating "you must use transactions". For example, with RocksDB
>> we
>>>>>>>> would
>>>>>>>>>> implement these as READ_COMMITTED == WBWI-based "transactions",
>>>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But with other
>>>>>>>> storage
>>>>>>>>>> engines, it might be preferable to *always* use transactions, even
>>>>>>>> when
>>>>>>>>>> unnecessary; or there may be storage engines that don't provide
>>>>>>>>>> transactions, but the isolation guarantees can be met using a
>>>>>>>> different
>>>>>>>>>> technique.
>>>>>>>>>> My idea was to try to keep the StateStore interface as loosely
>>>> coupled
>>>>>>>>>> from the Streams engine as possible, to give implementers more
>>>>>>>> freedom, and
>>>>>>>>>> reduce the amount of internal knowledge required.
>>>>>>>>>> That said, I understand that "IsolationLevel" might not be the
>> right
>>>>>>>>>> abstraction, and we can always make it much more explicit if
>>>>>>>> required, e.g.
>>>>>>>>>> boolean transactional()
>>>>>>>>>>
>>>>>>>>>> 7-8.
>>>>>>>>>> I can make these changes either later today or tomorrow.
>>>>>>>>>>
>>>>>>>>>> Small update:
>>>>>>>>>> I've rebased my branch on trunk and fixed a bunch of issues that
>>>>>>>> needed
>>>>>>>>>> addressing. Currently, all the tests pass, which is promising, but
>>>> it
>>>>>>>> will
>>>>>>>>>> need to undergo some performance testing. I haven't (yet) worked
>> on
>>>>>>>>>> removing the `newTransaction()` stuff, but I would expect that,
>>>>>>>>>> behaviourally, it should make no difference. The branch is
>> available
>>>>>>>> at
>>>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
>>>>>>>>>> interested in taking an early look.
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>> Nick
>>>>>>>>>>
>>>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org>
>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>
>>>>>>>>>>> 1.
>>>>>>>>>>> Yeah, I agree with you. That was actually also my point. I
>>>> understood
>>>>>>>>>>> that John was proposing the ingestion path as a way to avoid the
>>>>>>>> early
>>>>>>>>>>> commits. Probably, I misinterpreted the intent.
>>>>>>>>>>>
>>>>>>>>>>> 2.
>>>>>>>>>>> I agree with John here, that actually it is public API. My
>> question
>>>>>>>> is
>>>>>>>>>>> how this usage pattern affects normal processing.
>>>>>>>>>>>
>>>>>>>>>>> 3.
>>>>>>>>>>> My concern is that checking for the size of the transaction
>> buffer
>>>>>>>> and
>>>>>>>>>>> maybe triggering an early commit affects the whole processing of
>>>>>>>> Kafka
>>>>>>>>>>> Streams. The transactionality of a state store is not confined to
>>>> the
>>>>>>>>>>> state store itself, but spills over and changes the behavior of
>>>> other
>>>>>>>>>>> parts of the system. I agree with you that it is a decent
>>>>>>>> compromise. I
>>>>>>>>>>> just wanted to analyse the downsides and list the options to
>>>> overcome
>>>>>>>>>>> them. I also agree with you that all options seem quite heavy
>>>>>>>> compared
>>>>>>>>>>> with your KIP. I do not understand what you mean with "less
>>>>>>>> predictable
>>>>>>>>>>> for users", though.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> I found the discussions about the alternatives really
>> interesting.
>>>>>>>> But I
>>>>>>>>>>> also think that your plan sounds good and we should continue with
>>>> it!
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
>>>>>>>>>>>
>>>>>>>>>>> 3.
>>>>>>>>>>> Ah, now, I understand the reasoning behind putting isolation
>> level
>>>> in
>>>>>>>>>>> the state store context. Thanks! Should that also be a way to
>> give
>>>>>>>> the
>>>>>>>>>>> the state store the opportunity to decide whether to turn on
>>>>>>>>>>> transactions or not?
>>>>>>>>>>> With my comment, I was more concerned about how do you know if a
>>>>>>>>>>> checkpoint file needs to be written under EOS, if you do not
>> have a
>>>>>>>> way
>>>>>>>>>>> to know if the state store is transactional or not. If a state
>>>> store
>>>>>>>> is
>>>>>>>>>>> transactional, the checkpoint file can be written during normal
>>>>>>>>>>> processing under EOS. If the state store is not transactional,
>> the
>>>>>>>>>>> checkpoint file must not be written under EOS.
>>>>>>>>>>>
>>>>>>>>>>> 7.
>>>>>>>>>>> My point was about not only considering the bytes in memory in
>>>> config
>>>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that might be
>>>>>>>> spilled
>>>>>>>>>>> on disk. Basically, I was wondering whether you should remove the
>>>>>>>>>>> "memory" in "Maximum number of memory bytes to be used to
>>>>>>>>>>> buffer uncommitted state-store records." My thinking was that
>> even
>>>>>>>> if a
>>>>>>>>>>> state store spills uncommitted bytes to disk, limiting the
>> overall
>>>>>>>> bytes
>>>>>>>>>>> might make sense. Thinking about it again and considering the
>>>> recent
>>>>>>>>>>> discussions, it does not make too much sense anymore.
>>>>>>>>>>> I like the name statestore.transaction.buffer.max.bytes that you
>>>>>>>> proposed.
>>>>>>>>>>>
>>>>>>>>>>> 8.
>>>>>>>>>>> A high-level description (without implementation details) of how
>>>>>>>> Kafka
>>>>>>>>>>> Streams will manage the commit of changelog transactions, state
>>>> store
>>>>>>>>>>> transactions and checkpointing would be great. Would be great if
>>>> you
>>>>>>>>>>> could also add some sentences about the behavior in case of a
>>>>>>>> failure.
>>>>>>>>>>> For instance how does a transactional state store recover after a
>>>>>>>>>>> failure or what happens with the transaction buffer, etc. (that
>> is
>>>>>>>> what
>>>>>>>>>>> I meant by "fail-over" in point 9.)
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> Bruno
>>>>>>>>>>>
>>>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>
>>>>>>>>>>>> 1.
>>>>>>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
>>>>>>>> transactions
>>>>>>>>>>>> have, whereby exceeding (or likely to exceed) configured memory
>>>>>>>> needs to
>>>>>>>>>>>> trigger an early commit?
>>>>>>>>>>>>
>>>>>>>>>>>> 2.
>>>>>>>>>>>> This is one of my big concerns. Ultimately, any approach based
>> on
>>>>>>>>>>> cracking
>>>>>>>>>>>> open RocksDB internals and using it in ways it's not really
>>>> designed
>>>>>>>>>>> for is
>>>>>>>>>>>> likely to have some unforseen performance or consistency issues.
>>>>>>>>>>>>
>>>>>>>>>>>> 3.
>>>>>>>>>>>> What's your motivation for removing these early commits? While
>> not
>>>>>>>>>>> ideal, I
>>>>>>>>>>>> think they're a decent compromise to ensure consistency whilst
>>>>>>>>>>> maintaining
>>>>>>>>>>>> good and predictable performance.
>>>>>>>>>>>> All 3 of your suggested ideas seem *very* complicated, and might
>>>>>>>>>>> actually
>>>>>>>>>>>> make behaviour less predictable for users as a consequence.
>>>>>>>>>>>>
>>>>>>>>>>>> I'm a bit concerned that the scope of this KIP is growing a bit
>>>> out
>>>>>>>> of
>>>>>>>>>>>> control. While it's good to discuss ideas for future
>>>> improvements, I
>>>>>>>>>>> think
>>>>>>>>>>>> it's important to narrow the scope down to a design that
>> achieves
>>>>>>>> the
>>>>>>>>>>> most
>>>>>>>>>>>> pressing objectives (constant sized restorations during dirty
>>>>>>>>>>>> close/unexpected errors). Any design that this KIP produces can
>>>>>>>>>>> ultimately
>>>>>>>>>>>> be changed in the future, especially if the bulk of it is
>> internal
>>>>>>>>>>>> behaviour.
>>>>>>>>>>>>
>>>>>>>>>>>> I'm going to spend some time next week trying to re-work the
>>>>>>>> original
>>>>>>>>>>>> WriteBatchWithIndex design to remove the newTransaction()
>> method,
>>>>>>>> such
>>>>>>>>>>> that
>>>>>>>>>>>> it's just an implementation detail of RocksDBStore. That way, if
>>>> we
>>>>>>>>>>> want to
>>>>>>>>>>>> replace WBWI with something in the future, like the SST file
>>>>>>>> management
>>>>>>>>>>>> outlined by John, then we can do so with little/no API changes.
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>>
>>>>>>>>>>>> Nick
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>>>>
>>>
>>
> 

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

Posted by Nick Telford <ni...@gmail.com>.
Bruno,

Thinking about 3a. in addition to adding the IsolationLevel to
QueryStoreParameters and Query, what about also adding a method like
"ReadOnlyKeyValueStore<K,
V> view(IsolationLevel isolationLevel)" to ReadOnlyKeyValueStore?

This would enable us to easily select/switch between IsolationLevels, 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 <ni...@gmail.com> wrote:

> Hi Bruno,
>
> Thanks for getting back to me!
>
> 2.
> The fact that implementations can always track estimated memory usage in
> the wrapper is a good point. I can remove -1 as an option, and I'll clarify
> the JavaDoc that 0 is not just for non-transactional stores, which is
> currently misleading.
>
> 6.
> The problem with catching the exception in the downgrade process is that
> would require new code in the Kafka version being downgraded to. Since
> users could conceivably downgrade to almost *any* older version of Kafka
> Streams, I'm not sure how we could add that code?
> The only way I can think of doing it would be to provide a dedicated
> downgrade tool, that goes through every local store and removes the
> offsets column families. But that seems like an unnecessary amount of extra
> code to maintain just to handle a somewhat niche situation, when the
> alternative (automatically wipe and restore stores) should be acceptable.
>
> 1, 4, 5: Agreed. I'll make the changes you've requested.
>
> 3a.
> I agree that IsolationLevel makes more sense at query-time, and I actually
> initially attempted to place the IsolationLevel at query-time, but I ran
> into some problems:
> - The key issue is that, under ALOS we're not staging writes in
> transactions, so can't perform writes at the READ_COMMITTED isolation
> level. However, this may be addressed if we decide to *always* use
> transactions as discussed under 3b.
> - IQv1 and IQv2 have quite different implementations. I remember having
> some difficulty understanding the IQv1 internals, which made it difficult
> to determine what needed to be changed. However, I *think* this can be
> addressed for both implementations by wrapping the RocksDBStore in an
> IsolationLevel-dependent wrapper, that overrides read methods (get, etc.)
> to either read directly from the database or from the ongoing transaction.
> But IQv1 might still be difficult.
> - If IsolationLevel becomes a query constraint, then all other StateStores
> will need to respect it, including the in-memory stores. This would require
> us to adapt in-memory stores to stage their writes so they can be isolated
> from READ_COMMITTTED queries. It would also become an important
> consideration for third-party stores on upgrade, as without changes, they
> would not support READ_COMMITTED queries correctly.
>
> Ultimately, I may need some help making the necessary change to IQv1 to
> support this, but I don't think it's fundamentally impossible, if we want
> to pursue this route.
>
> 3b.
> The main reason I chose to keep ALOS un-transactional was to minimize
> behavioural change for most users (I believe most Streams users use the
> default configuration, which is ALOS). That said, it's clear that if ALOS
> also used transactional stores, the only change in behaviour would be that
> it would become *more correct*, which could be considered a "bug fix" by
> users, rather than a change they need to handle.
>
> I believe that performance using transactions (aka. RocksDB WriteBatches)
> should actually be *better* than the un-batched write-path that is
> currently used[1]. The only "performance" consideration will be the
> increased memory usage that transactions require. Given the mitigations for
> this memory that we have in place, I would expect that this is not a
> problem for most users.
>
> If we're happy to do so, we can make ALOS also use transactions.
>
> Regards,
> Nick
>
> Link 1:
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>
> On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <ca...@apache.org> wrote:
>
>> Hi Nick,
>>
>> Thanks for the updates and sorry for the delay on my side!
>>
>>
>> 1.
>> Making the default implementation for flush() a no-op sounds good to me.
>>
>>
>> 2.
>> I think what was bugging me here is that a third-party state store needs
>> to implement the state store interface. That means they need to
>> implement a wrapper around the actual state store as we do for RocksDB
>> with RocksDBStore. So, a third-party state store can always estimate the
>> uncommitted bytes, if it wants, because the wrapper can record the added
>> bytes.
>> One case I can think of where returning -1 makes sense is when Streams
>> does not need to estimate the size of the write batch and trigger
>> extraordinary commits, because the third-party state store takes care of
>> memory. But in that case the method could also just return 0. Even that
>> case would be better solved with a method that returns whether the state
>> store manages itself the memory used for uncommitted bytes or not.
>> Said that, I am fine with keeping the -1 return value, I was just
>> wondering when and if it will be used.
>>
>> Regarding returning 0 for transactional state stores when the batch is
>> empty, I was just wondering because you explicitly stated
>>
>> "or {@code 0} if this StateStore does not support transactions."
>>
>> So it seemed to me returning 0 could only happen for non-transactional
>> state stores.
>>
>>
>> 3.
>>
>> a) What do you think if we move the isolation level to IQ (v1 and v2)?
>> In the end this is the only component that really needs to specify the
>> isolation level. It is similar to the Kafka consumer that can choose
>> with what isolation level to read the input topic.
>> For IQv1 the isolation level should go into StoreQueryParameters. For
>> IQv2, I would add it to the Query interface.
>>
>> b) Point a) raises the question what should happen during at-least-once
>> processing when the state store does not use transactions? John in the
>> past proposed to also use transactions on state stores for
>> at-least-once. I like that idea, because it avoids aggregating the same
>> records over and over again in the case of a failure. We had a case in
>> the past where a Streams applications in at-least-once mode was failing
>> continuously for some reasons I do not remember before committing the
>> offsets. After each failover, the app aggregated again and again the
>> same records. Of course the aggregate increased to very wrong values
>> just because of the failover. With transactions on the state stores we
>> could have avoided this. The app would have output the same aggregate
>> multiple times (i.e., after each failover) but at least the value of the
>> aggregate would not depend on the number of failovers. Outputting the
>> same aggregate multiple times would be incorrect under exactly-once but
>> it is OK for at-least-once.
>> If it makes sense to add a config to turn on and off transactions on
>> state stores under at-least-once or just use transactions in any case is
>> a question we should also discuss in this KIP. It depends a bit on the
>> performance trade-off. Maybe to be safe, I would add a config.
>>
>>
>> 4.
>> Your points are all valid. I tend to say to keep the metrics around
>> flush() until we remove flush() completely from the interface. Calls to
>> flush() might still exist since existing processors might still call
>> flush() explicitly as you mentioned in 1). For sure, we need to document
>> how the metrics change due to the transactions in the upgrade notes.
>>
>>
>> 5.
>> I see. Then you should describe how the .position files are handled  in
>> a dedicated section of the KIP or incorporate the description in the
>> "Atomic Checkpointing" section instead of only mentioning it in the
>> "Compatibility, 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:
>>
>>
>> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
>>
>> It is a bit brittle, but it works.
>>
>>
>> Best,
>> Bruno
>>
>>
>> On 8/24/23 12:18 PM, Nick Telford wrote:
>> > Hi Bruno,
>> >
>> > Thanks for taking the time to review the KIP. I'm back from leave now
>> and
>> > intend to move this forwards as quickly as I can.
>> >
>> > Addressing your points:
>> >
>> > 1.
>> > Because flush() is part of the StateStore API, it's exposed to custom
>> > Processors, which might be making calls to flush(). This was actually
>> the
>> > case in a few integration tests.
>> > To maintain as much compatibility as possible, I'd prefer not to make
>> this
>> > an UnsupportedOperationException, as it will cause previously working
>> > Processors to start throwing exceptions at runtime.
>> > I agree that it doesn't make sense for it to proxy commit(), though, as
>> > that would cause it to violate the "StateStores commit only when the
>> Task
>> > commits" rule.
>> > Instead, I think we should make this a no-op. That way, existing user
>> > Processors will continue to work as-before, without violation of store
>> > consistency that would be caused by premature flush/commit of StateStore
>> > data to disk.
>> > What do you think?
>> >
>> > 2.
>> > As stated in the JavaDoc, when a StateStore implementation is
>> > transactional, but is unable to estimate the uncommitted memory usage,
>> the
>> > method will return -1.
>> > The intention here is to permit third-party implementations that may
>> not be
>> > able to estimate memory usage.
>> >
>> > Yes, it will be 0 when nothing has been written to the store yet. I
>> thought
>> > that was implied by "This method will return an approximation of the
>> memory
>> > would be freed by the next call to {@link #commit(Map)}" and "@return
>> The
>> > approximate size of all records awaiting {@link #commit(Map)}",
>> however, I
>> > can add it explicitly to the JavaDoc if you think this is unclear?
>> >
>> > 3.
>> > I realise this is probably the most contentious point in my design, and
>> I'm
>> > open to changing it if I'm unable to convince you of the benefits.
>> > Nevertheless, here's my argument:
>> > The Interactive Query (IQ) API(s) are directly provided StateStores to
>> > query, and it may be important for users to programmatically know which
>> > mode the StateStore is operating under. If we simply provide an
>> > "eosEnabled" boolean (as used throughout the internal streams engine),
>> or
>> > similar, then users will need to understand the operation and
>> consequences
>> > of each available processing mode and how it pertains to their
>> StateStore.
>> >
>> > Interactive Query users aren't the only people that care about the
>> > processing.mode/IsolationLevel of a StateStore: implementers of custom
>> > StateStores also need to understand the behaviour expected of their
>> > implementation. KIP-892 introduces some assumptions into the Streams
>> Engine
>> > about how StateStores operate under each processing mode, and it's
>> > important that custom implementations adhere to those assumptions in
>> order
>> > to maintain the consistency guarantees.
>> >
>> > IsolationLevels provide a high-level contract on the behaviour of the
>> > StateStore: a user knows that under READ_COMMITTED, they will see writes
>> > only after the Task has committed, and under READ_UNCOMMITTED they will
>> see
>> > writes immediately. No understanding of the details of each
>> processing.mode
>> > is required, either for IQ users or StateStore implementers.
>> >
>> > An argument can be made that these contractual guarantees can simply be
>> > documented for the processing.mode (i.e. that exactly-once and
>> > exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves
>> like
>> > READ_UNCOMMITTED), but there are several small issues with this I'd
>> prefer
>> > to avoid:
>> >
>> >     - Where would we document these contracts, in a way that is
>> difficult
>> >     for users/implementers to miss/ignore?
>> >     - It's not clear to users that the processing mode is communicating
>> >     an expectation of read isolation, unless they read the
>> documentation. Users
>> >     rarely consult documentation unless they feel they need to, so it's
>> likely
>> >     this detail would get missed by many users.
>> >     - It tightly couples processing modes to read isolation. Adding new
>> >     processing modes, or changing the read isolation of existing
>> processing
>> >     modes would be difficult/impossible.
>> >
>> > Ultimately, the cost of introducing IsolationLevels is just a single
>> > method, since we re-use the existing IsolationLevel enum from Kafka.
>> This
>> > gives us a clear place to document the contractual guarantees expected
>> > of/provided by StateStores, that is accessible both by the StateStore
>> > itself, and by IQ users.
>> >
>> > (Writing this I've just realised that the StateStore and IQ APIs
>> actually
>> > don't provide access to StateStoreContext that IQ users would have
>> direct
>> > access to... Perhaps StateStore should expose isolationLevel() itself
>> too?)
>> >
>> > 4.
>> > Yeah, I'm not comfortable renaming the metrics in-place either, as it's
>> a
>> > backwards incompatible change. My concern is that, if we leave the
>> existing
>> > "flush" metrics in place, they will be confusing to users. Right now,
>> > "flush" metrics record explicit flushes to disk, but under KIP-892,
>> even a
>> > commit() will not explicitly flush data to disk - RocksDB will decide on
>> > when to flush memtables to disk itself.
>> >
>> > If we keep the existing "flush" metrics, we'd have two options, which
>> both
>> > seem pretty bad to me:
>> >
>> >     1. Have them record calls to commit(), which would be misleading, as
>> >     data is no longer explicitly "flushed" to disk by this call.
>> >     2. Have them record nothing at all, which is equivalent to removing
>> the
>> >     metrics, except that users will see the metric still exists and so
>> assume
>> >     that the metric is correct, and that there's a problem with their
>> system
>> >     when there isn't.
>> >
>> > I agree that removing them is also a bad solution, and I'd like some
>> > guidance on the best path forward here.
>> >
>> > 5.
>> > Position files are updated on every write to a StateStore. Since our
>> writes
>> > are now buffered until commit(), we can't update the Position file until
>> > commit() has been called, otherwise it would be inconsistent with the
>> data
>> > in the event of a rollback. Consequently, we need to manage these
>> offsets
>> > the same way we manage the checkpoint offsets, and ensure they're only
>> > written on commit().
>> >
>> > 6.
>> > Agreed, although I'm not exactly sure yet what tests to write. How
>> explicit
>> > do we need to be here in the KIP?
>> >
>> > As for upgrade/downgrade: upgrade is designed to be seamless, and we
>> should
>> > definitely add some tests around that. Downgrade, it transpires, isn't
>> > currently possible, as the extra column family for offset storage is
>> > incompatible with the pre-KIP-892 implementation: when you open a
>> RocksDB
>> > database, you must open all available column families or receive an
>> error.
>> > What currently happens on downgrade is that it attempts to open the
>> store,
>> > throws an error about the offsets column family not being opened, which
>> > triggers a wipe and rebuild of the Task. Given that downgrades should be
>> > uncommon, I think this is acceptable behaviour, as the end-state is
>> > consistent, even if it results in an undesirable state restore.
>> >
>> > Should I document the upgrade/downgrade behaviour explicitly in the KIP?
>> >
>> > --
>> >
>> > Regards,
>> > Nick
>> >
>> >
>> > On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org> wrote:
>> >
>> >> Hi Nick!
>> >>
>> >> Thanks for the updates!
>> >>
>> >> 1.
>> >> Why does StateStore#flush() default to
>> >> StateStore#commit(Collections.emptyMap())?
>> >> Since calls to flush() will not exist anymore after this KIP is
>> >> released, I would rather throw an unsupported operation exception by
>> >> default.
>> >>
>> >>
>> >> 2.
>> >> When would a state store return -1 from
>> >> StateStore#approximateNumUncommittedBytes() while being transactional?
>> >>
>> >> Wouldn't StateStore#approximateNumUncommittedBytes() also return 0 if
>> >> the state store is transactional but nothing has been written to the
>> >> state store yet?
>> >>
>> >>
>> >> 3.
>> >> Sorry for bringing this up again. Does this KIP really need to
>> introduce
>> >> StateStoreContext#isolationLevel()? StateStoreContext has already
>> >> appConfigs() which basically exposes the same information, i.e., if EOS
>> >> is enabled or not.
>> >> In one of your previous e-mails you wrote:
>> >>
>> >> "My idea was to try to keep the StateStore interface as loosely coupled
>> >> from the Streams engine as possible, to give implementers more freedom,
>> >> and reduce the amount of internal knowledge required."
>> >>
>> >> While I understand the intent, I doubt that it decreases the coupling
>> of
>> >> a StateStore interface and the Streams engine. READ_COMMITTED only
>> >> applies to IQ but not to reads by processors. Thus, implementers need
>> to
>> >> understand how Streams accesses the state stores.
>> >>
>> >> I would like to hear what others think about this.
>> >>
>> >>
>> >> 4.
>> >> Great exposing new metrics for transactional state stores! However, I
>> >> would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides unit
>> >> tests. Additionally, we probably need integration or system tests to
>> >> verify that upgrades and downgrades between transactional and
>> >> non-transactional state stores work as expected.
>> >>
>> >>
>> >> Best,
>> >> Bruno
>> >>
>> >>
>> >>
>> >>
>> >>
>> >> On 7/21/23 10:34 PM, Nick Telford wrote:
>> >>> One more thing: I noted John's suggestion in the KIP, under "Rejected
>> >>> Alternatives". I still think it's an idea worth pursuing, but I
>> believe
>> >>> that it's out of the scope of this KIP, because it solves a different
>> set
>> >>> of problems to this KIP, and the scope of this one has already grown
>> >> quite
>> >>> large!
>> >>>
>> >>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <ni...@gmail.com>
>> >> wrote:
>> >>>
>> >>>> Hi everyone,
>> >>>>
>> >>>> I've updated the KIP (
>> >>>>
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> >> )
>> >>>> with the latest changes; mostly bringing back "Atomic Checkpointing"
>> >> (for
>> >>>> what feels like the 10th time!). I think the one thing missing is
>> some
>> >>>> changes to metrics (notably the store "flush" metrics will need to be
>> >>>> renamed to "commit").
>> >>>>
>> >>>> The reason I brought back Atomic Checkpointing was to decouple store
>> >> flush
>> >>>> from store commit. This is important, because with Transactional
>> >>>> StateStores, we now need to call "flush" on *every* Task commit, and
>> not
>> >>>> just when the StateStore is closing, otherwise our transaction buffer
>> >> will
>> >>>> never be written and persisted, instead growing unbounded! I
>> >> experimented
>> >>>> with some simple solutions, like forcing a store flush whenever the
>> >>>> transaction buffer was likely to exceed its configured size, but this
>> >> was
>> >>>> brittle: it prevented the transaction buffer from being configured
>> to be
>> >>>> unbounded, and it still would have required explicit flushes of
>> RocksDB,
>> >>>> yielding sub-optimal performance and memory utilization.
>> >>>>
>> >>>> I deemed Atomic Checkpointing to be the "right" way to resolve this
>> >>>> problem. By ensuring that the changelog offsets that correspond to
>> the
>> >> most
>> >>>> recently written records are always atomically written to the
>> StateStore
>> >>>> (by writing them to the same transaction buffer), we can avoid
>> forcibly
>> >>>> flushing the RocksDB memtables to disk, letting RocksDB flush them
>> only
>> >>>> when necessary, without losing any of our consistency guarantees. See
>> >> the
>> >>>> updated KIP for more info.
>> >>>>
>> >>>> I have fully implemented these changes, although I'm still not
>> entirely
>> >>>> happy with the implementation for segmented StateStores, so I plan to
>> >>>> refactor that. Despite that, all tests pass. If you'd like to try
>> out or
>> >>>> review this highly experimental and incomplete branch, it's available
>> >> here:
>> >>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note: it's
>> >> built
>> >>>> against Kafka 3.5.0 so that I had a stable base to build and test it
>> on,
>> >>>> and to enable easy apples-to-apples comparisons in a live
>> environment. I
>> >>>> plan to rebase it against trunk once it's nearer completion and has
>> been
>> >>>> proven on our main application.
>> >>>>
>> >>>> I would really appreciate help in reviewing and testing:
>> >>>> - Segmented (Versioned, Session and Window) stores
>> >>>> - Global stores
>> >>>>
>> >>>> As I do not currently use either of these, so my primary test
>> >> environment
>> >>>> doesn't test these areas.
>> >>>>
>> >>>> I'm going on Parental Leave starting next week for a few weeks, so
>> will
>> >>>> not have time to move this forward until late August. That said, your
>> >>>> feedback is welcome and appreciated, I just won't be able to respond
>> as
>> >>>> quickly as usual.
>> >>>>
>> >>>> Regards,
>> >>>> Nick
>> >>>>
>> >>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com>
>> >> wrote:
>> >>>>
>> >>>>> Hi Bruno
>> >>>>>
>> >>>>> Yes, that's correct, although the impact on IQ is not something I
>> had
>> >>>>> considered.
>> >>>>>
>> >>>>> What about atomically updating the state store from the transaction
>> >>>>>> buffer every commit interval and writing the checkpoint (thus,
>> >> flushing
>> >>>>>> the memtable) every configured amount of data and/or number of
>> commit
>> >>>>>> intervals?
>> >>>>>>
>> >>>>>
>> >>>>> I'm not quite sure I follow. Are you suggesting that we add an
>> >> additional
>> >>>>> config for the max number of commit intervals between checkpoints?
>> That
>> >>>>> way, we would checkpoint *either* when the transaction buffers are
>> >> nearly
>> >>>>> full, *OR* whenever a certain number of commit intervals have
>> elapsed,
>> >>>>> whichever comes first?
>> >>>>>
>> >>>>> That certainly seems reasonable, although this re-ignites an earlier
>> >>>>> debate about whether a config should be measured in "number of
>> commit
>> >>>>> intervals", instead of just an absolute time.
>> >>>>>
>> >>>>> FWIW, I realised that this issue is the reason I was pursuing the
>> >> Atomic
>> >>>>> Checkpoints, as it de-couples memtable flush from checkpointing,
>> which
>> >>>>> enables us to just checkpoint on every commit without any
>> performance
>> >>>>> impact. Atomic Checkpointing is definitely the "best" solution, but
>> >> I'm not
>> >>>>> sure if this is enough to bring it back into this KIP.
>> >>>>>
>> >>>>> I'm currently working on moving all the transactional logic directly
>> >> into
>> >>>>> RocksDBStore itself, which does away with the
>> StateStore#newTransaction
>> >>>>> method, and reduces the number of new classes introduced,
>> significantly
>> >>>>> reducing the complexity. If it works, and the complexity is
>> drastically
>> >>>>> reduced, I may try bringing back Atomic Checkpoints into this KIP.
>> >>>>>
>> >>>>> Regards,
>> >>>>> Nick
>> >>>>>
>> >>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org>
>> wrote:
>> >>>>>
>> >>>>>> Hi Nick,
>> >>>>>>
>> >>>>>> Thanks for the insights! Very interesting!
>> >>>>>>
>> >>>>>> As far as I understand, you want to atomically update the state
>> store
>> >>>>>> from the transaction buffer, flush the memtable of a state store
>> and
>> >>>>>> write the checkpoint not after the commit time elapsed but after
>> the
>> >>>>>> transaction buffer reached a size that would lead to exceeding
>> >>>>>> statestore.transaction.buffer.max.bytes before the next commit
>> >> interval
>> >>>>>> ends.
>> >>>>>> That means, the Kafka transaction would commit every commit
>> interval
>> >> but
>> >>>>>> the state store will only be atomically updated roughly every
>> >>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ would then
>> >> only
>> >>>>>> see new data roughly every statestore.transaction.buffer.max.bytes.
>> >>>>>> After a failure the state store needs to restore up to
>> >>>>>> statestore.transaction.buffer.max.bytes.
>> >>>>>>
>> >>>>>> Is this correct?
>> >>>>>>
>> >>>>>> What about atomically updating the state store from the transaction
>> >>>>>> buffer every commit interval and writing the checkpoint (thus,
>> >> flushing
>> >>>>>> the memtable) every configured amount of data and/or number of
>> commit
>> >>>>>> intervals? In such a way, we would have the same delay for records
>> >>>>>> appearing in output topics and IQ because both would appear when
>> the
>> >>>>>> Kafka transaction is committed. However, after a failure the state
>> >> store
>> >>>>>> still needs to restore up to
>> statestore.transaction.buffer.max.bytes
>> >> and
>> >>>>>> it might restore data that is already in the state store because
>> the
>> >>>>>> checkpoint lags behind the last stable offset (i.e. the last
>> committed
>> >>>>>> offset) of the changelog topics. Restoring data that is already in
>> the
>> >>>>>> state store is idempotent, so eos should not violated.
>> >>>>>> This solution needs at least one new config to specify when a
>> >> checkpoint
>> >>>>>> should be written.
>> >>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>> A small correction to your previous e-mail that does not change
>> >> anything
>> >>>>>> you said: Under alos the default commit interval is 30 seconds, not
>> >> five
>> >>>>>> seconds.
>> >>>>>>
>> >>>>>>
>> >>>>>> Best,
>> >>>>>> Bruno
>> >>>>>>
>> >>>>>>
>> >>>>>> On 01.07.23 12:37, Nick Telford wrote:
>> >>>>>>> Hi everyone,
>> >>>>>>>
>> >>>>>>> I've begun performance testing my branch on our staging
>> environment,
>> >>>>>>> putting it through its paces in our non-trivial application. I'm
>> >>>>>> already
>> >>>>>>> observing the same increased flush rate that we saw the last time
>> we
>> >>>>>>> attempted to use a version of this KIP, but this time, I think I
>> know
>> >>>>>> why.
>> >>>>>>>
>> >>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end of
>> the
>> >>>>>> Task
>> >>>>>>> commit process, has the following behaviour:
>> >>>>>>>
>> >>>>>>>       - Under ALOS: checkpoint the state stores. This includes
>> >>>>>>>       flushing memtables in RocksDB. This is acceptable because
>> the
>> >>>>>> default
>> >>>>>>>       commit.interval.ms is 5 seconds, so forcibly flushing
>> memtables
>> >>>>>> every 5
>> >>>>>>>       seconds is acceptable for most applications.
>> >>>>>>>       - Under EOS: checkpointing is not done, *unless* it's being
>> >>>>>> forced, due
>> >>>>>>>       to e.g. the Task closing or being revoked. This means that
>> under
>> >>>>>> normal
>> >>>>>>>       processing conditions, the state stores will not be
>> >> checkpointed,
>> >>>>>> and will
>> >>>>>>>       not have memtables flushed at all , unless RocksDB decides
>> to
>> >>>>>> flush them on
>> >>>>>>>       its own. Checkpointing stores and force-flushing their
>> memtables
>> >>>>>> is only
>> >>>>>>>       done when a Task is being closed.
>> >>>>>>>
>> >>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least *some*
>> >> normal
>> >>>>>>> Task commits, in order to write the RocksDB transaction buffers to
>> >> the
>> >>>>>>> state stores, and to ensure the offsets are synced to disk to
>> prevent
>> >>>>>>> restores from getting out of hand. Consequently, my current
>> >>>>>> implementation
>> >>>>>>> calls maybeCheckpoint on *every* Task commit, which is far too
>> >>>>>> frequent.
>> >>>>>>> This causes checkpoints every 10,000 records, which is a change in
>> >>>>>> flush
>> >>>>>>> behaviour, potentially causing performance problems for some
>> >>>>>> applications.
>> >>>>>>>
>> >>>>>>> I'm looking into possible solutions, and I'm currently leaning
>> >> towards
>> >>>>>>> using the statestore.transaction.buffer.max.bytes configuration to
>> >>>>>>> checkpoint Tasks once we are likely to exceed it. This would
>> >>>>>> complement the
>> >>>>>>> existing "early Task commit" functionality that this configuration
>> >>>>>>> provides, in the following way:
>> >>>>>>>
>> >>>>>>>       - Currently, we use statestore.transaction.buffer.max.bytes
>> to
>> >>>>>> force an
>> >>>>>>>       early Task commit if processing more records would cause our
>> >> state
>> >>>>>> store
>> >>>>>>>       transactions to exceed the memory assigned to them.
>> >>>>>>>       - New functionality: when a Task *does* commit, we will not
>> >>>>>> checkpoint
>> >>>>>>>       the stores (and hence flush the transaction buffers) unless
>> we
>> >>>>>> expect to
>> >>>>>>>       cross the statestore.transaction.buffer.max.bytes threshold
>> >> before
>> >>>>>> the next
>> >>>>>>>       commit
>> >>>>>>>
>> >>>>>>> I'm also open to suggestions.
>> >>>>>>>
>> >>>>>>> Regards,
>> >>>>>>> Nick
>> >>>>>>>
>> >>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
>> nick.telford@gmail.com>
>> >>>>>> wrote:
>> >>>>>>>
>> >>>>>>>> Hi Bruno!
>> >>>>>>>>
>> >>>>>>>> 3.
>> >>>>>>>> By "less predictable for users", I meant in terms of
>> understanding
>> >> the
>> >>>>>>>> performance profile under various circumstances. The more complex
>> >> the
>> >>>>>>>> solution, the more difficult it would be for users to understand
>> the
>> >>>>>>>> performance they see. For example, spilling records to disk when
>> the
>> >>>>>>>> transaction buffer reaches a threshold would, I expect, reduce
>> write
>> >>>>>>>> throughput. This reduction in write throughput could be
>> unexpected,
>> >>>>>> and
>> >>>>>>>> potentially difficult to diagnose/understand for users.
>> >>>>>>>> At the moment, I think the "early commit" concept is relatively
>> >>>>>>>> straightforward; it's easy to document, and conceptually fairly
>> >>>>>> obvious to
>> >>>>>>>> users. We could probably add a metric to make it easier to
>> >> understand
>> >>>>>> when
>> >>>>>>>> it happens though.
>> >>>>>>>>
>> >>>>>>>> 3. (the second one)
>> >>>>>>>> The IsolationLevel is *essentially* an indirect way of telling
>> >>>>>> StateStores
>> >>>>>>>> whether they should be transactional. READ_COMMITTED essentially
>> >>>>>> requires
>> >>>>>>>> transactions, because it dictates that two threads calling
>> >>>>>>>> `newTransaction()` should not see writes from the other
>> transaction
>> >>>>>> until
>> >>>>>>>> they have been committed. With READ_UNCOMMITTED, all bets are
>> off,
>> >> and
>> >>>>>>>> stores can allow threads to observe written records at any time,
>> >>>>>> which is
>> >>>>>>>> essentially "no transactions". That said, StateStores are free to
>> >>>>>> implement
>> >>>>>>>> these guarantees however they can, which is a bit more relaxed
>> than
>> >>>>>>>> dictating "you must use transactions". For example, with RocksDB
>> we
>> >>>>>> would
>> >>>>>>>> implement these as READ_COMMITTED == WBWI-based "transactions",
>> >>>>>>>> READ_UNCOMMITTED == direct writes to the database. But with other
>> >>>>>> storage
>> >>>>>>>> engines, it might be preferable to *always* use transactions,
>> even
>> >>>>>> when
>> >>>>>>>> unnecessary; or there may be storage engines that don't provide
>> >>>>>>>> transactions, but the isolation guarantees can be met using a
>> >>>>>> different
>> >>>>>>>> technique.
>> >>>>>>>> My idea was to try to keep the StateStore interface as loosely
>> >> coupled
>> >>>>>>>> from the Streams engine as possible, to give implementers more
>> >>>>>> freedom, and
>> >>>>>>>> reduce the amount of internal knowledge required.
>> >>>>>>>> That said, I understand that "IsolationLevel" might not be the
>> right
>> >>>>>>>> abstraction, and we can always make it much more explicit if
>> >>>>>> required, e.g.
>> >>>>>>>> boolean transactional()
>> >>>>>>>>
>> >>>>>>>> 7-8.
>> >>>>>>>> I can make these changes either later today or tomorrow.
>> >>>>>>>>
>> >>>>>>>> Small update:
>> >>>>>>>> I've rebased my branch on trunk and fixed a bunch of issues that
>> >>>>>> needed
>> >>>>>>>> addressing. Currently, all the tests pass, which is promising,
>> but
>> >> it
>> >>>>>> will
>> >>>>>>>> need to undergo some performance testing. I haven't (yet) worked
>> on
>> >>>>>>>> removing the `newTransaction()` stuff, but I would expect that,
>> >>>>>>>> behaviourally, it should make no difference. The branch is
>> available
>> >>>>>> at
>> >>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
>> >>>>>>>> interested in taking an early look.
>> >>>>>>>>
>> >>>>>>>> Regards,
>> >>>>>>>> Nick
>> >>>>>>>>
>> >>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org>
>> >>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>>> Hi Nick,
>> >>>>>>>>>
>> >>>>>>>>> 1.
>> >>>>>>>>> Yeah, I agree with you. That was actually also my point. I
>> >> understood
>> >>>>>>>>> that John was proposing the ingestion path as a way to avoid the
>> >>>>>> early
>> >>>>>>>>> commits. Probably, I misinterpreted the intent.
>> >>>>>>>>>
>> >>>>>>>>> 2.
>> >>>>>>>>> I agree with John here, that actually it is public API. My
>> question
>> >>>>>> is
>> >>>>>>>>> how this usage pattern affects normal processing.
>> >>>>>>>>>
>> >>>>>>>>> 3.
>> >>>>>>>>> My concern is that checking for the size of the transaction
>> buffer
>> >>>>>> and
>> >>>>>>>>> maybe triggering an early commit affects the whole processing of
>> >>>>>> Kafka
>> >>>>>>>>> Streams. The transactionality of a state store is not confined
>> to
>> >> the
>> >>>>>>>>> state store itself, but spills over and changes the behavior of
>> >> other
>> >>>>>>>>> parts of the system. I agree with you that it is a decent
>> >>>>>> compromise. I
>> >>>>>>>>> just wanted to analyse the downsides and list the options to
>> >> overcome
>> >>>>>>>>> them. I also agree with you that all options seem quite heavy
>> >>>>>> compared
>> >>>>>>>>> with your KIP. I do not understand what you mean with "less
>> >>>>>> predictable
>> >>>>>>>>> for users", though.
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> I found the discussions about the alternatives really
>> interesting.
>> >>>>>> But I
>> >>>>>>>>> also think that your plan sounds good and we should continue
>> with
>> >> it!
>> >>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>> Some comments on your reply to my e-mail on June 20th:
>> >>>>>>>>>
>> >>>>>>>>> 3.
>> >>>>>>>>> Ah, now, I understand the reasoning behind putting isolation
>> level
>> >> in
>> >>>>>>>>> the state store context. Thanks! Should that also be a way to
>> give
>> >>>>>> the
>> >>>>>>>>> the state store the opportunity to decide whether to turn on
>> >>>>>>>>> transactions or not?
>> >>>>>>>>> With my comment, I was more concerned about how do you know if a
>> >>>>>>>>> checkpoint file needs to be written under EOS, if you do not
>> have a
>> >>>>>> way
>> >>>>>>>>> to know if the state store is transactional or not. If a state
>> >> store
>> >>>>>> is
>> >>>>>>>>> transactional, the checkpoint file can be written during normal
>> >>>>>>>>> processing under EOS. If the state store is not transactional,
>> the
>> >>>>>>>>> checkpoint file must not be written under EOS.
>> >>>>>>>>>
>> >>>>>>>>> 7.
>> >>>>>>>>> My point was about not only considering the bytes in memory in
>> >> config
>> >>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that might be
>> >>>>>> spilled
>> >>>>>>>>> on disk. Basically, I was wondering whether you should remove
>> the
>> >>>>>>>>> "memory" in "Maximum number of memory bytes to be used to
>> >>>>>>>>> buffer uncommitted state-store records." My thinking was that
>> even
>> >>>>>> if a
>> >>>>>>>>> state store spills uncommitted bytes to disk, limiting the
>> overall
>> >>>>>> bytes
>> >>>>>>>>> might make sense. Thinking about it again and considering the
>> >> recent
>> >>>>>>>>> discussions, it does not make too much sense anymore.
>> >>>>>>>>> I like the name statestore.transaction.buffer.max.bytes that you
>> >>>>>> proposed.
>> >>>>>>>>>
>> >>>>>>>>> 8.
>> >>>>>>>>> A high-level description (without implementation details) of how
>> >>>>>> Kafka
>> >>>>>>>>> Streams will manage the commit of changelog transactions, state
>> >> store
>> >>>>>>>>> transactions and checkpointing would be great. Would be great if
>> >> you
>> >>>>>>>>> could also add some sentences about the behavior in case of a
>> >>>>>> failure.
>> >>>>>>>>> For instance how does a transactional state store recover after
>> a
>> >>>>>>>>> failure or what happens with the transaction buffer, etc. (that
>> is
>> >>>>>> what
>> >>>>>>>>> I meant by "fail-over" in point 9.)
>> >>>>>>>>>
>> >>>>>>>>> Best,
>> >>>>>>>>> Bruno
>> >>>>>>>>>
>> >>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
>> >>>>>>>>>> Hi Bruno,
>> >>>>>>>>>>
>> >>>>>>>>>> 1.
>> >>>>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
>> >>>>>> transactions
>> >>>>>>>>>> have, whereby exceeding (or likely to exceed) configured memory
>> >>>>>> needs to
>> >>>>>>>>>> trigger an early commit?
>> >>>>>>>>>>
>> >>>>>>>>>> 2.
>> >>>>>>>>>> This is one of my big concerns. Ultimately, any approach based
>> on
>> >>>>>>>>> cracking
>> >>>>>>>>>> open RocksDB internals and using it in ways it's not really
>> >> designed
>> >>>>>>>>> for is
>> >>>>>>>>>> likely to have some unforseen performance or consistency
>> issues.
>> >>>>>>>>>>
>> >>>>>>>>>> 3.
>> >>>>>>>>>> What's your motivation for removing these early commits? While
>> not
>> >>>>>>>>> ideal, I
>> >>>>>>>>>> think they're a decent compromise to ensure consistency whilst
>> >>>>>>>>> maintaining
>> >>>>>>>>>> good and predictable performance.
>> >>>>>>>>>> All 3 of your suggested ideas seem *very* complicated, and
>> might
>> >>>>>>>>> actually
>> >>>>>>>>>> make behaviour less predictable for users as a consequence.
>> >>>>>>>>>>
>> >>>>>>>>>> I'm a bit concerned that the scope of this KIP is growing a bit
>> >> out
>> >>>>>> of
>> >>>>>>>>>> control. While it's good to discuss ideas for future
>> >> improvements, I
>> >>>>>>>>> think
>> >>>>>>>>>> it's important to narrow the scope down to a design that
>> achieves
>> >>>>>> the
>> >>>>>>>>> most
>> >>>>>>>>>> pressing objectives (constant sized restorations during dirty
>> >>>>>>>>>> close/unexpected errors). Any design that this KIP produces can
>> >>>>>>>>> ultimately
>> >>>>>>>>>> be changed in the future, especially if the bulk of it is
>> internal
>> >>>>>>>>>> behaviour.
>> >>>>>>>>>>
>> >>>>>>>>>> I'm going to spend some time next week trying to re-work the
>> >>>>>> original
>> >>>>>>>>>> WriteBatchWithIndex design to remove the newTransaction()
>> method,
>> >>>>>> such
>> >>>>>>>>> that
>> >>>>>>>>>> it's just an implementation detail of RocksDBStore. That way,
>> if
>> >> we
>> >>>>>>>>> want to
>> >>>>>>>>>> replace WBWI with something in the future, like the SST file
>> >>>>>> management
>> >>>>>>>>>> outlined by John, then we can do so with little/no API changes.
>> >>>>>>>>>>
>> >>>>>>>>>> Regards,
>> >>>>>>>>>>
>> >>>>>>>>>> Nick
>> >>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>
>> >>>
>> >>
>> >
>>
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Bruno,

Thanks for getting back to me!

2.
The fact that implementations can always track estimated memory usage in
the wrapper is a good point. I can remove -1 as an option, and I'll clarify
the JavaDoc that 0 is not just for non-transactional stores, which is
currently misleading.

6.
The problem with catching the exception in the downgrade process is that
would require new code in the Kafka version being downgraded to. Since
users could conceivably downgrade to almost *any* older version of Kafka
Streams, I'm not sure how we could add that code?
The only way I can think of doing it would be to provide a dedicated
downgrade tool, that goes through every local store and removes the
offsets column families. But that seems like an unnecessary amount of extra
code to maintain just to handle a somewhat niche situation, when the
alternative (automatically wipe and restore stores) should be acceptable.

1, 4, 5: Agreed. I'll make the changes you've requested.

3a.
I agree that IsolationLevel makes more sense at query-time, and I actually
initially attempted to place the IsolationLevel at query-time, but I ran
into some problems:
- The key issue is that, under ALOS we're not staging writes in
transactions, so can't perform writes at the READ_COMMITTED isolation
level. However, this may be addressed if we decide to *always* use
transactions as discussed under 3b.
- IQv1 and IQv2 have quite different implementations. I remember having
some difficulty understanding the IQv1 internals, which made it difficult
to determine what needed to be changed. However, I *think* this can be
addressed for both implementations by wrapping the RocksDBStore in an
IsolationLevel-dependent wrapper, that overrides read methods (get, etc.)
to either read directly from the database or from the ongoing transaction.
But IQv1 might still be difficult.
- If IsolationLevel becomes a query constraint, then all other StateStores
will need to respect it, including the in-memory stores. This would require
us to adapt in-memory stores to stage their writes so they can be isolated
from READ_COMMITTTED queries. It would also become an important
consideration for third-party stores on upgrade, as without changes, they
would not support READ_COMMITTED queries correctly.

Ultimately, I may need some help making the necessary change to IQv1 to
support this, but I don't think it's fundamentally impossible, if we want
to pursue this route.

3b.
The main reason I chose to keep ALOS un-transactional was to minimize
behavioural change for most users (I believe most Streams users use the
default configuration, which is ALOS). That said, it's clear that if ALOS
also used transactional stores, the only change in behaviour would be that
it would become *more correct*, which could be considered a "bug fix" by
users, rather than a change they need to handle.

I believe that performance using transactions (aka. RocksDB WriteBatches)
should actually be *better* than the un-batched write-path that is
currently used[1]. The only "performance" consideration will be the
increased memory usage that transactions require. Given the mitigations for
this memory that we have in place, I would expect that this is not a
problem for most users.

If we're happy to do so, we can make ALOS also use transactions.

Regards,
Nick

Link 1:
https://github.com/adamretter/rocksjava-write-methods-benchmark#results

On Wed, 13 Sept 2023 at 09:41, Bruno Cadonna <ca...@apache.org> wrote:

> Hi Nick,
>
> Thanks for the updates and sorry for the delay on my side!
>
>
> 1.
> Making the default implementation for flush() a no-op sounds good to me.
>
>
> 2.
> I think what was bugging me here is that a third-party state store needs
> to implement the state store interface. That means they need to
> implement a wrapper around the actual state store as we do for RocksDB
> with RocksDBStore. So, a third-party state store can always estimate the
> uncommitted bytes, if it wants, because the wrapper can record the added
> bytes.
> One case I can think of where returning -1 makes sense is when Streams
> does not need to estimate the size of the write batch and trigger
> extraordinary commits, because the third-party state store takes care of
> memory. But in that case the method could also just return 0. Even that
> case would be better solved with a method that returns whether the state
> store manages itself the memory used for uncommitted bytes or not.
> Said that, I am fine with keeping the -1 return value, I was just
> wondering when and if it will be used.
>
> Regarding returning 0 for transactional state stores when the batch is
> empty, I was just wondering because you explicitly stated
>
> "or {@code 0} if this StateStore does not support transactions."
>
> So it seemed to me returning 0 could only happen for non-transactional
> state stores.
>
>
> 3.
>
> a) What do you think if we move the isolation level to IQ (v1 and v2)?
> In the end this is the only component that really needs to specify the
> isolation level. It is similar to the Kafka consumer that can choose
> with what isolation level to read the input topic.
> For IQv1 the isolation level should go into StoreQueryParameters. For
> IQv2, I would add it to the Query interface.
>
> b) Point a) raises the question what should happen during at-least-once
> processing when the state store does not use transactions? John in the
> past proposed to also use transactions on state stores for
> at-least-once. I like that idea, because it avoids aggregating the same
> records over and over again in the case of a failure. We had a case in
> the past where a Streams applications in at-least-once mode was failing
> continuously for some reasons I do not remember before committing the
> offsets. After each failover, the app aggregated again and again the
> same records. Of course the aggregate increased to very wrong values
> just because of the failover. With transactions on the state stores we
> could have avoided this. The app would have output the same aggregate
> multiple times (i.e., after each failover) but at least the value of the
> aggregate would not depend on the number of failovers. Outputting the
> same aggregate multiple times would be incorrect under exactly-once but
> it is OK for at-least-once.
> If it makes sense to add a config to turn on and off transactions on
> state stores under at-least-once or just use transactions in any case is
> a question we should also discuss in this KIP. It depends a bit on the
> performance trade-off. Maybe to be safe, I would add a config.
>
>
> 4.
> Your points are all valid. I tend to say to keep the metrics around
> flush() until we remove flush() completely from the interface. Calls to
> flush() might still exist since existing processors might still call
> flush() explicitly as you mentioned in 1). For sure, we need to document
> how the metrics change due to the transactions in the upgrade notes.
>
>
> 5.
> I see. Then you should describe how the .position files are handled  in
> a dedicated section of the KIP or incorporate the description in the
> "Atomic Checkpointing" section instead of only mentioning it in the
> "Compatibility, 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:
>
>
> https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75
>
> It is a bit brittle, but it works.
>
>
> Best,
> Bruno
>
>
> On 8/24/23 12:18 PM, Nick Telford wrote:
> > Hi Bruno,
> >
> > Thanks for taking the time to review the KIP. I'm back from leave now and
> > intend to move this forwards as quickly as I can.
> >
> > Addressing your points:
> >
> > 1.
> > Because flush() is part of the StateStore API, it's exposed to custom
> > Processors, which might be making calls to flush(). This was actually the
> > case in a few integration tests.
> > To maintain as much compatibility as possible, I'd prefer not to make
> this
> > an UnsupportedOperationException, as it will cause previously working
> > Processors to start throwing exceptions at runtime.
> > I agree that it doesn't make sense for it to proxy commit(), though, as
> > that would cause it to violate the "StateStores commit only when the Task
> > commits" rule.
> > Instead, I think we should make this a no-op. That way, existing user
> > Processors will continue to work as-before, without violation of store
> > consistency that would be caused by premature flush/commit of StateStore
> > data to disk.
> > What do you think?
> >
> > 2.
> > As stated in the JavaDoc, when a StateStore implementation is
> > transactional, but is unable to estimate the uncommitted memory usage,
> the
> > method will return -1.
> > The intention here is to permit third-party implementations that may not
> be
> > able to estimate memory usage.
> >
> > Yes, it will be 0 when nothing has been written to the store yet. I
> thought
> > that was implied by "This method will return an approximation of the
> memory
> > would be freed by the next call to {@link #commit(Map)}" and "@return The
> > approximate size of all records awaiting {@link #commit(Map)}", however,
> I
> > can add it explicitly to the JavaDoc if you think this is unclear?
> >
> > 3.
> > I realise this is probably the most contentious point in my design, and
> I'm
> > open to changing it if I'm unable to convince you of the benefits.
> > Nevertheless, here's my argument:
> > The Interactive Query (IQ) API(s) are directly provided StateStores to
> > query, and it may be important for users to programmatically know which
> > mode the StateStore is operating under. If we simply provide an
> > "eosEnabled" boolean (as used throughout the internal streams engine), or
> > similar, then users will need to understand the operation and
> consequences
> > of each available processing mode and how it pertains to their
> StateStore.
> >
> > Interactive Query users aren't the only people that care about the
> > processing.mode/IsolationLevel of a StateStore: implementers of custom
> > StateStores also need to understand the behaviour expected of their
> > implementation. KIP-892 introduces some assumptions into the Streams
> Engine
> > about how StateStores operate under each processing mode, and it's
> > important that custom implementations adhere to those assumptions in
> order
> > to maintain the consistency guarantees.
> >
> > IsolationLevels provide a high-level contract on the behaviour of the
> > StateStore: a user knows that under READ_COMMITTED, they will see writes
> > only after the Task has committed, and under READ_UNCOMMITTED they will
> see
> > writes immediately. No understanding of the details of each
> processing.mode
> > is required, either for IQ users or StateStore implementers.
> >
> > An argument can be made that these contractual guarantees can simply be
> > documented for the processing.mode (i.e. that exactly-once and
> > exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves like
> > READ_UNCOMMITTED), but there are several small issues with this I'd
> prefer
> > to avoid:
> >
> >     - Where would we document these contracts, in a way that is difficult
> >     for users/implementers to miss/ignore?
> >     - It's not clear to users that the processing mode is communicating
> >     an expectation of read isolation, unless they read the
> documentation. Users
> >     rarely consult documentation unless they feel they need to, so it's
> likely
> >     this detail would get missed by many users.
> >     - It tightly couples processing modes to read isolation. Adding new
> >     processing modes, or changing the read isolation of existing
> processing
> >     modes would be difficult/impossible.
> >
> > Ultimately, the cost of introducing IsolationLevels is just a single
> > method, since we re-use the existing IsolationLevel enum from Kafka. This
> > gives us a clear place to document the contractual guarantees expected
> > of/provided by StateStores, that is accessible both by the StateStore
> > itself, and by IQ users.
> >
> > (Writing this I've just realised that the StateStore and IQ APIs actually
> > don't provide access to StateStoreContext that IQ users would have direct
> > access to... Perhaps StateStore should expose isolationLevel() itself
> too?)
> >
> > 4.
> > Yeah, I'm not comfortable renaming the metrics in-place either, as it's a
> > backwards incompatible change. My concern is that, if we leave the
> existing
> > "flush" metrics in place, they will be confusing to users. Right now,
> > "flush" metrics record explicit flushes to disk, but under KIP-892, even
> a
> > commit() will not explicitly flush data to disk - RocksDB will decide on
> > when to flush memtables to disk itself.
> >
> > If we keep the existing "flush" metrics, we'd have two options, which
> both
> > seem pretty bad to me:
> >
> >     1. Have them record calls to commit(), which would be misleading, as
> >     data is no longer explicitly "flushed" to disk by this call.
> >     2. Have them record nothing at all, which is equivalent to removing
> the
> >     metrics, except that users will see the metric still exists and so
> assume
> >     that the metric is correct, and that there's a problem with their
> system
> >     when there isn't.
> >
> > I agree that removing them is also a bad solution, and I'd like some
> > guidance on the best path forward here.
> >
> > 5.
> > Position files are updated on every write to a StateStore. Since our
> writes
> > are now buffered until commit(), we can't update the Position file until
> > commit() has been called, otherwise it would be inconsistent with the
> data
> > in the event of a rollback. Consequently, we need to manage these offsets
> > the same way we manage the checkpoint offsets, and ensure they're only
> > written on commit().
> >
> > 6.
> > Agreed, although I'm not exactly sure yet what tests to write. How
> explicit
> > do we need to be here in the KIP?
> >
> > As for upgrade/downgrade: upgrade is designed to be seamless, and we
> should
> > definitely add some tests around that. Downgrade, it transpires, isn't
> > currently possible, as the extra column family for offset storage is
> > incompatible with the pre-KIP-892 implementation: when you open a RocksDB
> > database, you must open all available column families or receive an
> error.
> > What currently happens on downgrade is that it attempts to open the
> store,
> > throws an error about the offsets column family not being opened, which
> > triggers a wipe and rebuild of the Task. Given that downgrades should be
> > uncommon, I think this is acceptable behaviour, as the end-state is
> > consistent, even if it results in an undesirable state restore.
> >
> > Should I document the upgrade/downgrade behaviour explicitly in the KIP?
> >
> > --
> >
> > Regards,
> > Nick
> >
> >
> > On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org> wrote:
> >
> >> Hi Nick!
> >>
> >> Thanks for the updates!
> >>
> >> 1.
> >> Why does StateStore#flush() default to
> >> StateStore#commit(Collections.emptyMap())?
> >> Since calls to flush() will not exist anymore after this KIP is
> >> released, I would rather throw an unsupported operation exception by
> >> default.
> >>
> >>
> >> 2.
> >> When would a state store return -1 from
> >> StateStore#approximateNumUncommittedBytes() while being transactional?
> >>
> >> Wouldn't StateStore#approximateNumUncommittedBytes() also return 0 if
> >> the state store is transactional but nothing has been written to the
> >> state store yet?
> >>
> >>
> >> 3.
> >> Sorry for bringing this up again. Does this KIP really need to introduce
> >> StateStoreContext#isolationLevel()? StateStoreContext has already
> >> appConfigs() which basically exposes the same information, i.e., if EOS
> >> is enabled or not.
> >> In one of your previous e-mails you wrote:
> >>
> >> "My idea was to try to keep the StateStore interface as loosely coupled
> >> from the Streams engine as possible, to give implementers more freedom,
> >> and reduce the amount of internal knowledge required."
> >>
> >> While I understand the intent, I doubt that it decreases the coupling of
> >> a StateStore interface and the Streams engine. READ_COMMITTED only
> >> applies to IQ but not to reads by processors. Thus, implementers need to
> >> understand how Streams accesses the state stores.
> >>
> >> I would like to hear what others think about this.
> >>
> >>
> >> 4.
> >> Great exposing new metrics for transactional state stores! However, I
> >> would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides unit
> >> tests. Additionally, we probably need integration or system tests to
> >> verify that upgrades and downgrades between transactional and
> >> non-transactional state stores work as expected.
> >>
> >>
> >> Best,
> >> Bruno
> >>
> >>
> >>
> >>
> >>
> >> On 7/21/23 10:34 PM, Nick Telford wrote:
> >>> One more thing: I noted John's suggestion in the KIP, under "Rejected
> >>> Alternatives". I still think it's an idea worth pursuing, but I believe
> >>> that it's out of the scope of this KIP, because it solves a different
> set
> >>> of problems to this KIP, and the scope of this one has already grown
> >> quite
> >>> large!
> >>>
> >>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <ni...@gmail.com>
> >> wrote:
> >>>
> >>>> Hi everyone,
> >>>>
> >>>> I've updated the KIP (
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >> )
> >>>> with the latest changes; mostly bringing back "Atomic Checkpointing"
> >> (for
> >>>> what feels like the 10th time!). I think the one thing missing is some
> >>>> changes to metrics (notably the store "flush" metrics will need to be
> >>>> renamed to "commit").
> >>>>
> >>>> The reason I brought back Atomic Checkpointing was to decouple store
> >> flush
> >>>> from store commit. This is important, because with Transactional
> >>>> StateStores, we now need to call "flush" on *every* Task commit, and
> not
> >>>> just when the StateStore is closing, otherwise our transaction buffer
> >> will
> >>>> never be written and persisted, instead growing unbounded! I
> >> experimented
> >>>> with some simple solutions, like forcing a store flush whenever the
> >>>> transaction buffer was likely to exceed its configured size, but this
> >> was
> >>>> brittle: it prevented the transaction buffer from being configured to
> be
> >>>> unbounded, and it still would have required explicit flushes of
> RocksDB,
> >>>> yielding sub-optimal performance and memory utilization.
> >>>>
> >>>> I deemed Atomic Checkpointing to be the "right" way to resolve this
> >>>> problem. By ensuring that the changelog offsets that correspond to the
> >> most
> >>>> recently written records are always atomically written to the
> StateStore
> >>>> (by writing them to the same transaction buffer), we can avoid
> forcibly
> >>>> flushing the RocksDB memtables to disk, letting RocksDB flush them
> only
> >>>> when necessary, without losing any of our consistency guarantees. See
> >> the
> >>>> updated KIP for more info.
> >>>>
> >>>> I have fully implemented these changes, although I'm still not
> entirely
> >>>> happy with the implementation for segmented StateStores, so I plan to
> >>>> refactor that. Despite that, all tests pass. If you'd like to try out
> or
> >>>> review this highly experimental and incomplete branch, it's available
> >> here:
> >>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note: it's
> >> built
> >>>> against Kafka 3.5.0 so that I had a stable base to build and test it
> on,
> >>>> and to enable easy apples-to-apples comparisons in a live
> environment. I
> >>>> plan to rebase it against trunk once it's nearer completion and has
> been
> >>>> proven on our main application.
> >>>>
> >>>> I would really appreciate help in reviewing and testing:
> >>>> - Segmented (Versioned, Session and Window) stores
> >>>> - Global stores
> >>>>
> >>>> As I do not currently use either of these, so my primary test
> >> environment
> >>>> doesn't test these areas.
> >>>>
> >>>> I'm going on Parental Leave starting next week for a few weeks, so
> will
> >>>> not have time to move this forward until late August. That said, your
> >>>> feedback is welcome and appreciated, I just won't be able to respond
> as
> >>>> quickly as usual.
> >>>>
> >>>> Regards,
> >>>> Nick
> >>>>
> >>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com>
> >> wrote:
> >>>>
> >>>>> Hi Bruno
> >>>>>
> >>>>> Yes, that's correct, although the impact on IQ is not something I had
> >>>>> considered.
> >>>>>
> >>>>> What about atomically updating the state store from the transaction
> >>>>>> buffer every commit interval and writing the checkpoint (thus,
> >> flushing
> >>>>>> the memtable) every configured amount of data and/or number of
> commit
> >>>>>> intervals?
> >>>>>>
> >>>>>
> >>>>> I'm not quite sure I follow. Are you suggesting that we add an
> >> additional
> >>>>> config for the max number of commit intervals between checkpoints?
> That
> >>>>> way, we would checkpoint *either* when the transaction buffers are
> >> nearly
> >>>>> full, *OR* whenever a certain number of commit intervals have
> elapsed,
> >>>>> whichever comes first?
> >>>>>
> >>>>> That certainly seems reasonable, although this re-ignites an earlier
> >>>>> debate about whether a config should be measured in "number of commit
> >>>>> intervals", instead of just an absolute time.
> >>>>>
> >>>>> FWIW, I realised that this issue is the reason I was pursuing the
> >> Atomic
> >>>>> Checkpoints, as it de-couples memtable flush from checkpointing,
> which
> >>>>> enables us to just checkpoint on every commit without any performance
> >>>>> impact. Atomic Checkpointing is definitely the "best" solution, but
> >> I'm not
> >>>>> sure if this is enough to bring it back into this KIP.
> >>>>>
> >>>>> I'm currently working on moving all the transactional logic directly
> >> into
> >>>>> RocksDBStore itself, which does away with the
> StateStore#newTransaction
> >>>>> method, and reduces the number of new classes introduced,
> significantly
> >>>>> reducing the complexity. If it works, and the complexity is
> drastically
> >>>>> reduced, I may try bringing back Atomic Checkpoints into this KIP.
> >>>>>
> >>>>> Regards,
> >>>>> Nick
> >>>>>
> >>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org>
> wrote:
> >>>>>
> >>>>>> Hi Nick,
> >>>>>>
> >>>>>> Thanks for the insights! Very interesting!
> >>>>>>
> >>>>>> As far as I understand, you want to atomically update the state
> store
> >>>>>> from the transaction buffer, flush the memtable of a state store and
> >>>>>> write the checkpoint not after the commit time elapsed but after the
> >>>>>> transaction buffer reached a size that would lead to exceeding
> >>>>>> statestore.transaction.buffer.max.bytes before the next commit
> >> interval
> >>>>>> ends.
> >>>>>> That means, the Kafka transaction would commit every commit interval
> >> but
> >>>>>> the state store will only be atomically updated roughly every
> >>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ would then
> >> only
> >>>>>> see new data roughly every statestore.transaction.buffer.max.bytes.
> >>>>>> After a failure the state store needs to restore up to
> >>>>>> statestore.transaction.buffer.max.bytes.
> >>>>>>
> >>>>>> Is this correct?
> >>>>>>
> >>>>>> What about atomically updating the state store from the transaction
> >>>>>> buffer every commit interval and writing the checkpoint (thus,
> >> flushing
> >>>>>> the memtable) every configured amount of data and/or number of
> commit
> >>>>>> intervals? In such a way, we would have the same delay for records
> >>>>>> appearing in output topics and IQ because both would appear when the
> >>>>>> Kafka transaction is committed. However, after a failure the state
> >> store
> >>>>>> still needs to restore up to statestore.transaction.buffer.max.bytes
> >> and
> >>>>>> it might restore data that is already in the state store because the
> >>>>>> checkpoint lags behind the last stable offset (i.e. the last
> committed
> >>>>>> offset) of the changelog topics. Restoring data that is already in
> the
> >>>>>> state store is idempotent, so eos should not violated.
> >>>>>> This solution needs at least one new config to specify when a
> >> checkpoint
> >>>>>> should be written.
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> A small correction to your previous e-mail that does not change
> >> anything
> >>>>>> you said: Under alos the default commit interval is 30 seconds, not
> >> five
> >>>>>> seconds.
> >>>>>>
> >>>>>>
> >>>>>> Best,
> >>>>>> Bruno
> >>>>>>
> >>>>>>
> >>>>>> On 01.07.23 12:37, Nick Telford wrote:
> >>>>>>> Hi everyone,
> >>>>>>>
> >>>>>>> I've begun performance testing my branch on our staging
> environment,
> >>>>>>> putting it through its paces in our non-trivial application. I'm
> >>>>>> already
> >>>>>>> observing the same increased flush rate that we saw the last time
> we
> >>>>>>> attempted to use a version of this KIP, but this time, I think I
> know
> >>>>>> why.
> >>>>>>>
> >>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end of
> the
> >>>>>> Task
> >>>>>>> commit process, has the following behaviour:
> >>>>>>>
> >>>>>>>       - Under ALOS: checkpoint the state stores. This includes
> >>>>>>>       flushing memtables in RocksDB. This is acceptable because the
> >>>>>> default
> >>>>>>>       commit.interval.ms is 5 seconds, so forcibly flushing
> memtables
> >>>>>> every 5
> >>>>>>>       seconds is acceptable for most applications.
> >>>>>>>       - Under EOS: checkpointing is not done, *unless* it's being
> >>>>>> forced, due
> >>>>>>>       to e.g. the Task closing or being revoked. This means that
> under
> >>>>>> normal
> >>>>>>>       processing conditions, the state stores will not be
> >> checkpointed,
> >>>>>> and will
> >>>>>>>       not have memtables flushed at all , unless RocksDB decides to
> >>>>>> flush them on
> >>>>>>>       its own. Checkpointing stores and force-flushing their
> memtables
> >>>>>> is only
> >>>>>>>       done when a Task is being closed.
> >>>>>>>
> >>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least *some*
> >> normal
> >>>>>>> Task commits, in order to write the RocksDB transaction buffers to
> >> the
> >>>>>>> state stores, and to ensure the offsets are synced to disk to
> prevent
> >>>>>>> restores from getting out of hand. Consequently, my current
> >>>>>> implementation
> >>>>>>> calls maybeCheckpoint on *every* Task commit, which is far too
> >>>>>> frequent.
> >>>>>>> This causes checkpoints every 10,000 records, which is a change in
> >>>>>> flush
> >>>>>>> behaviour, potentially causing performance problems for some
> >>>>>> applications.
> >>>>>>>
> >>>>>>> I'm looking into possible solutions, and I'm currently leaning
> >> towards
> >>>>>>> using the statestore.transaction.buffer.max.bytes configuration to
> >>>>>>> checkpoint Tasks once we are likely to exceed it. This would
> >>>>>> complement the
> >>>>>>> existing "early Task commit" functionality that this configuration
> >>>>>>> provides, in the following way:
> >>>>>>>
> >>>>>>>       - Currently, we use statestore.transaction.buffer.max.bytes
> to
> >>>>>> force an
> >>>>>>>       early Task commit if processing more records would cause our
> >> state
> >>>>>> store
> >>>>>>>       transactions to exceed the memory assigned to them.
> >>>>>>>       - New functionality: when a Task *does* commit, we will not
> >>>>>> checkpoint
> >>>>>>>       the stores (and hence flush the transaction buffers) unless
> we
> >>>>>> expect to
> >>>>>>>       cross the statestore.transaction.buffer.max.bytes threshold
> >> before
> >>>>>> the next
> >>>>>>>       commit
> >>>>>>>
> >>>>>>> I'm also open to suggestions.
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>> Nick
> >>>>>>>
> >>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <nick.telford@gmail.com
> >
> >>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi Bruno!
> >>>>>>>>
> >>>>>>>> 3.
> >>>>>>>> By "less predictable for users", I meant in terms of understanding
> >> the
> >>>>>>>> performance profile under various circumstances. The more complex
> >> the
> >>>>>>>> solution, the more difficult it would be for users to understand
> the
> >>>>>>>> performance they see. For example, spilling records to disk when
> the
> >>>>>>>> transaction buffer reaches a threshold would, I expect, reduce
> write
> >>>>>>>> throughput. This reduction in write throughput could be
> unexpected,
> >>>>>> and
> >>>>>>>> potentially difficult to diagnose/understand for users.
> >>>>>>>> At the moment, I think the "early commit" concept is relatively
> >>>>>>>> straightforward; it's easy to document, and conceptually fairly
> >>>>>> obvious to
> >>>>>>>> users. We could probably add a metric to make it easier to
> >> understand
> >>>>>> when
> >>>>>>>> it happens though.
> >>>>>>>>
> >>>>>>>> 3. (the second one)
> >>>>>>>> The IsolationLevel is *essentially* an indirect way of telling
> >>>>>> StateStores
> >>>>>>>> whether they should be transactional. READ_COMMITTED essentially
> >>>>>> requires
> >>>>>>>> transactions, because it dictates that two threads calling
> >>>>>>>> `newTransaction()` should not see writes from the other
> transaction
> >>>>>> until
> >>>>>>>> they have been committed. With READ_UNCOMMITTED, all bets are off,
> >> and
> >>>>>>>> stores can allow threads to observe written records at any time,
> >>>>>> which is
> >>>>>>>> essentially "no transactions". That said, StateStores are free to
> >>>>>> implement
> >>>>>>>> these guarantees however they can, which is a bit more relaxed
> than
> >>>>>>>> dictating "you must use transactions". For example, with RocksDB
> we
> >>>>>> would
> >>>>>>>> implement these as READ_COMMITTED == WBWI-based "transactions",
> >>>>>>>> READ_UNCOMMITTED == direct writes to the database. But with other
> >>>>>> storage
> >>>>>>>> engines, it might be preferable to *always* use transactions, even
> >>>>>> when
> >>>>>>>> unnecessary; or there may be storage engines that don't provide
> >>>>>>>> transactions, but the isolation guarantees can be met using a
> >>>>>> different
> >>>>>>>> technique.
> >>>>>>>> My idea was to try to keep the StateStore interface as loosely
> >> coupled
> >>>>>>>> from the Streams engine as possible, to give implementers more
> >>>>>> freedom, and
> >>>>>>>> reduce the amount of internal knowledge required.
> >>>>>>>> That said, I understand that "IsolationLevel" might not be the
> right
> >>>>>>>> abstraction, and we can always make it much more explicit if
> >>>>>> required, e.g.
> >>>>>>>> boolean transactional()
> >>>>>>>>
> >>>>>>>> 7-8.
> >>>>>>>> I can make these changes either later today or tomorrow.
> >>>>>>>>
> >>>>>>>> Small update:
> >>>>>>>> I've rebased my branch on trunk and fixed a bunch of issues that
> >>>>>> needed
> >>>>>>>> addressing. Currently, all the tests pass, which is promising, but
> >> it
> >>>>>> will
> >>>>>>>> need to undergo some performance testing. I haven't (yet) worked
> on
> >>>>>>>> removing the `newTransaction()` stuff, but I would expect that,
> >>>>>>>> behaviourally, it should make no difference. The branch is
> available
> >>>>>> at
> >>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
> >>>>>>>> interested in taking an early look.
> >>>>>>>>
> >>>>>>>> Regards,
> >>>>>>>> Nick
> >>>>>>>>
> >>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org>
> >>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi Nick,
> >>>>>>>>>
> >>>>>>>>> 1.
> >>>>>>>>> Yeah, I agree with you. That was actually also my point. I
> >> understood
> >>>>>>>>> that John was proposing the ingestion path as a way to avoid the
> >>>>>> early
> >>>>>>>>> commits. Probably, I misinterpreted the intent.
> >>>>>>>>>
> >>>>>>>>> 2.
> >>>>>>>>> I agree with John here, that actually it is public API. My
> question
> >>>>>> is
> >>>>>>>>> how this usage pattern affects normal processing.
> >>>>>>>>>
> >>>>>>>>> 3.
> >>>>>>>>> My concern is that checking for the size of the transaction
> buffer
> >>>>>> and
> >>>>>>>>> maybe triggering an early commit affects the whole processing of
> >>>>>> Kafka
> >>>>>>>>> Streams. The transactionality of a state store is not confined to
> >> the
> >>>>>>>>> state store itself, but spills over and changes the behavior of
> >> other
> >>>>>>>>> parts of the system. I agree with you that it is a decent
> >>>>>> compromise. I
> >>>>>>>>> just wanted to analyse the downsides and list the options to
> >> overcome
> >>>>>>>>> them. I also agree with you that all options seem quite heavy
> >>>>>> compared
> >>>>>>>>> with your KIP. I do not understand what you mean with "less
> >>>>>> predictable
> >>>>>>>>> for users", though.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> I found the discussions about the alternatives really
> interesting.
> >>>>>> But I
> >>>>>>>>> also think that your plan sounds good and we should continue with
> >> it!
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Some comments on your reply to my e-mail on June 20th:
> >>>>>>>>>
> >>>>>>>>> 3.
> >>>>>>>>> Ah, now, I understand the reasoning behind putting isolation
> level
> >> in
> >>>>>>>>> the state store context. Thanks! Should that also be a way to
> give
> >>>>>> the
> >>>>>>>>> the state store the opportunity to decide whether to turn on
> >>>>>>>>> transactions or not?
> >>>>>>>>> With my comment, I was more concerned about how do you know if a
> >>>>>>>>> checkpoint file needs to be written under EOS, if you do not
> have a
> >>>>>> way
> >>>>>>>>> to know if the state store is transactional or not. If a state
> >> store
> >>>>>> is
> >>>>>>>>> transactional, the checkpoint file can be written during normal
> >>>>>>>>> processing under EOS. If the state store is not transactional,
> the
> >>>>>>>>> checkpoint file must not be written under EOS.
> >>>>>>>>>
> >>>>>>>>> 7.
> >>>>>>>>> My point was about not only considering the bytes in memory in
> >> config
> >>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that might be
> >>>>>> spilled
> >>>>>>>>> on disk. Basically, I was wondering whether you should remove the
> >>>>>>>>> "memory" in "Maximum number of memory bytes to be used to
> >>>>>>>>> buffer uncommitted state-store records." My thinking was that
> even
> >>>>>> if a
> >>>>>>>>> state store spills uncommitted bytes to disk, limiting the
> overall
> >>>>>> bytes
> >>>>>>>>> might make sense. Thinking about it again and considering the
> >> recent
> >>>>>>>>> discussions, it does not make too much sense anymore.
> >>>>>>>>> I like the name statestore.transaction.buffer.max.bytes that you
> >>>>>> proposed.
> >>>>>>>>>
> >>>>>>>>> 8.
> >>>>>>>>> A high-level description (without implementation details) of how
> >>>>>> Kafka
> >>>>>>>>> Streams will manage the commit of changelog transactions, state
> >> store
> >>>>>>>>> transactions and checkpointing would be great. Would be great if
> >> you
> >>>>>>>>> could also add some sentences about the behavior in case of a
> >>>>>> failure.
> >>>>>>>>> For instance how does a transactional state store recover after a
> >>>>>>>>> failure or what happens with the transaction buffer, etc. (that
> is
> >>>>>> what
> >>>>>>>>> I meant by "fail-over" in point 9.)
> >>>>>>>>>
> >>>>>>>>> Best,
> >>>>>>>>> Bruno
> >>>>>>>>>
> >>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> >>>>>>>>>> Hi Bruno,
> >>>>>>>>>>
> >>>>>>>>>> 1.
> >>>>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
> >>>>>> transactions
> >>>>>>>>>> have, whereby exceeding (or likely to exceed) configured memory
> >>>>>> needs to
> >>>>>>>>>> trigger an early commit?
> >>>>>>>>>>
> >>>>>>>>>> 2.
> >>>>>>>>>> This is one of my big concerns. Ultimately, any approach based
> on
> >>>>>>>>> cracking
> >>>>>>>>>> open RocksDB internals and using it in ways it's not really
> >> designed
> >>>>>>>>> for is
> >>>>>>>>>> likely to have some unforseen performance or consistency issues.
> >>>>>>>>>>
> >>>>>>>>>> 3.
> >>>>>>>>>> What's your motivation for removing these early commits? While
> not
> >>>>>>>>> ideal, I
> >>>>>>>>>> think they're a decent compromise to ensure consistency whilst
> >>>>>>>>> maintaining
> >>>>>>>>>> good and predictable performance.
> >>>>>>>>>> All 3 of your suggested ideas seem *very* complicated, and might
> >>>>>>>>> actually
> >>>>>>>>>> make behaviour less predictable for users as a consequence.
> >>>>>>>>>>
> >>>>>>>>>> I'm a bit concerned that the scope of this KIP is growing a bit
> >> out
> >>>>>> of
> >>>>>>>>>> control. While it's good to discuss ideas for future
> >> improvements, I
> >>>>>>>>> think
> >>>>>>>>>> it's important to narrow the scope down to a design that
> achieves
> >>>>>> the
> >>>>>>>>> most
> >>>>>>>>>> pressing objectives (constant sized restorations during dirty
> >>>>>>>>>> close/unexpected errors). Any design that this KIP produces can
> >>>>>>>>> ultimately
> >>>>>>>>>> be changed in the future, especially if the bulk of it is
> internal
> >>>>>>>>>> behaviour.
> >>>>>>>>>>
> >>>>>>>>>> I'm going to spend some time next week trying to re-work the
> >>>>>> original
> >>>>>>>>>> WriteBatchWithIndex design to remove the newTransaction()
> method,
> >>>>>> such
> >>>>>>>>> that
> >>>>>>>>>> it's just an implementation detail of RocksDBStore. That way, if
> >> we
> >>>>>>>>> want to
> >>>>>>>>>> replace WBWI with something in the future, like the SST file
> >>>>>> management
> >>>>>>>>>> outlined by John, then we can do so with little/no API changes.
> >>>>>>>>>>
> >>>>>>>>>> Regards,
> >>>>>>>>>>
> >>>>>>>>>> Nick
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>
> >>
> >
>

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

Posted by Bruno Cadonna <ca...@apache.org>.
Hi Nick,

Thanks for the updates and sorry for the delay on my side!


1.
Making the default implementation for flush() a no-op sounds good to me.


2.
I think what was bugging me here is that a third-party state store needs 
to implement the state store interface. That means they need to 
implement a wrapper around the actual state store as we do for RocksDB 
with RocksDBStore. So, a third-party state store can always estimate the 
uncommitted bytes, if it wants, because the wrapper can record the added 
bytes.
One case I can think of where returning -1 makes sense is when Streams 
does not need to estimate the size of the write batch and trigger 
extraordinary commits, because the third-party state store takes care of 
memory. But in that case the method could also just return 0. Even that 
case would be better solved with a method that returns whether the state 
store manages itself the memory used for uncommitted bytes or not.
Said that, I am fine with keeping the -1 return value, I was just 
wondering when and if it will be used.

Regarding returning 0 for transactional state stores when the batch is 
empty, I was just wondering because you explicitly stated

"or {@code 0} if this StateStore does not support transactions."

So it seemed to me returning 0 could only happen for non-transactional 
state stores.


3.

a) What do you think if we move the isolation level to IQ (v1 and v2)?
In the end this is the only component that really needs to specify the 
isolation level. It is similar to the Kafka consumer that can choose 
with what isolation level to read the input topic.
For IQv1 the isolation level should go into StoreQueryParameters. For 
IQv2, I would add it to the Query interface.

b) Point a) raises the question what should happen during at-least-once 
processing when the state store does not use transactions? John in the 
past proposed to also use transactions on state stores for 
at-least-once. I like that idea, because it avoids aggregating the same 
records over and over again in the case of a failure. We had a case in 
the past where a Streams applications in at-least-once mode was failing 
continuously for some reasons I do not remember before committing the 
offsets. After each failover, the app aggregated again and again the 
same records. Of course the aggregate increased to very wrong values 
just because of the failover. With transactions on the state stores we 
could have avoided this. The app would have output the same aggregate 
multiple times (i.e., after each failover) but at least the value of the 
aggregate would not depend on the number of failovers. Outputting the 
same aggregate multiple times would be incorrect under exactly-once but 
it is OK for at-least-once.
If it makes sense to add a config to turn on and off transactions on 
state stores under at-least-once or just use transactions in any case is 
a question we should also discuss in this KIP. It depends a bit on the 
performance trade-off. Maybe to be safe, I would add a config.


4.
Your points are all valid. I tend to say to keep the metrics around 
flush() until we remove flush() completely from the interface. Calls to 
flush() might still exist since existing processors might still call 
flush() explicitly as you mentioned in 1). For sure, we need to document 
how the metrics change due to the transactions in the upgrade notes.


5.
I see. Then you should describe how the .position files are handled  in 
a dedicated section of the KIP or incorporate the description in the 
"Atomic Checkpointing" section instead of only mentioning it in the 
"Compatibility, 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:

https://github.com/apache/kafka/blob/63fee01366e6ce98b9dfafd279a45d40b80e282d/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java#L75

It is a bit brittle, but it works.


Best,
Bruno


On 8/24/23 12:18 PM, Nick Telford wrote:
> Hi Bruno,
> 
> Thanks for taking the time to review the KIP. I'm back from leave now and
> intend to move this forwards as quickly as I can.
> 
> Addressing your points:
> 
> 1.
> Because flush() is part of the StateStore API, it's exposed to custom
> Processors, which might be making calls to flush(). This was actually the
> case in a few integration tests.
> To maintain as much compatibility as possible, I'd prefer not to make this
> an UnsupportedOperationException, as it will cause previously working
> Processors to start throwing exceptions at runtime.
> I agree that it doesn't make sense for it to proxy commit(), though, as
> that would cause it to violate the "StateStores commit only when the Task
> commits" rule.
> Instead, I think we should make this a no-op. That way, existing user
> Processors will continue to work as-before, without violation of store
> consistency that would be caused by premature flush/commit of StateStore
> data to disk.
> What do you think?
> 
> 2.
> As stated in the JavaDoc, when a StateStore implementation is
> transactional, but is unable to estimate the uncommitted memory usage, the
> method will return -1.
> The intention here is to permit third-party implementations that may not be
> able to estimate memory usage.
> 
> Yes, it will be 0 when nothing has been written to the store yet. I thought
> that was implied by "This method will return an approximation of the memory
> would be freed by the next call to {@link #commit(Map)}" and "@return The
> approximate size of all records awaiting {@link #commit(Map)}", however, I
> can add it explicitly to the JavaDoc if you think this is unclear?
> 
> 3.
> I realise this is probably the most contentious point in my design, and I'm
> open to changing it if I'm unable to convince you of the benefits.
> Nevertheless, here's my argument:
> The Interactive Query (IQ) API(s) are directly provided StateStores to
> query, and it may be important for users to programmatically know which
> mode the StateStore is operating under. If we simply provide an
> "eosEnabled" boolean (as used throughout the internal streams engine), or
> similar, then users will need to understand the operation and consequences
> of each available processing mode and how it pertains to their StateStore.
> 
> Interactive Query users aren't the only people that care about the
> processing.mode/IsolationLevel of a StateStore: implementers of custom
> StateStores also need to understand the behaviour expected of their
> implementation. KIP-892 introduces some assumptions into the Streams Engine
> about how StateStores operate under each processing mode, and it's
> important that custom implementations adhere to those assumptions in order
> to maintain the consistency guarantees.
> 
> IsolationLevels provide a high-level contract on the behaviour of the
> StateStore: a user knows that under READ_COMMITTED, they will see writes
> only after the Task has committed, and under READ_UNCOMMITTED they will see
> writes immediately. No understanding of the details of each processing.mode
> is required, either for IQ users or StateStore implementers.
> 
> An argument can be made that these contractual guarantees can simply be
> documented for the processing.mode (i.e. that exactly-once and
> exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves like
> READ_UNCOMMITTED), but there are several small issues with this I'd prefer
> to avoid:
> 
>     - Where would we document these contracts, in a way that is difficult
>     for users/implementers to miss/ignore?
>     - It's not clear to users that the processing mode is communicating
>     an expectation of read isolation, unless they read the documentation. Users
>     rarely consult documentation unless they feel they need to, so it's likely
>     this detail would get missed by many users.
>     - It tightly couples processing modes to read isolation. Adding new
>     processing modes, or changing the read isolation of existing processing
>     modes would be difficult/impossible.
> 
> Ultimately, the cost of introducing IsolationLevels is just a single
> method, since we re-use the existing IsolationLevel enum from Kafka. This
> gives us a clear place to document the contractual guarantees expected
> of/provided by StateStores, that is accessible both by the StateStore
> itself, and by IQ users.
> 
> (Writing this I've just realised that the StateStore and IQ APIs actually
> don't provide access to StateStoreContext that IQ users would have direct
> access to... Perhaps StateStore should expose isolationLevel() itself too?)
> 
> 4.
> Yeah, I'm not comfortable renaming the metrics in-place either, as it's a
> backwards incompatible change. My concern is that, if we leave the existing
> "flush" metrics in place, they will be confusing to users. Right now,
> "flush" metrics record explicit flushes to disk, but under KIP-892, even a
> commit() will not explicitly flush data to disk - RocksDB will decide on
> when to flush memtables to disk itself.
> 
> If we keep the existing "flush" metrics, we'd have two options, which both
> seem pretty bad to me:
> 
>     1. Have them record calls to commit(), which would be misleading, as
>     data is no longer explicitly "flushed" to disk by this call.
>     2. Have them record nothing at all, which is equivalent to removing the
>     metrics, except that users will see the metric still exists and so assume
>     that the metric is correct, and that there's a problem with their system
>     when there isn't.
> 
> I agree that removing them is also a bad solution, and I'd like some
> guidance on the best path forward here.
> 
> 5.
> Position files are updated on every write to a StateStore. Since our writes
> are now buffered until commit(), we can't update the Position file until
> commit() has been called, otherwise it would be inconsistent with the data
> in the event of a rollback. Consequently, we need to manage these offsets
> the same way we manage the checkpoint offsets, and ensure they're only
> written on commit().
> 
> 6.
> Agreed, although I'm not exactly sure yet what tests to write. How explicit
> do we need to be here in the KIP?
> 
> As for upgrade/downgrade: upgrade is designed to be seamless, and we should
> definitely add some tests around that. Downgrade, it transpires, isn't
> currently possible, as the extra column family for offset storage is
> incompatible with the pre-KIP-892 implementation: when you open a RocksDB
> database, you must open all available column families or receive an error.
> What currently happens on downgrade is that it attempts to open the store,
> throws an error about the offsets column family not being opened, which
> triggers a wipe and rebuild of the Task. Given that downgrades should be
> uncommon, I think this is acceptable behaviour, as the end-state is
> consistent, even if it results in an undesirable state restore.
> 
> Should I document the upgrade/downgrade behaviour explicitly in the KIP?
> 
> --
> 
> Regards,
> Nick
> 
> 
> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org> wrote:
> 
>> Hi Nick!
>>
>> Thanks for the updates!
>>
>> 1.
>> Why does StateStore#flush() default to
>> StateStore#commit(Collections.emptyMap())?
>> Since calls to flush() will not exist anymore after this KIP is
>> released, I would rather throw an unsupported operation exception by
>> default.
>>
>>
>> 2.
>> When would a state store return -1 from
>> StateStore#approximateNumUncommittedBytes() while being transactional?
>>
>> Wouldn't StateStore#approximateNumUncommittedBytes() also return 0 if
>> the state store is transactional but nothing has been written to the
>> state store yet?
>>
>>
>> 3.
>> Sorry for bringing this up again. Does this KIP really need to introduce
>> StateStoreContext#isolationLevel()? StateStoreContext has already
>> appConfigs() which basically exposes the same information, i.e., if EOS
>> is enabled or not.
>> In one of your previous e-mails you wrote:
>>
>> "My idea was to try to keep the StateStore interface as loosely coupled
>> from the Streams engine as possible, to give implementers more freedom,
>> and reduce the amount of internal knowledge required."
>>
>> While I understand the intent, I doubt that it decreases the coupling of
>> a StateStore interface and the Streams engine. READ_COMMITTED only
>> applies to IQ but not to reads by processors. Thus, implementers need to
>> understand how Streams accesses the state stores.
>>
>> I would like to hear what others think about this.
>>
>>
>> 4.
>> Great exposing new metrics for transactional state stores! However, I
>> would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides unit
>> tests. Additionally, we probably need integration or system tests to
>> verify that upgrades and downgrades between transactional and
>> non-transactional state stores work as expected.
>>
>>
>> Best,
>> Bruno
>>
>>
>>
>>
>>
>> On 7/21/23 10:34 PM, Nick Telford wrote:
>>> One more thing: I noted John's suggestion in the KIP, under "Rejected
>>> Alternatives". I still think it's an idea worth pursuing, but I believe
>>> that it's out of the scope of this KIP, because it solves a different set
>>> of problems to this KIP, and the scope of this one has already grown
>> quite
>>> large!
>>>
>>> On Fri, 21 Jul 2023 at 21:33, Nick Telford <ni...@gmail.com>
>> wrote:
>>>
>>>> Hi everyone,
>>>>
>>>> I've updated the KIP (
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> )
>>>> with the latest changes; mostly bringing back "Atomic Checkpointing"
>> (for
>>>> what feels like the 10th time!). I think the one thing missing is some
>>>> changes to metrics (notably the store "flush" metrics will need to be
>>>> renamed to "commit").
>>>>
>>>> The reason I brought back Atomic Checkpointing was to decouple store
>> flush
>>>> from store commit. This is important, because with Transactional
>>>> StateStores, we now need to call "flush" on *every* Task commit, and not
>>>> just when the StateStore is closing, otherwise our transaction buffer
>> will
>>>> never be written and persisted, instead growing unbounded! I
>> experimented
>>>> with some simple solutions, like forcing a store flush whenever the
>>>> transaction buffer was likely to exceed its configured size, but this
>> was
>>>> brittle: it prevented the transaction buffer from being configured to be
>>>> unbounded, and it still would have required explicit flushes of RocksDB,
>>>> yielding sub-optimal performance and memory utilization.
>>>>
>>>> I deemed Atomic Checkpointing to be the "right" way to resolve this
>>>> problem. By ensuring that the changelog offsets that correspond to the
>> most
>>>> recently written records are always atomically written to the StateStore
>>>> (by writing them to the same transaction buffer), we can avoid forcibly
>>>> flushing the RocksDB memtables to disk, letting RocksDB flush them only
>>>> when necessary, without losing any of our consistency guarantees. See
>> the
>>>> updated KIP for more info.
>>>>
>>>> I have fully implemented these changes, although I'm still not entirely
>>>> happy with the implementation for segmented StateStores, so I plan to
>>>> refactor that. Despite that, all tests pass. If you'd like to try out or
>>>> review this highly experimental and incomplete branch, it's available
>> here:
>>>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note: it's
>> built
>>>> against Kafka 3.5.0 so that I had a stable base to build and test it on,
>>>> and to enable easy apples-to-apples comparisons in a live environment. I
>>>> plan to rebase it against trunk once it's nearer completion and has been
>>>> proven on our main application.
>>>>
>>>> I would really appreciate help in reviewing and testing:
>>>> - Segmented (Versioned, Session and Window) stores
>>>> - Global stores
>>>>
>>>> As I do not currently use either of these, so my primary test
>> environment
>>>> doesn't test these areas.
>>>>
>>>> I'm going on Parental Leave starting next week for a few weeks, so will
>>>> not have time to move this forward until late August. That said, your
>>>> feedback is welcome and appreciated, I just won't be able to respond as
>>>> quickly as usual.
>>>>
>>>> Regards,
>>>> Nick
>>>>
>>>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com>
>> wrote:
>>>>
>>>>> Hi Bruno
>>>>>
>>>>> Yes, that's correct, although the impact on IQ is not something I had
>>>>> considered.
>>>>>
>>>>> What about atomically updating the state store from the transaction
>>>>>> buffer every commit interval and writing the checkpoint (thus,
>> flushing
>>>>>> the memtable) every configured amount of data and/or number of commit
>>>>>> intervals?
>>>>>>
>>>>>
>>>>> I'm not quite sure I follow. Are you suggesting that we add an
>> additional
>>>>> config for the max number of commit intervals between checkpoints? That
>>>>> way, we would checkpoint *either* when the transaction buffers are
>> nearly
>>>>> full, *OR* whenever a certain number of commit intervals have elapsed,
>>>>> whichever comes first?
>>>>>
>>>>> That certainly seems reasonable, although this re-ignites an earlier
>>>>> debate about whether a config should be measured in "number of commit
>>>>> intervals", instead of just an absolute time.
>>>>>
>>>>> FWIW, I realised that this issue is the reason I was pursuing the
>> Atomic
>>>>> Checkpoints, as it de-couples memtable flush from checkpointing, which
>>>>> enables us to just checkpoint on every commit without any performance
>>>>> impact. Atomic Checkpointing is definitely the "best" solution, but
>> I'm not
>>>>> sure if this is enough to bring it back into this KIP.
>>>>>
>>>>> I'm currently working on moving all the transactional logic directly
>> into
>>>>> RocksDBStore itself, which does away with the StateStore#newTransaction
>>>>> method, and reduces the number of new classes introduced, significantly
>>>>> reducing the complexity. If it works, and the complexity is drastically
>>>>> reduced, I may try bringing back Atomic Checkpoints into this KIP.
>>>>>
>>>>> Regards,
>>>>> Nick
>>>>>
>>>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org> wrote:
>>>>>
>>>>>> Hi Nick,
>>>>>>
>>>>>> Thanks for the insights! Very interesting!
>>>>>>
>>>>>> As far as I understand, you want to atomically update the state store
>>>>>> from the transaction buffer, flush the memtable of a state store and
>>>>>> write the checkpoint not after the commit time elapsed but after the
>>>>>> transaction buffer reached a size that would lead to exceeding
>>>>>> statestore.transaction.buffer.max.bytes before the next commit
>> interval
>>>>>> ends.
>>>>>> That means, the Kafka transaction would commit every commit interval
>> but
>>>>>> the state store will only be atomically updated roughly every
>>>>>> statestore.transaction.buffer.max.bytes of data. Also IQ would then
>> only
>>>>>> see new data roughly every statestore.transaction.buffer.max.bytes.
>>>>>> After a failure the state store needs to restore up to
>>>>>> statestore.transaction.buffer.max.bytes.
>>>>>>
>>>>>> Is this correct?
>>>>>>
>>>>>> What about atomically updating the state store from the transaction
>>>>>> buffer every commit interval and writing the checkpoint (thus,
>> flushing
>>>>>> the memtable) every configured amount of data and/or number of commit
>>>>>> intervals? In such a way, we would have the same delay for records
>>>>>> appearing in output topics and IQ because both would appear when the
>>>>>> Kafka transaction is committed. However, after a failure the state
>> store
>>>>>> still needs to restore up to statestore.transaction.buffer.max.bytes
>> and
>>>>>> it might restore data that is already in the state store because the
>>>>>> checkpoint lags behind the last stable offset (i.e. the last committed
>>>>>> offset) of the changelog topics. Restoring data that is already in the
>>>>>> state store is idempotent, so eos should not violated.
>>>>>> This solution needs at least one new config to specify when a
>> checkpoint
>>>>>> should be written.
>>>>>>
>>>>>>
>>>>>>
>>>>>> A small correction to your previous e-mail that does not change
>> anything
>>>>>> you said: Under alos the default commit interval is 30 seconds, not
>> five
>>>>>> seconds.
>>>>>>
>>>>>>
>>>>>> Best,
>>>>>> Bruno
>>>>>>
>>>>>>
>>>>>> On 01.07.23 12:37, Nick Telford wrote:
>>>>>>> Hi everyone,
>>>>>>>
>>>>>>> I've begun performance testing my branch on our staging environment,
>>>>>>> putting it through its paces in our non-trivial application. I'm
>>>>>> already
>>>>>>> observing the same increased flush rate that we saw the last time we
>>>>>>> attempted to use a version of this KIP, but this time, I think I know
>>>>>> why.
>>>>>>>
>>>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end of the
>>>>>> Task
>>>>>>> commit process, has the following behaviour:
>>>>>>>
>>>>>>>       - Under ALOS: checkpoint the state stores. This includes
>>>>>>>       flushing memtables in RocksDB. This is acceptable because the
>>>>>> default
>>>>>>>       commit.interval.ms is 5 seconds, so forcibly flushing memtables
>>>>>> every 5
>>>>>>>       seconds is acceptable for most applications.
>>>>>>>       - Under EOS: checkpointing is not done, *unless* it's being
>>>>>> forced, due
>>>>>>>       to e.g. the Task closing or being revoked. This means that under
>>>>>> normal
>>>>>>>       processing conditions, the state stores will not be
>> checkpointed,
>>>>>> and will
>>>>>>>       not have memtables flushed at all , unless RocksDB decides to
>>>>>> flush them on
>>>>>>>       its own. Checkpointing stores and force-flushing their memtables
>>>>>> is only
>>>>>>>       done when a Task is being closed.
>>>>>>>
>>>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least *some*
>> normal
>>>>>>> Task commits, in order to write the RocksDB transaction buffers to
>> the
>>>>>>> state stores, and to ensure the offsets are synced to disk to prevent
>>>>>>> restores from getting out of hand. Consequently, my current
>>>>>> implementation
>>>>>>> calls maybeCheckpoint on *every* Task commit, which is far too
>>>>>> frequent.
>>>>>>> This causes checkpoints every 10,000 records, which is a change in
>>>>>> flush
>>>>>>> behaviour, potentially causing performance problems for some
>>>>>> applications.
>>>>>>>
>>>>>>> I'm looking into possible solutions, and I'm currently leaning
>> towards
>>>>>>> using the statestore.transaction.buffer.max.bytes configuration to
>>>>>>> checkpoint Tasks once we are likely to exceed it. This would
>>>>>> complement the
>>>>>>> existing "early Task commit" functionality that this configuration
>>>>>>> provides, in the following way:
>>>>>>>
>>>>>>>       - Currently, we use statestore.transaction.buffer.max.bytes to
>>>>>> force an
>>>>>>>       early Task commit if processing more records would cause our
>> state
>>>>>> store
>>>>>>>       transactions to exceed the memory assigned to them.
>>>>>>>       - New functionality: when a Task *does* commit, we will not
>>>>>> checkpoint
>>>>>>>       the stores (and hence flush the transaction buffers) unless we
>>>>>> expect to
>>>>>>>       cross the statestore.transaction.buffer.max.bytes threshold
>> before
>>>>>> the next
>>>>>>>       commit
>>>>>>>
>>>>>>> I'm also open to suggestions.
>>>>>>>
>>>>>>> Regards,
>>>>>>> Nick
>>>>>>>
>>>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <ni...@gmail.com>
>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Bruno!
>>>>>>>>
>>>>>>>> 3.
>>>>>>>> By "less predictable for users", I meant in terms of understanding
>> the
>>>>>>>> performance profile under various circumstances. The more complex
>> the
>>>>>>>> solution, the more difficult it would be for users to understand the
>>>>>>>> performance they see. For example, spilling records to disk when the
>>>>>>>> transaction buffer reaches a threshold would, I expect, reduce write
>>>>>>>> throughput. This reduction in write throughput could be unexpected,
>>>>>> and
>>>>>>>> potentially difficult to diagnose/understand for users.
>>>>>>>> At the moment, I think the "early commit" concept is relatively
>>>>>>>> straightforward; it's easy to document, and conceptually fairly
>>>>>> obvious to
>>>>>>>> users. We could probably add a metric to make it easier to
>> understand
>>>>>> when
>>>>>>>> it happens though.
>>>>>>>>
>>>>>>>> 3. (the second one)
>>>>>>>> The IsolationLevel is *essentially* an indirect way of telling
>>>>>> StateStores
>>>>>>>> whether they should be transactional. READ_COMMITTED essentially
>>>>>> requires
>>>>>>>> transactions, because it dictates that two threads calling
>>>>>>>> `newTransaction()` should not see writes from the other transaction
>>>>>> until
>>>>>>>> they have been committed. With READ_UNCOMMITTED, all bets are off,
>> and
>>>>>>>> stores can allow threads to observe written records at any time,
>>>>>> which is
>>>>>>>> essentially "no transactions". That said, StateStores are free to
>>>>>> implement
>>>>>>>> these guarantees however they can, which is a bit more relaxed than
>>>>>>>> dictating "you must use transactions". For example, with RocksDB we
>>>>>> would
>>>>>>>> implement these as READ_COMMITTED == WBWI-based "transactions",
>>>>>>>> READ_UNCOMMITTED == direct writes to the database. But with other
>>>>>> storage
>>>>>>>> engines, it might be preferable to *always* use transactions, even
>>>>>> when
>>>>>>>> unnecessary; or there may be storage engines that don't provide
>>>>>>>> transactions, but the isolation guarantees can be met using a
>>>>>> different
>>>>>>>> technique.
>>>>>>>> My idea was to try to keep the StateStore interface as loosely
>> coupled
>>>>>>>> from the Streams engine as possible, to give implementers more
>>>>>> freedom, and
>>>>>>>> reduce the amount of internal knowledge required.
>>>>>>>> That said, I understand that "IsolationLevel" might not be the right
>>>>>>>> abstraction, and we can always make it much more explicit if
>>>>>> required, e.g.
>>>>>>>> boolean transactional()
>>>>>>>>
>>>>>>>> 7-8.
>>>>>>>> I can make these changes either later today or tomorrow.
>>>>>>>>
>>>>>>>> Small update:
>>>>>>>> I've rebased my branch on trunk and fixed a bunch of issues that
>>>>>> needed
>>>>>>>> addressing. Currently, all the tests pass, which is promising, but
>> it
>>>>>> will
>>>>>>>> need to undergo some performance testing. I haven't (yet) worked on
>>>>>>>> removing the `newTransaction()` stuff, but I would expect that,
>>>>>>>> behaviourally, it should make no difference. The branch is available
>>>>>> at
>>>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
>>>>>>>> interested in taking an early look.
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>> Nick
>>>>>>>>
>>>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org>
>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Nick,
>>>>>>>>>
>>>>>>>>> 1.
>>>>>>>>> Yeah, I agree with you. That was actually also my point. I
>> understood
>>>>>>>>> that John was proposing the ingestion path as a way to avoid the
>>>>>> early
>>>>>>>>> commits. Probably, I misinterpreted the intent.
>>>>>>>>>
>>>>>>>>> 2.
>>>>>>>>> I agree with John here, that actually it is public API. My question
>>>>>> is
>>>>>>>>> how this usage pattern affects normal processing.
>>>>>>>>>
>>>>>>>>> 3.
>>>>>>>>> My concern is that checking for the size of the transaction buffer
>>>>>> and
>>>>>>>>> maybe triggering an early commit affects the whole processing of
>>>>>> Kafka
>>>>>>>>> Streams. The transactionality of a state store is not confined to
>> the
>>>>>>>>> state store itself, but spills over and changes the behavior of
>> other
>>>>>>>>> parts of the system. I agree with you that it is a decent
>>>>>> compromise. I
>>>>>>>>> just wanted to analyse the downsides and list the options to
>> overcome
>>>>>>>>> them. I also agree with you that all options seem quite heavy
>>>>>> compared
>>>>>>>>> with your KIP. I do not understand what you mean with "less
>>>>>> predictable
>>>>>>>>> for users", though.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> I found the discussions about the alternatives really interesting.
>>>>>> But I
>>>>>>>>> also think that your plan sounds good and we should continue with
>> it!
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Some comments on your reply to my e-mail on June 20th:
>>>>>>>>>
>>>>>>>>> 3.
>>>>>>>>> Ah, now, I understand the reasoning behind putting isolation level
>> in
>>>>>>>>> the state store context. Thanks! Should that also be a way to give
>>>>>> the
>>>>>>>>> the state store the opportunity to decide whether to turn on
>>>>>>>>> transactions or not?
>>>>>>>>> With my comment, I was more concerned about how do you know if a
>>>>>>>>> checkpoint file needs to be written under EOS, if you do not have a
>>>>>> way
>>>>>>>>> to know if the state store is transactional or not. If a state
>> store
>>>>>> is
>>>>>>>>> transactional, the checkpoint file can be written during normal
>>>>>>>>> processing under EOS. If the state store is not transactional, the
>>>>>>>>> checkpoint file must not be written under EOS.
>>>>>>>>>
>>>>>>>>> 7.
>>>>>>>>> My point was about not only considering the bytes in memory in
>> config
>>>>>>>>> statestore.uncommitted.max.bytes, but also bytes that might be
>>>>>> spilled
>>>>>>>>> on disk. Basically, I was wondering whether you should remove the
>>>>>>>>> "memory" in "Maximum number of memory bytes to be used to
>>>>>>>>> buffer uncommitted state-store records." My thinking was that even
>>>>>> if a
>>>>>>>>> state store spills uncommitted bytes to disk, limiting the overall
>>>>>> bytes
>>>>>>>>> might make sense. Thinking about it again and considering the
>> recent
>>>>>>>>> discussions, it does not make too much sense anymore.
>>>>>>>>> I like the name statestore.transaction.buffer.max.bytes that you
>>>>>> proposed.
>>>>>>>>>
>>>>>>>>> 8.
>>>>>>>>> A high-level description (without implementation details) of how
>>>>>> Kafka
>>>>>>>>> Streams will manage the commit of changelog transactions, state
>> store
>>>>>>>>> transactions and checkpointing would be great. Would be great if
>> you
>>>>>>>>> could also add some sentences about the behavior in case of a
>>>>>> failure.
>>>>>>>>> For instance how does a transactional state store recover after a
>>>>>>>>> failure or what happens with the transaction buffer, etc. (that is
>>>>>> what
>>>>>>>>> I meant by "fail-over" in point 9.)
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Bruno
>>>>>>>>>
>>>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
>>>>>>>>>> Hi Bruno,
>>>>>>>>>>
>>>>>>>>>> 1.
>>>>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
>>>>>> transactions
>>>>>>>>>> have, whereby exceeding (or likely to exceed) configured memory
>>>>>> needs to
>>>>>>>>>> trigger an early commit?
>>>>>>>>>>
>>>>>>>>>> 2.
>>>>>>>>>> This is one of my big concerns. Ultimately, any approach based on
>>>>>>>>> cracking
>>>>>>>>>> open RocksDB internals and using it in ways it's not really
>> designed
>>>>>>>>> for is
>>>>>>>>>> likely to have some unforseen performance or consistency issues.
>>>>>>>>>>
>>>>>>>>>> 3.
>>>>>>>>>> What's your motivation for removing these early commits? While not
>>>>>>>>> ideal, I
>>>>>>>>>> think they're a decent compromise to ensure consistency whilst
>>>>>>>>> maintaining
>>>>>>>>>> good and predictable performance.
>>>>>>>>>> All 3 of your suggested ideas seem *very* complicated, and might
>>>>>>>>> actually
>>>>>>>>>> make behaviour less predictable for users as a consequence.
>>>>>>>>>>
>>>>>>>>>> I'm a bit concerned that the scope of this KIP is growing a bit
>> out
>>>>>> of
>>>>>>>>>> control. While it's good to discuss ideas for future
>> improvements, I
>>>>>>>>> think
>>>>>>>>>> it's important to narrow the scope down to a design that achieves
>>>>>> the
>>>>>>>>> most
>>>>>>>>>> pressing objectives (constant sized restorations during dirty
>>>>>>>>>> close/unexpected errors). Any design that this KIP produces can
>>>>>>>>> ultimately
>>>>>>>>>> be changed in the future, especially if the bulk of it is internal
>>>>>>>>>> behaviour.
>>>>>>>>>>
>>>>>>>>>> I'm going to spend some time next week trying to re-work the
>>>>>> original
>>>>>>>>>> WriteBatchWithIndex design to remove the newTransaction() method,
>>>>>> such
>>>>>>>>> that
>>>>>>>>>> it's just an implementation detail of RocksDBStore. That way, if
>> we
>>>>>>>>> want to
>>>>>>>>>> replace WBWI with something in the future, like the SST file
>>>>>> management
>>>>>>>>>> outlined by John, then we can do so with little/no API changes.
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>>
>>>>>>>>>> Nick
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>
>>
> 

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

Posted by Colt McNealy <co...@littlehorse.io>.
Nick,

Thanks for the response.

>Can you clarify how much state was restored in those 11 seconds?
That was a full restoration of ~650MB of state after I wiped the state
directory. The restoration after a crash with your branch is nearly
instantaneous, whereas with plain Kafka 3.5.0 a crash triggers a full
restoration (8 seconds).

Additionally, I pulled, rebuilt, and re-tested your changes and now the
restoration time with your branch is the same as with vanilla Streams!
Fantastic work!

I plan to do some more testing with larger state stores over the next
couple weeks, both with RocksDB and Speedb OSS. And perhaps I might even
try enabling some of the experimental Speedb OSS features, such as the
[Improved Write Flow](https://docs.speedb.io/speedb-features/write-flow).
As far as I understand, this isn't possible to do through the standard
RocksDBConfigSetter since some of the config options are Speedb-specific.

Cheers,
Colt McNealy

*Founder, LittleHorse.dev*


On Mon, Sep 11, 2023 at 4:29 AM Nick Telford <ni...@gmail.com> wrote:

> Hi Colt,
>
> Thanks for taking the time to run your benchmarks on this, that's
> incredibly helpful.
>
> > With KIP 892, I verified that unclean shutdown does not cause a fresh
> > restore (!!!!). I got the following benchmark results:
> > - Benchmark took 216 seconds
> > - 1,401 tasks per second on one partition
> > - 11 seconds to restore the state
>
> Can you clarify how much state was restored in those 11 seconds? Was this
> the time to do the full restore regardless, or was it the time to only
> restore a small fraction of the state (e.g. the last aborted transaction)?
>
> > -- QUESTION: Because we observed a significant (30% or so) and
> reproducible
> > slowdown during restoration, it seems like KIP-892 uses the checkpointing
> > behavior during restoration as well? If so, I would argue that this might
> > not be necessary, because everything we write is already committed, so we
> > don't need to change the behavior during restoration or standby tasks.
> > Perhaps we could write the offsets to RocksDB on every batch (or even
> every
> > 5 seconds or so).
>
> Restore has always used a completely separate code-path to regular writes,
> and continues to do so. I had a quick pass over the code and I suspect I
> know what's causing the performance degradation: for every restored record,
> I was adding the changelog offset of that record to the batch along with
> the record. This is different to the regular write-path, which only adds
> the current offsets once, on-commit. This writeOffset method is fairly
> expensive, since it has to serialize the TopicPartition and offset that's
> being written to the database.
>
> Assuming this is the cause, I've already pushed a fix to my branch that
> will only call writeOffset once per-batch, and also adds some caching to
> the serialization in writeOffset, that should also enhance performance of
> state commit in the normal write-path.
>
> Please let me know if this addresses the issue!
>
> Regards,
> Nick
>
>
> On Mon, 11 Sept 2023 at 05:38, Colt McNealy <co...@littlehorse.io> wrote:
>
> > Howdy folks,
> >
> > First I wanted to say fantastic work and thank you to Nick. I built your
> > branch (https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0) and did
> > some testing on our Streams app with Kafka 3.5.0, your `kip-892-3.5.0`
> > branch, and your `kip-892-3.5.0` branch built with Speedb OSS 2.3.0.1.
> And
> > it worked! Including the global store (we don't have any segmented
> stores,
> > unfortunately).
> >
> > The test I ran involved running 3,000 workflows with 100 tasks each, and
> > roughly 650MB state total.
> >
> > With Streams 3.5.0, I indeed verified that unclean shutdown caused a
> fresh
> > restore from scratch. I also benchmarked my application at:
> > - Running the benchmark took 211 seconds
> > - 1,421 tasks per second on one partition
> > - 8 seconds to restore the state (650MB or so)
> >
> > With KIP 892, I verified that unclean shutdown does not cause a fresh
> > restore (!!!!). I got the following benchmark results:
> > - Benchmark took 216 seconds
> > - 1,401 tasks per second on one partition
> > - 11 seconds to restore the state
> >
> > I ran the restorations many times to ensure that there was no rounding
> > error or noise; the results were remarkably consistent. Additionally, I
> ran
> > the restorations with KIP-892 built with Speedb OSS. The restoration time
> > consistently came out as 10 seconds, which was an improvement from the 11
> > seconds observed with RocksDB + KIP-892.
> >
> > My application is bottlenecked mostly by serialization and
> deserialization,
> > so improving the performance of the state store doesn't really impact our
> > throughput that much. And the processing performance (benchmark time,
> > tasks/second) are pretty close in KIP-892 vs Streams 3.5.0. However, at
> > larger state store sizes, RocksDB performance begins to degrade, so that
> > might not be true once we pass 20GB per partition.
> >
> > -- QUESTION: Because we observed a significant (30% or so) and
> reproducible
> > slowdown during restoration, it seems like KIP-892 uses the checkpointing
> > behavior during restoration as well? If so, I would argue that this might
> > not be necessary, because everything we write is already committed, so we
> > don't need to change the behavior during restoration or standby tasks.
> > Perhaps we could write the offsets to RocksDB on every batch (or even
> every
> > 5 seconds or so).
> >
> > -- Note: This was a very small-scale test, with <1GB of state (as I
> didn't
> > have time to spend hours building up state). In the past I have noted
> that
> > RocksDB performance degrades significantly after 25GB of state in one
> > store. Future work involves determining the performance impact of KIP-892
> > relative to trunk at larger scale, since it's possible that the relative
> > behaviors are far different (i.e. relative to trunk, 892's processing and
> > restoration throughput might be much better or much worse).
> >
> > -- Note: For those who want to replicate the tests, you can find the
> branch
> > of our streams app here:
> >
> >
> https://github.com/littlehorse-enterprises/littlehorse/tree/minor/testing-streams-forks
> > . The example I ran was `examples/hundred-tasks`, and I ran the server
> with
> > `./local-dev/do-server.sh one-partition`. The `STREAMS_TESTS.md` file
> 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.
> >
> > Thanks Nick!
> >
> > Ride well,
> > Colt McNealy
> >
> > *Founder, LittleHorse.dev*
> >
> >
> > On Thu, Aug 24, 2023 at 3:19 AM Nick Telford <ni...@gmail.com>
> > wrote:
> >
> > > Hi Bruno,
> > >
> > > Thanks for taking the time to review the KIP. I'm back from leave now
> and
> > > intend to move this forwards as quickly as I can.
> > >
> > > Addressing your points:
> > >
> > > 1.
> > > Because flush() is part of the StateStore API, it's exposed to custom
> > > Processors, which might be making calls to flush(). This was actually
> the
> > > case in a few integration tests.
> > > To maintain as much compatibility as possible, I'd prefer not to make
> > this
> > > an UnsupportedOperationException, as it will cause previously working
> > > Processors to start throwing exceptions at runtime.
> > > I agree that it doesn't make sense for it to proxy commit(), though, as
> > > that would cause it to violate the "StateStores commit only when the
> Task
> > > commits" rule.
> > > Instead, I think we should make this a no-op. That way, existing user
> > > Processors will continue to work as-before, without violation of store
> > > consistency that would be caused by premature flush/commit of
> StateStore
> > > data to disk.
> > > What do you think?
> > >
> > > 2.
> > > As stated in the JavaDoc, when a StateStore implementation is
> > > transactional, but is unable to estimate the uncommitted memory usage,
> > the
> > > method will return -1.
> > > The intention here is to permit third-party implementations that may
> not
> > be
> > > able to estimate memory usage.
> > >
> > > Yes, it will be 0 when nothing has been written to the store yet. I
> > thought
> > > that was implied by "This method will return an approximation of the
> > memory
> > > would be freed by the next call to {@link #commit(Map)}" and "@return
> The
> > > approximate size of all records awaiting {@link #commit(Map)}",
> however,
> > I
> > > can add it explicitly to the JavaDoc if you think this is unclear?
> > >
> > > 3.
> > > I realise this is probably the most contentious point in my design, and
> > I'm
> > > open to changing it if I'm unable to convince you of the benefits.
> > > Nevertheless, here's my argument:
> > > The Interactive Query (IQ) API(s) are directly provided StateStores to
> > > query, and it may be important for users to programmatically know which
> > > mode the StateStore is operating under. If we simply provide an
> > > "eosEnabled" boolean (as used throughout the internal streams engine),
> or
> > > similar, then users will need to understand the operation and
> > consequences
> > > of each available processing mode and how it pertains to their
> > StateStore.
> > >
> > > Interactive Query users aren't the only people that care about the
> > > processing.mode/IsolationLevel of a StateStore: implementers of custom
> > > StateStores also need to understand the behaviour expected of their
> > > implementation. KIP-892 introduces some assumptions into the Streams
> > Engine
> > > about how StateStores operate under each processing mode, and it's
> > > important that custom implementations adhere to those assumptions in
> > order
> > > to maintain the consistency guarantees.
> > >
> > > IsolationLevels provide a high-level contract on the behaviour of the
> > > StateStore: a user knows that under READ_COMMITTED, they will see
> writes
> > > only after the Task has committed, and under READ_UNCOMMITTED they will
> > see
> > > writes immediately. No understanding of the details of each
> > processing.mode
> > > is required, either for IQ users or StateStore implementers.
> > >
> > > An argument can be made that these contractual guarantees can simply be
> > > documented for the processing.mode (i.e. that exactly-once and
> > > exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves
> like
> > > READ_UNCOMMITTED), but there are several small issues with this I'd
> > prefer
> > > to avoid:
> > >
> > >    - Where would we document these contracts, in a way that is
> difficult
> > >    for users/implementers to miss/ignore?
> > >    - It's not clear to users that the processing mode is communicating
> > >    an expectation of read isolation, unless they read the
> documentation.
> > > Users
> > >    rarely consult documentation unless they feel they need to, so it's
> > > likely
> > >    this detail would get missed by many users.
> > >    - It tightly couples processing modes to read isolation. Adding new
> > >    processing modes, or changing the read isolation of existing
> > processing
> > >    modes would be difficult/impossible.
> > >
> > > Ultimately, the cost of introducing IsolationLevels is just a single
> > > method, since we re-use the existing IsolationLevel enum from Kafka.
> This
> > > gives us a clear place to document the contractual guarantees expected
> > > of/provided by StateStores, that is accessible both by the StateStore
> > > itself, and by IQ users.
> > >
> > > (Writing this I've just realised that the StateStore and IQ APIs
> actually
> > > don't provide access to StateStoreContext that IQ users would have
> direct
> > > access to... Perhaps StateStore should expose isolationLevel() itself
> > too?)
> > >
> > > 4.
> > > Yeah, I'm not comfortable renaming the metrics in-place either, as
> it's a
> > > backwards incompatible change. My concern is that, if we leave the
> > existing
> > > "flush" metrics in place, they will be confusing to users. Right now,
> > > "flush" metrics record explicit flushes to disk, but under KIP-892,
> even
> > a
> > > commit() will not explicitly flush data to disk - RocksDB will decide
> on
> > > when to flush memtables to disk itself.
> > >
> > > If we keep the existing "flush" metrics, we'd have two options, which
> > both
> > > seem pretty bad to me:
> > >
> > >    1. Have them record calls to commit(), which would be misleading, as
> > >    data is no longer explicitly "flushed" to disk by this call.
> > >    2. Have them record nothing at all, which is equivalent to removing
> > the
> > >    metrics, except that users will see the metric still exists and so
> > > assume
> > >    that the metric is correct, and that there's a problem with their
> > system
> > >    when there isn't.
> > >
> > > I agree that removing them is also a bad solution, and I'd like some
> > > guidance on the best path forward here.
> > >
> > > 5.
> > > Position files are updated on every write to a StateStore. Since our
> > writes
> > > are now buffered until commit(), we can't update the Position file
> until
> > > commit() has been called, otherwise it would be inconsistent with the
> > data
> > > in the event of a rollback. Consequently, we need to manage these
> offsets
> > > the same way we manage the checkpoint offsets, and ensure they're only
> > > written on commit().
> > >
> > > 6.
> > > Agreed, although I'm not exactly sure yet what tests to write. How
> > explicit
> > > do we need to be here in the KIP?
> > >
> > > As for upgrade/downgrade: upgrade is designed to be seamless, and we
> > should
> > > definitely add some tests around that. Downgrade, it transpires, isn't
> > > currently possible, as the extra column family for offset storage is
> > > incompatible with the pre-KIP-892 implementation: when you open a
> RocksDB
> > > database, you must open all available column families or receive an
> > error.
> > > What currently happens on downgrade is that it attempts to open the
> > store,
> > > throws an error about the offsets column family not being opened, which
> > > triggers a wipe and rebuild of the Task. Given that downgrades should
> be
> > > uncommon, I think this is acceptable behaviour, as the end-state is
> > > consistent, even if it results in an undesirable state restore.
> > >
> > > Should I document the upgrade/downgrade behaviour explicitly in the
> KIP?
> > >
> > > --
> > >
> > > Regards,
> > > Nick
> > >
> > >
> > > On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org>
> wrote:
> > >
> > > > Hi Nick!
> > > >
> > > > Thanks for the updates!
> > > >
> > > > 1.
> > > > Why does StateStore#flush() default to
> > > > StateStore#commit(Collections.emptyMap())?
> > > > Since calls to flush() will not exist anymore after this KIP is
> > > > released, I would rather throw an unsupported operation exception by
> > > > default.
> > > >
> > > >
> > > > 2.
> > > > When would a state store return -1 from
> > > > StateStore#approximateNumUncommittedBytes() while being
> transactional?
> > > >
> > > > Wouldn't StateStore#approximateNumUncommittedBytes() also return 0 if
> > > > the state store is transactional but nothing has been written to the
> > > > state store yet?
> > > >
> > > >
> > > > 3.
> > > > Sorry for bringing this up again. Does this KIP really need to
> > introduce
> > > > StateStoreContext#isolationLevel()? StateStoreContext has already
> > > > appConfigs() which basically exposes the same information, i.e., if
> EOS
> > > > is enabled or not.
> > > > In one of your previous e-mails you wrote:
> > > >
> > > > "My idea was to try to keep the StateStore interface as loosely
> coupled
> > > > from the Streams engine as possible, to give implementers more
> freedom,
> > > > and reduce the amount of internal knowledge required."
> > > >
> > > > While I understand the intent, I doubt that it decreases the coupling
> > of
> > > > a StateStore interface and the Streams engine. READ_COMMITTED only
> > > > applies to IQ but not to reads by processors. Thus, implementers need
> > to
> > > > understand how Streams accesses the state stores.
> > > >
> > > > I would like to hear what others think about this.
> > > >
> > > >
> > > > 4.
> > > > Great exposing new metrics for transactional state stores! However, I
> > > > would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides unit
> > > > tests. Additionally, we probably need integration or system tests to
> > > > verify that upgrades and downgrades between transactional and
> > > > non-transactional state stores work as expected.
> > > >
> > > >
> > > > Best,
> > > > Bruno
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > On 7/21/23 10:34 PM, Nick Telford wrote:
> > > > > One more thing: I noted John's suggestion in the KIP, under
> "Rejected
> > > > > Alternatives". I still think it's an idea worth pursuing, but I
> > believe
> > > > > that it's out of the scope of this KIP, because it solves a
> different
> > > set
> > > > > of problems to this KIP, and the scope of this one has already
> grown
> > > > quite
> > > > > large!
> > > > >
> > > > > On Fri, 21 Jul 2023 at 21:33, Nick Telford <nick.telford@gmail.com
> >
> > > > wrote:
> > > > >
> > > > >> Hi everyone,
> > > > >>
> > > > >> I've updated the KIP (
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > )
> > > > >> with the latest changes; mostly bringing back "Atomic
> Checkpointing"
> > > > (for
> > > > >> what feels like the 10th time!). I think the one thing missing is
> > some
> > > > >> changes to metrics (notably the store "flush" metrics will need to
> > be
> > > > >> renamed to "commit").
> > > > >>
> > > > >> The reason I brought back Atomic Checkpointing was to decouple
> store
> > > > flush
> > > > >> from store commit. This is important, because with Transactional
> > > > >> StateStores, we now need to call "flush" on *every* Task commit,
> and
> > > not
> > > > >> just when the StateStore is closing, otherwise our transaction
> > buffer
> > > > will
> > > > >> never be written and persisted, instead growing unbounded! I
> > > > experimented
> > > > >> with some simple solutions, like forcing a store flush whenever
> the
> > > > >> transaction buffer was likely to exceed its configured size, but
> > this
> > > > was
> > > > >> brittle: it prevented the transaction buffer from being configured
> > to
> > > be
> > > > >> unbounded, and it still would have required explicit flushes of
> > > RocksDB,
> > > > >> yielding sub-optimal performance and memory utilization.
> > > > >>
> > > > >> I deemed Atomic Checkpointing to be the "right" way to resolve
> this
> > > > >> problem. By ensuring that the changelog offsets that correspond to
> > the
> > > > most
> > > > >> recently written records are always atomically written to the
> > > StateStore
> > > > >> (by writing them to the same transaction buffer), we can avoid
> > > forcibly
> > > > >> flushing the RocksDB memtables to disk, letting RocksDB flush them
> > > only
> > > > >> when necessary, without losing any of our consistency guarantees.
> > See
> > > > the
> > > > >> updated KIP for more info.
> > > > >>
> > > > >> I have fully implemented these changes, although I'm still not
> > > entirely
> > > > >> happy with the implementation for segmented StateStores, so I plan
> > to
> > > > >> refactor that. Despite that, all tests pass. If you'd like to try
> > out
> > > or
> > > > >> review this highly experimental and incomplete branch, it's
> > available
> > > > here:
> > > > >> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note:
> it's
> > > > built
> > > > >> against Kafka 3.5.0 so that I had a stable base to build and test
> it
> > > on,
> > > > >> and to enable easy apples-to-apples comparisons in a live
> > > environment. I
> > > > >> plan to rebase it against trunk once it's nearer completion and
> has
> > > been
> > > > >> proven on our main application.
> > > > >>
> > > > >> I would really appreciate help in reviewing and testing:
> > > > >> - Segmented (Versioned, Session and Window) stores
> > > > >> - Global stores
> > > > >>
> > > > >> As I do not currently use either of these, so my primary test
> > > > environment
> > > > >> doesn't test these areas.
> > > > >>
> > > > >> I'm going on Parental Leave starting next week for a few weeks, so
> > > will
> > > > >> not have time to move this forward until late August. That said,
> > your
> > > > >> feedback is welcome and appreciated, I just won't be able to
> respond
> > > as
> > > > >> quickly as usual.
> > > > >>
> > > > >> Regards,
> > > > >> Nick
> > > > >>
> > > > >> On Mon, 3 Jul 2023 at 16:23, Nick Telford <nick.telford@gmail.com
> >
> > > > wrote:
> > > > >>
> > > > >>> Hi Bruno
> > > > >>>
> > > > >>> Yes, that's correct, although the impact on IQ is not something I
> > had
> > > > >>> considered.
> > > > >>>
> > > > >>> What about atomically updating the state store from the
> transaction
> > > > >>>> buffer every commit interval and writing the checkpoint (thus,
> > > > flushing
> > > > >>>> the memtable) every configured amount of data and/or number of
> > > commit
> > > > >>>> intervals?
> > > > >>>>
> > > > >>>
> > > > >>> I'm not quite sure I follow. Are you suggesting that we add an
> > > > additional
> > > > >>> config for the max number of commit intervals between
> checkpoints?
> > > That
> > > > >>> way, we would checkpoint *either* when the transaction buffers
> are
> > > > nearly
> > > > >>> full, *OR* whenever a certain number of commit intervals have
> > > elapsed,
> > > > >>> whichever comes first?
> > > > >>>
> > > > >>> That certainly seems reasonable, although this re-ignites an
> > earlier
> > > > >>> debate about whether a config should be measured in "number of
> > commit
> > > > >>> intervals", instead of just an absolute time.
> > > > >>>
> > > > >>> FWIW, I realised that this issue is the reason I was pursuing the
> > > > Atomic
> > > > >>> Checkpoints, as it de-couples memtable flush from checkpointing,
> > > which
> > > > >>> enables us to just checkpoint on every commit without any
> > performance
> > > > >>> impact. Atomic Checkpointing is definitely the "best" solution,
> but
> > > > I'm not
> > > > >>> sure if this is enough to bring it back into this KIP.
> > > > >>>
> > > > >>> I'm currently working on moving all the transactional logic
> > directly
> > > > into
> > > > >>> RocksDBStore itself, which does away with the
> > > StateStore#newTransaction
> > > > >>> method, and reduces the number of new classes introduced,
> > > significantly
> > > > >>> reducing the complexity. If it works, and the complexity is
> > > drastically
> > > > >>> reduced, I may try bringing back Atomic Checkpoints into this
> KIP.
> > > > >>>
> > > > >>> Regards,
> > > > >>> Nick
> > > > >>>
> > > > >>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org>
> > > wrote:
> > > > >>>
> > > > >>>> Hi Nick,
> > > > >>>>
> > > > >>>> Thanks for the insights! Very interesting!
> > > > >>>>
> > > > >>>> As far as I understand, you want to atomically update the state
> > > store
> > > > >>>> from the transaction buffer, flush the memtable of a state store
> > and
> > > > >>>> write the checkpoint not after the commit time elapsed but after
> > the
> > > > >>>> transaction buffer reached a size that would lead to exceeding
> > > > >>>> statestore.transaction.buffer.max.bytes before the next commit
> > > > interval
> > > > >>>> ends.
> > > > >>>> That means, the Kafka transaction would commit every commit
> > interval
> > > > but
> > > > >>>> the state store will only be atomically updated roughly every
> > > > >>>> statestore.transaction.buffer.max.bytes of data. Also IQ would
> > then
> > > > only
> > > > >>>> see new data roughly every
> > statestore.transaction.buffer.max.bytes.
> > > > >>>> After a failure the state store needs to restore up to
> > > > >>>> statestore.transaction.buffer.max.bytes.
> > > > >>>>
> > > > >>>> Is this correct?
> > > > >>>>
> > > > >>>> What about atomically updating the state store from the
> > transaction
> > > > >>>> buffer every commit interval and writing the checkpoint (thus,
> > > > flushing
> > > > >>>> the memtable) every configured amount of data and/or number of
> > > commit
> > > > >>>> intervals? In such a way, we would have the same delay for
> records
> > > > >>>> appearing in output topics and IQ because both would appear when
> > the
> > > > >>>> Kafka transaction is committed. However, after a failure the
> state
> > > > store
> > > > >>>> still needs to restore up to
> > statestore.transaction.buffer.max.bytes
> > > > and
> > > > >>>> it might restore data that is already in the state store because
> > the
> > > > >>>> checkpoint lags behind the last stable offset (i.e. the last
> > > committed
> > > > >>>> offset) of the changelog topics. Restoring data that is already
> in
> > > the
> > > > >>>> state store is idempotent, so eos should not violated.
> > > > >>>> This solution needs at least one new config to specify when a
> > > > checkpoint
> > > > >>>> should be written.
> > > > >>>>
> > > > >>>>
> > > > >>>>
> > > > >>>> A small correction to your previous e-mail that does not change
> > > > anything
> > > > >>>> you said: Under alos the default commit interval is 30 seconds,
> > not
> > > > five
> > > > >>>> seconds.
> > > > >>>>
> > > > >>>>
> > > > >>>> Best,
> > > > >>>> Bruno
> > > > >>>>
> > > > >>>>
> > > > >>>> On 01.07.23 12:37, Nick Telford wrote:
> > > > >>>>> Hi everyone,
> > > > >>>>>
> > > > >>>>> I've begun performance testing my branch on our staging
> > > environment,
> > > > >>>>> putting it through its paces in our non-trivial application.
> I'm
> > > > >>>> already
> > > > >>>>> observing the same increased flush rate that we saw the last
> time
> > > we
> > > > >>>>> attempted to use a version of this KIP, but this time, I think
> I
> > > know
> > > > >>>> why.
> > > > >>>>>
> > > > >>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end
> of
> > > the
> > > > >>>> Task
> > > > >>>>> commit process, has the following behaviour:
> > > > >>>>>
> > > > >>>>>      - Under ALOS: checkpoint the state stores. This includes
> > > > >>>>>      flushing memtables in RocksDB. This is acceptable because
> > the
> > > > >>>> default
> > > > >>>>>      commit.interval.ms is 5 seconds, so forcibly flushing
> > > memtables
> > > > >>>> every 5
> > > > >>>>>      seconds is acceptable for most applications.
> > > > >>>>>      - Under EOS: checkpointing is not done, *unless* it's
> being
> > > > >>>> forced, due
> > > > >>>>>      to e.g. the Task closing or being revoked. This means that
> > > under
> > > > >>>> normal
> > > > >>>>>      processing conditions, the state stores will not be
> > > > checkpointed,
> > > > >>>> and will
> > > > >>>>>      not have memtables flushed at all , unless RocksDB decides
> > to
> > > > >>>> flush them on
> > > > >>>>>      its own. Checkpointing stores and force-flushing their
> > > memtables
> > > > >>>> is only
> > > > >>>>>      done when a Task is being closed.
> > > > >>>>>
> > > > >>>>> Under EOS, KIP-892 needs to checkpoint stores on at least
> *some*
> > > > normal
> > > > >>>>> Task commits, in order to write the RocksDB transaction buffers
> > to
> > > > the
> > > > >>>>> state stores, and to ensure the offsets are synced to disk to
> > > prevent
> > > > >>>>> restores from getting out of hand. Consequently, my current
> > > > >>>> implementation
> > > > >>>>> calls maybeCheckpoint on *every* Task commit, which is far too
> > > > >>>> frequent.
> > > > >>>>> This causes checkpoints every 10,000 records, which is a change
> > in
> > > > >>>> flush
> > > > >>>>> behaviour, potentially causing performance problems for some
> > > > >>>> applications.
> > > > >>>>>
> > > > >>>>> I'm looking into possible solutions, and I'm currently leaning
> > > > towards
> > > > >>>>> using the statestore.transaction.buffer.max.bytes configuration
> > to
> > > > >>>>> checkpoint Tasks once we are likely to exceed it. This would
> > > > >>>> complement the
> > > > >>>>> existing "early Task commit" functionality that this
> > configuration
> > > > >>>>> provides, in the following way:
> > > > >>>>>
> > > > >>>>>      - Currently, we use
> statestore.transaction.buffer.max.bytes
> > to
> > > > >>>> force an
> > > > >>>>>      early Task commit if processing more records would cause
> our
> > > > state
> > > > >>>> store
> > > > >>>>>      transactions to exceed the memory assigned to them.
> > > > >>>>>      - New functionality: when a Task *does* commit, we will
> not
> > > > >>>> checkpoint
> > > > >>>>>      the stores (and hence flush the transaction buffers)
> unless
> > we
> > > > >>>> expect to
> > > > >>>>>      cross the statestore.transaction.buffer.max.bytes
> threshold
> > > > before
> > > > >>>> the next
> > > > >>>>>      commit
> > > > >>>>>
> > > > >>>>> I'm also open to suggestions.
> > > > >>>>>
> > > > >>>>> Regards,
> > > > >>>>> Nick
> > > > >>>>>
> > > > >>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> > nick.telford@gmail.com
> > > >
> > > > >>>> wrote:
> > > > >>>>>
> > > > >>>>>> Hi Bruno!
> > > > >>>>>>
> > > > >>>>>> 3.
> > > > >>>>>> By "less predictable for users", I meant in terms of
> > understanding
> > > > the
> > > > >>>>>> performance profile under various circumstances. The more
> > complex
> > > > the
> > > > >>>>>> solution, the more difficult it would be for users to
> understand
> > > the
> > > > >>>>>> performance they see. For example, spilling records to disk
> when
> > > the
> > > > >>>>>> transaction buffer reaches a threshold would, I expect, reduce
> > > write
> > > > >>>>>> throughput. This reduction in write throughput could be
> > > unexpected,
> > > > >>>> and
> > > > >>>>>> potentially difficult to diagnose/understand for users.
> > > > >>>>>> At the moment, I think the "early commit" concept is
> relatively
> > > > >>>>>> straightforward; it's easy to document, and conceptually
> fairly
> > > > >>>> obvious to
> > > > >>>>>> users. We could probably add a metric to make it easier to
> > > > understand
> > > > >>>> when
> > > > >>>>>> it happens though.
> > > > >>>>>>
> > > > >>>>>> 3. (the second one)
> > > > >>>>>> The IsolationLevel is *essentially* an indirect way of telling
> > > > >>>> StateStores
> > > > >>>>>> whether they should be transactional. READ_COMMITTED
> essentially
> > > > >>>> requires
> > > > >>>>>> transactions, because it dictates that two threads calling
> > > > >>>>>> `newTransaction()` should not see writes from the other
> > > transaction
> > > > >>>> until
> > > > >>>>>> they have been committed. With READ_UNCOMMITTED, all bets are
> > off,
> > > > and
> > > > >>>>>> stores can allow threads to observe written records at any
> time,
> > > > >>>> which is
> > > > >>>>>> essentially "no transactions". That said, StateStores are free
> > to
> > > > >>>> implement
> > > > >>>>>> these guarantees however they can, which is a bit more relaxed
> > > than
> > > > >>>>>> dictating "you must use transactions". For example, with
> RocksDB
> > > we
> > > > >>>> would
> > > > >>>>>> implement these as READ_COMMITTED == WBWI-based
> "transactions",
> > > > >>>>>> READ_UNCOMMITTED == direct writes to the database. But with
> > other
> > > > >>>> storage
> > > > >>>>>> engines, it might be preferable to *always* use transactions,
> > even
> > > > >>>> when
> > > > >>>>>> unnecessary; or there may be storage engines that don't
> provide
> > > > >>>>>> transactions, but the isolation guarantees can be met using a
> > > > >>>> different
> > > > >>>>>> technique.
> > > > >>>>>> My idea was to try to keep the StateStore interface as loosely
> > > > coupled
> > > > >>>>>> from the Streams engine as possible, to give implementers more
> > > > >>>> freedom, and
> > > > >>>>>> reduce the amount of internal knowledge required.
> > > > >>>>>> That said, I understand that "IsolationLevel" might not be the
> > > right
> > > > >>>>>> abstraction, and we can always make it much more explicit if
> > > > >>>> required, e.g.
> > > > >>>>>> boolean transactional()
> > > > >>>>>>
> > > > >>>>>> 7-8.
> > > > >>>>>> I can make these changes either later today or tomorrow.
> > > > >>>>>>
> > > > >>>>>> Small update:
> > > > >>>>>> I've rebased my branch on trunk and fixed a bunch of issues
> that
> > > > >>>> needed
> > > > >>>>>> addressing. Currently, all the tests pass, which is promising,
> > but
> > > > it
> > > > >>>> will
> > > > >>>>>> need to undergo some performance testing. I haven't (yet)
> worked
> > > on
> > > > >>>>>> removing the `newTransaction()` stuff, but I would expect
> that,
> > > > >>>>>> behaviourally, it should make no difference. The branch is
> > > available
> > > > >>>> at
> > > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone
> > is
> > > > >>>>>> interested in taking an early look.
> > > > >>>>>>
> > > > >>>>>> Regards,
> > > > >>>>>> Nick
> > > > >>>>>>
> > > > >>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <
> cadonna@apache.org
> > >
> > > > >>>> wrote:
> > > > >>>>>>
> > > > >>>>>>> Hi Nick,
> > > > >>>>>>>
> > > > >>>>>>> 1.
> > > > >>>>>>> Yeah, I agree with you. That was actually also my point. I
> > > > understood
> > > > >>>>>>> that John was proposing the ingestion path as a way to avoid
> > the
> > > > >>>> early
> > > > >>>>>>> commits. Probably, I misinterpreted the intent.
> > > > >>>>>>>
> > > > >>>>>>> 2.
> > > > >>>>>>> I agree with John here, that actually it is public API. My
> > > question
> > > > >>>> is
> > > > >>>>>>> how this usage pattern affects normal processing.
> > > > >>>>>>>
> > > > >>>>>>> 3.
> > > > >>>>>>> My concern is that checking for the size of the transaction
> > > buffer
> > > > >>>> and
> > > > >>>>>>> maybe triggering an early commit affects the whole processing
> > of
> > > > >>>> Kafka
> > > > >>>>>>> Streams. The transactionality of a state store is not
> confined
> > to
> > > > the
> > > > >>>>>>> state store itself, but spills over and changes the behavior
> of
> > > > other
> > > > >>>>>>> parts of the system. I agree with you that it is a decent
> > > > >>>> compromise. I
> > > > >>>>>>> just wanted to analyse the downsides and list the options to
> > > > overcome
> > > > >>>>>>> them. I also agree with you that all options seem quite heavy
> > > > >>>> compared
> > > > >>>>>>> with your KIP. I do not understand what you mean with "less
> > > > >>>> predictable
> > > > >>>>>>> for users", though.
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>> I found the discussions about the alternatives really
> > > interesting.
> > > > >>>> But I
> > > > >>>>>>> also think that your plan sounds good and we should continue
> > with
> > > > it!
> > > > >>>>>>>
> > > > >>>>>>>
> > > > >>>>>>> Some comments on your reply to my e-mail on June 20th:
> > > > >>>>>>>
> > > > >>>>>>> 3.
> > > > >>>>>>> Ah, now, I understand the reasoning behind putting isolation
> > > level
> > > > in
> > > > >>>>>>> the state store context. Thanks! Should that also be a way to
> > > give
> > > > >>>> the
> > > > >>>>>>> the state store the opportunity to decide whether to turn on
> > > > >>>>>>> transactions or not?
> > > > >>>>>>> With my comment, I was more concerned about how do you know
> if
> > a
> > > > >>>>>>> checkpoint file needs to be written under EOS, if you do not
> > > have a
> > > > >>>> way
> > > > >>>>>>> to know if the state store is transactional or not. If a
> state
> > > > store
> > > > >>>> is
> > > > >>>>>>> transactional, the checkpoint file can be written during
> normal
> > > > >>>>>>> processing under EOS. If the state store is not
> transactional,
> > > the
> > > > >>>>>>> checkpoint file must not be written under EOS.
> > > > >>>>>>>
> > > > >>>>>>> 7.
> > > > >>>>>>> My point was about not only considering the bytes in memory
> in
> > > > config
> > > > >>>>>>> statestore.uncommitted.max.bytes, but also bytes that might
> be
> > > > >>>> spilled
> > > > >>>>>>> on disk. Basically, I was wondering whether you should remove
> > the
> > > > >>>>>>> "memory" in "Maximum number of memory bytes to be used to
> > > > >>>>>>> buffer uncommitted state-store records." My thinking was that
> > > even
> > > > >>>> if a
> > > > >>>>>>> state store spills uncommitted bytes to disk, limiting the
> > > overall
> > > > >>>> bytes
> > > > >>>>>>> might make sense. Thinking about it again and considering the
> > > > recent
> > > > >>>>>>> discussions, it does not make too much sense anymore.
> > > > >>>>>>> I like the name statestore.transaction.buffer.max.bytes that
> > you
> > > > >>>> proposed.
> > > > >>>>>>>
> > > > >>>>>>> 8.
> > > > >>>>>>> A high-level description (without implementation details) of
> > how
> > > > >>>> Kafka
> > > > >>>>>>> Streams will manage the commit of changelog transactions,
> state
> > > > store
> > > > >>>>>>> transactions and checkpointing would be great. Would be great
> > if
> > > > you
> > > > >>>>>>> could also add some sentences about the behavior in case of a
> > > > >>>> failure.
> > > > >>>>>>> For instance how does a transactional state store recover
> > after a
> > > > >>>>>>> failure or what happens with the transaction buffer, etc.
> (that
> > > is
> > > > >>>> what
> > > > >>>>>>> I meant by "fail-over" in point 9.)
> > > > >>>>>>>
> > > > >>>>>>> Best,
> > > > >>>>>>> Bruno
> > > > >>>>>>>
> > > > >>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > > > >>>>>>>> Hi Bruno,
> > > > >>>>>>>>
> > > > >>>>>>>> 1.
> > > > >>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
> > > > >>>> transactions
> > > > >>>>>>>> have, whereby exceeding (or likely to exceed) configured
> > memory
> > > > >>>> needs to
> > > > >>>>>>>> trigger an early commit?
> > > > >>>>>>>>
> > > > >>>>>>>> 2.
> > > > >>>>>>>> This is one of my big concerns. Ultimately, any approach
> based
> > > on
> > > > >>>>>>> cracking
> > > > >>>>>>>> open RocksDB internals and using it in ways it's not really
> > > > designed
> > > > >>>>>>> for is
> > > > >>>>>>>> likely to have some unforseen performance or consistency
> > issues.
> > > > >>>>>>>>
> > > > >>>>>>>> 3.
> > > > >>>>>>>> What's your motivation for removing these early commits?
> While
> > > not
> > > > >>>>>>> ideal, I
> > > > >>>>>>>> think they're a decent compromise to ensure consistency
> whilst
> > > > >>>>>>> maintaining
> > > > >>>>>>>> good and predictable performance.
> > > > >>>>>>>> All 3 of your suggested ideas seem *very* complicated, and
> > might
> > > > >>>>>>> actually
> > > > >>>>>>>> make behaviour less predictable for users as a consequence.
> > > > >>>>>>>>
> > > > >>>>>>>> I'm a bit concerned that the scope of this KIP is growing a
> > bit
> > > > out
> > > > >>>> of
> > > > >>>>>>>> control. While it's good to discuss ideas for future
> > > > improvements, I
> > > > >>>>>>> think
> > > > >>>>>>>> it's important to narrow the scope down to a design that
> > > achieves
> > > > >>>> the
> > > > >>>>>>> most
> > > > >>>>>>>> pressing objectives (constant sized restorations during
> dirty
> > > > >>>>>>>> close/unexpected errors). Any design that this KIP produces
> > can
> > > > >>>>>>> ultimately
> > > > >>>>>>>> be changed in the future, especially if the bulk of it is
> > > internal
> > > > >>>>>>>> behaviour.
> > > > >>>>>>>>
> > > > >>>>>>>> I'm going to spend some time next week trying to re-work the
> > > > >>>> original
> > > > >>>>>>>> WriteBatchWithIndex design to remove the newTransaction()
> > > method,
> > > > >>>> such
> > > > >>>>>>> that
> > > > >>>>>>>> it's just an implementation detail of RocksDBStore. That
> way,
> > if
> > > > we
> > > > >>>>>>> want to
> > > > >>>>>>>> replace WBWI with something in the future, like the SST file
> > > > >>>> management
> > > > >>>>>>>> outlined by John, then we can do so with little/no API
> > changes.
> > > > >>>>>>>>
> > > > >>>>>>>> Regards,
> > > > >>>>>>>>
> > > > >>>>>>>> Nick
> > > > >>>>>>>>
> > > > >>>>>>>
> > > > >>>>>>
> > > > >>>>>
> > > > >>>>
> > > > >>>
> > > > >
> > > >
> > >
> >
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Colt,

Thanks for taking the time to run your benchmarks on this, that's
incredibly helpful.

> With KIP 892, I verified that unclean shutdown does not cause a fresh
> restore (!!!!). I got the following benchmark results:
> - Benchmark took 216 seconds
> - 1,401 tasks per second on one partition
> - 11 seconds to restore the state

Can you clarify how much state was restored in those 11 seconds? Was this
the time to do the full restore regardless, or was it the time to only
restore a small fraction of the state (e.g. the last aborted transaction)?

> -- QUESTION: Because we observed a significant (30% or so) and
reproducible
> slowdown during restoration, it seems like KIP-892 uses the checkpointing
> behavior during restoration as well? If so, I would argue that this might
> not be necessary, because everything we write is already committed, so we
> don't need to change the behavior during restoration or standby tasks.
> Perhaps we could write the offsets to RocksDB on every batch (or even
every
> 5 seconds or so).

Restore has always used a completely separate code-path to regular writes,
and continues to do so. I had a quick pass over the code and I suspect I
know what's causing the performance degradation: for every restored record,
I was adding the changelog offset of that record to the batch along with
the record. This is different to the regular write-path, which only adds
the current offsets once, on-commit. This writeOffset method is fairly
expensive, since it has to serialize the TopicPartition and offset that's
being written to the database.

Assuming this is the cause, I've already pushed a fix to my branch that
will only call writeOffset once per-batch, and also adds some caching to
the serialization in writeOffset, that should also enhance performance of
state commit in the normal write-path.

Please let me know if this addresses the issue!

Regards,
Nick


On Mon, 11 Sept 2023 at 05:38, Colt McNealy <co...@littlehorse.io> wrote:

> Howdy folks,
>
> First I wanted to say fantastic work and thank you to Nick. I built your
> branch (https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0) and did
> some testing on our Streams app with Kafka 3.5.0, your `kip-892-3.5.0`
> branch, and your `kip-892-3.5.0` branch built with Speedb OSS 2.3.0.1. And
> it worked! Including the global store (we don't have any segmented stores,
> unfortunately).
>
> The test I ran involved running 3,000 workflows with 100 tasks each, and
> roughly 650MB state total.
>
> With Streams 3.5.0, I indeed verified that unclean shutdown caused a fresh
> restore from scratch. I also benchmarked my application at:
> - Running the benchmark took 211 seconds
> - 1,421 tasks per second on one partition
> - 8 seconds to restore the state (650MB or so)
>
> With KIP 892, I verified that unclean shutdown does not cause a fresh
> restore (!!!!). I got the following benchmark results:
> - Benchmark took 216 seconds
> - 1,401 tasks per second on one partition
> - 11 seconds to restore the state
>
> I ran the restorations many times to ensure that there was no rounding
> error or noise; the results were remarkably consistent. Additionally, I ran
> the restorations with KIP-892 built with Speedb OSS. The restoration time
> consistently came out as 10 seconds, which was an improvement from the 11
> seconds observed with RocksDB + KIP-892.
>
> My application is bottlenecked mostly by serialization and deserialization,
> so improving the performance of the state store doesn't really impact our
> throughput that much. And the processing performance (benchmark time,
> tasks/second) are pretty close in KIP-892 vs Streams 3.5.0. However, at
> larger state store sizes, RocksDB performance begins to degrade, so that
> might not be true once we pass 20GB per partition.
>
> -- QUESTION: Because we observed a significant (30% or so) and reproducible
> slowdown during restoration, it seems like KIP-892 uses the checkpointing
> behavior during restoration as well? If so, I would argue that this might
> not be necessary, because everything we write is already committed, so we
> don't need to change the behavior during restoration or standby tasks.
> Perhaps we could write the offsets to RocksDB on every batch (or even every
> 5 seconds or so).
>
> -- Note: This was a very small-scale test, with <1GB of state (as I didn't
> have time to spend hours building up state). In the past I have noted that
> RocksDB performance degrades significantly after 25GB of state in one
> store. Future work involves determining the performance impact of KIP-892
> relative to trunk at larger scale, since it's possible that the relative
> behaviors are far different (i.e. relative to trunk, 892's processing and
> restoration throughput might be much better or much worse).
>
> -- Note: For those who want to replicate the tests, you can find the branch
> of our streams app here:
>
> https://github.com/littlehorse-enterprises/littlehorse/tree/minor/testing-streams-forks
> . The example I ran was `examples/hundred-tasks`, and I ran the server with
> `./local-dev/do-server.sh one-partition`. The `STREAMS_TESTS.md` file 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.
>
> Thanks Nick!
>
> Ride well,
> Colt McNealy
>
> *Founder, LittleHorse.dev*
>
>
> On Thu, Aug 24, 2023 at 3:19 AM Nick Telford <ni...@gmail.com>
> wrote:
>
> > Hi Bruno,
> >
> > Thanks for taking the time to review the KIP. I'm back from leave now and
> > intend to move this forwards as quickly as I can.
> >
> > Addressing your points:
> >
> > 1.
> > Because flush() is part of the StateStore API, it's exposed to custom
> > Processors, which might be making calls to flush(). This was actually the
> > case in a few integration tests.
> > To maintain as much compatibility as possible, I'd prefer not to make
> this
> > an UnsupportedOperationException, as it will cause previously working
> > Processors to start throwing exceptions at runtime.
> > I agree that it doesn't make sense for it to proxy commit(), though, as
> > that would cause it to violate the "StateStores commit only when the Task
> > commits" rule.
> > Instead, I think we should make this a no-op. That way, existing user
> > Processors will continue to work as-before, without violation of store
> > consistency that would be caused by premature flush/commit of StateStore
> > data to disk.
> > What do you think?
> >
> > 2.
> > As stated in the JavaDoc, when a StateStore implementation is
> > transactional, but is unable to estimate the uncommitted memory usage,
> the
> > method will return -1.
> > The intention here is to permit third-party implementations that may not
> be
> > able to estimate memory usage.
> >
> > Yes, it will be 0 when nothing has been written to the store yet. I
> thought
> > that was implied by "This method will return an approximation of the
> memory
> > would be freed by the next call to {@link #commit(Map)}" and "@return The
> > approximate size of all records awaiting {@link #commit(Map)}", however,
> I
> > can add it explicitly to the JavaDoc if you think this is unclear?
> >
> > 3.
> > I realise this is probably the most contentious point in my design, and
> I'm
> > open to changing it if I'm unable to convince you of the benefits.
> > Nevertheless, here's my argument:
> > The Interactive Query (IQ) API(s) are directly provided StateStores to
> > query, and it may be important for users to programmatically know which
> > mode the StateStore is operating under. If we simply provide an
> > "eosEnabled" boolean (as used throughout the internal streams engine), or
> > similar, then users will need to understand the operation and
> consequences
> > of each available processing mode and how it pertains to their
> StateStore.
> >
> > Interactive Query users aren't the only people that care about the
> > processing.mode/IsolationLevel of a StateStore: implementers of custom
> > StateStores also need to understand the behaviour expected of their
> > implementation. KIP-892 introduces some assumptions into the Streams
> Engine
> > about how StateStores operate under each processing mode, and it's
> > important that custom implementations adhere to those assumptions in
> order
> > to maintain the consistency guarantees.
> >
> > IsolationLevels provide a high-level contract on the behaviour of the
> > StateStore: a user knows that under READ_COMMITTED, they will see writes
> > only after the Task has committed, and under READ_UNCOMMITTED they will
> see
> > writes immediately. No understanding of the details of each
> processing.mode
> > is required, either for IQ users or StateStore implementers.
> >
> > An argument can be made that these contractual guarantees can simply be
> > documented for the processing.mode (i.e. that exactly-once and
> > exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves like
> > READ_UNCOMMITTED), but there are several small issues with this I'd
> prefer
> > to avoid:
> >
> >    - Where would we document these contracts, in a way that is difficult
> >    for users/implementers to miss/ignore?
> >    - It's not clear to users that the processing mode is communicating
> >    an expectation of read isolation, unless they read the documentation.
> > Users
> >    rarely consult documentation unless they feel they need to, so it's
> > likely
> >    this detail would get missed by many users.
> >    - It tightly couples processing modes to read isolation. Adding new
> >    processing modes, or changing the read isolation of existing
> processing
> >    modes would be difficult/impossible.
> >
> > Ultimately, the cost of introducing IsolationLevels is just a single
> > method, since we re-use the existing IsolationLevel enum from Kafka. This
> > gives us a clear place to document the contractual guarantees expected
> > of/provided by StateStores, that is accessible both by the StateStore
> > itself, and by IQ users.
> >
> > (Writing this I've just realised that the StateStore and IQ APIs actually
> > don't provide access to StateStoreContext that IQ users would have direct
> > access to... Perhaps StateStore should expose isolationLevel() itself
> too?)
> >
> > 4.
> > Yeah, I'm not comfortable renaming the metrics in-place either, as it's a
> > backwards incompatible change. My concern is that, if we leave the
> existing
> > "flush" metrics in place, they will be confusing to users. Right now,
> > "flush" metrics record explicit flushes to disk, but under KIP-892, even
> a
> > commit() will not explicitly flush data to disk - RocksDB will decide on
> > when to flush memtables to disk itself.
> >
> > If we keep the existing "flush" metrics, we'd have two options, which
> both
> > seem pretty bad to me:
> >
> >    1. Have them record calls to commit(), which would be misleading, as
> >    data is no longer explicitly "flushed" to disk by this call.
> >    2. Have them record nothing at all, which is equivalent to removing
> the
> >    metrics, except that users will see the metric still exists and so
> > assume
> >    that the metric is correct, and that there's a problem with their
> system
> >    when there isn't.
> >
> > I agree that removing them is also a bad solution, and I'd like some
> > guidance on the best path forward here.
> >
> > 5.
> > Position files are updated on every write to a StateStore. Since our
> writes
> > are now buffered until commit(), we can't update the Position file until
> > commit() has been called, otherwise it would be inconsistent with the
> data
> > in the event of a rollback. Consequently, we need to manage these offsets
> > the same way we manage the checkpoint offsets, and ensure they're only
> > written on commit().
> >
> > 6.
> > Agreed, although I'm not exactly sure yet what tests to write. How
> explicit
> > do we need to be here in the KIP?
> >
> > As for upgrade/downgrade: upgrade is designed to be seamless, and we
> should
> > definitely add some tests around that. Downgrade, it transpires, isn't
> > currently possible, as the extra column family for offset storage is
> > incompatible with the pre-KIP-892 implementation: when you open a RocksDB
> > database, you must open all available column families or receive an
> error.
> > What currently happens on downgrade is that it attempts to open the
> store,
> > throws an error about the offsets column family not being opened, which
> > triggers a wipe and rebuild of the Task. Given that downgrades should be
> > uncommon, I think this is acceptable behaviour, as the end-state is
> > consistent, even if it results in an undesirable state restore.
> >
> > Should I document the upgrade/downgrade behaviour explicitly in the KIP?
> >
> > --
> >
> > Regards,
> > Nick
> >
> >
> > On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org> wrote:
> >
> > > Hi Nick!
> > >
> > > Thanks for the updates!
> > >
> > > 1.
> > > Why does StateStore#flush() default to
> > > StateStore#commit(Collections.emptyMap())?
> > > Since calls to flush() will not exist anymore after this KIP is
> > > released, I would rather throw an unsupported operation exception by
> > > default.
> > >
> > >
> > > 2.
> > > When would a state store return -1 from
> > > StateStore#approximateNumUncommittedBytes() while being transactional?
> > >
> > > Wouldn't StateStore#approximateNumUncommittedBytes() also return 0 if
> > > the state store is transactional but nothing has been written to the
> > > state store yet?
> > >
> > >
> > > 3.
> > > Sorry for bringing this up again. Does this KIP really need to
> introduce
> > > StateStoreContext#isolationLevel()? StateStoreContext has already
> > > appConfigs() which basically exposes the same information, i.e., if EOS
> > > is enabled or not.
> > > In one of your previous e-mails you wrote:
> > >
> > > "My idea was to try to keep the StateStore interface as loosely coupled
> > > from the Streams engine as possible, to give implementers more freedom,
> > > and reduce the amount of internal knowledge required."
> > >
> > > While I understand the intent, I doubt that it decreases the coupling
> of
> > > a StateStore interface and the Streams engine. READ_COMMITTED only
> > > applies to IQ but not to reads by processors. Thus, implementers need
> to
> > > understand how Streams accesses the state stores.
> > >
> > > I would like to hear what others think about this.
> > >
> > >
> > > 4.
> > > Great exposing new metrics for transactional state stores! However, I
> > > would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides unit
> > > tests. Additionally, we probably need integration or system tests to
> > > verify that upgrades and downgrades between transactional and
> > > non-transactional state stores work as expected.
> > >
> > >
> > > Best,
> > > Bruno
> > >
> > >
> > >
> > >
> > >
> > > On 7/21/23 10:34 PM, Nick Telford wrote:
> > > > One more thing: I noted John's suggestion in the KIP, under "Rejected
> > > > Alternatives". I still think it's an idea worth pursuing, but I
> believe
> > > > that it's out of the scope of this KIP, because it solves a different
> > set
> > > > of problems to this KIP, and the scope of this one has already grown
> > > quite
> > > > large!
> > > >
> > > > On Fri, 21 Jul 2023 at 21:33, Nick Telford <ni...@gmail.com>
> > > wrote:
> > > >
> > > >> Hi everyone,
> > > >>
> > > >> I've updated the KIP (
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > )
> > > >> with the latest changes; mostly bringing back "Atomic Checkpointing"
> > > (for
> > > >> what feels like the 10th time!). I think the one thing missing is
> some
> > > >> changes to metrics (notably the store "flush" metrics will need to
> be
> > > >> renamed to "commit").
> > > >>
> > > >> The reason I brought back Atomic Checkpointing was to decouple store
> > > flush
> > > >> from store commit. This is important, because with Transactional
> > > >> StateStores, we now need to call "flush" on *every* Task commit, and
> > not
> > > >> just when the StateStore is closing, otherwise our transaction
> buffer
> > > will
> > > >> never be written and persisted, instead growing unbounded! I
> > > experimented
> > > >> with some simple solutions, like forcing a store flush whenever the
> > > >> transaction buffer was likely to exceed its configured size, but
> this
> > > was
> > > >> brittle: it prevented the transaction buffer from being configured
> to
> > be
> > > >> unbounded, and it still would have required explicit flushes of
> > RocksDB,
> > > >> yielding sub-optimal performance and memory utilization.
> > > >>
> > > >> I deemed Atomic Checkpointing to be the "right" way to resolve this
> > > >> problem. By ensuring that the changelog offsets that correspond to
> the
> > > most
> > > >> recently written records are always atomically written to the
> > StateStore
> > > >> (by writing them to the same transaction buffer), we can avoid
> > forcibly
> > > >> flushing the RocksDB memtables to disk, letting RocksDB flush them
> > only
> > > >> when necessary, without losing any of our consistency guarantees.
> See
> > > the
> > > >> updated KIP for more info.
> > > >>
> > > >> I have fully implemented these changes, although I'm still not
> > entirely
> > > >> happy with the implementation for segmented StateStores, so I plan
> to
> > > >> refactor that. Despite that, all tests pass. If you'd like to try
> out
> > or
> > > >> review this highly experimental and incomplete branch, it's
> available
> > > here:
> > > >> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note: it's
> > > built
> > > >> against Kafka 3.5.0 so that I had a stable base to build and test it
> > on,
> > > >> and to enable easy apples-to-apples comparisons in a live
> > environment. I
> > > >> plan to rebase it against trunk once it's nearer completion and has
> > been
> > > >> proven on our main application.
> > > >>
> > > >> I would really appreciate help in reviewing and testing:
> > > >> - Segmented (Versioned, Session and Window) stores
> > > >> - Global stores
> > > >>
> > > >> As I do not currently use either of these, so my primary test
> > > environment
> > > >> doesn't test these areas.
> > > >>
> > > >> I'm going on Parental Leave starting next week for a few weeks, so
> > will
> > > >> not have time to move this forward until late August. That said,
> your
> > > >> feedback is welcome and appreciated, I just won't be able to respond
> > as
> > > >> quickly as usual.
> > > >>
> > > >> Regards,
> > > >> Nick
> > > >>
> > > >> On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com>
> > > wrote:
> > > >>
> > > >>> Hi Bruno
> > > >>>
> > > >>> Yes, that's correct, although the impact on IQ is not something I
> had
> > > >>> considered.
> > > >>>
> > > >>> What about atomically updating the state store from the transaction
> > > >>>> buffer every commit interval and writing the checkpoint (thus,
> > > flushing
> > > >>>> the memtable) every configured amount of data and/or number of
> > commit
> > > >>>> intervals?
> > > >>>>
> > > >>>
> > > >>> I'm not quite sure I follow. Are you suggesting that we add an
> > > additional
> > > >>> config for the max number of commit intervals between checkpoints?
> > That
> > > >>> way, we would checkpoint *either* when the transaction buffers are
> > > nearly
> > > >>> full, *OR* whenever a certain number of commit intervals have
> > elapsed,
> > > >>> whichever comes first?
> > > >>>
> > > >>> That certainly seems reasonable, although this re-ignites an
> earlier
> > > >>> debate about whether a config should be measured in "number of
> commit
> > > >>> intervals", instead of just an absolute time.
> > > >>>
> > > >>> FWIW, I realised that this issue is the reason I was pursuing the
> > > Atomic
> > > >>> Checkpoints, as it de-couples memtable flush from checkpointing,
> > which
> > > >>> enables us to just checkpoint on every commit without any
> performance
> > > >>> impact. Atomic Checkpointing is definitely the "best" solution, but
> > > I'm not
> > > >>> sure if this is enough to bring it back into this KIP.
> > > >>>
> > > >>> I'm currently working on moving all the transactional logic
> directly
> > > into
> > > >>> RocksDBStore itself, which does away with the
> > StateStore#newTransaction
> > > >>> method, and reduces the number of new classes introduced,
> > significantly
> > > >>> reducing the complexity. If it works, and the complexity is
> > drastically
> > > >>> reduced, I may try bringing back Atomic Checkpoints into this KIP.
> > > >>>
> > > >>> Regards,
> > > >>> Nick
> > > >>>
> > > >>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org>
> > wrote:
> > > >>>
> > > >>>> Hi Nick,
> > > >>>>
> > > >>>> Thanks for the insights! Very interesting!
> > > >>>>
> > > >>>> As far as I understand, you want to atomically update the state
> > store
> > > >>>> from the transaction buffer, flush the memtable of a state store
> and
> > > >>>> write the checkpoint not after the commit time elapsed but after
> the
> > > >>>> transaction buffer reached a size that would lead to exceeding
> > > >>>> statestore.transaction.buffer.max.bytes before the next commit
> > > interval
> > > >>>> ends.
> > > >>>> That means, the Kafka transaction would commit every commit
> interval
> > > but
> > > >>>> the state store will only be atomically updated roughly every
> > > >>>> statestore.transaction.buffer.max.bytes of data. Also IQ would
> then
> > > only
> > > >>>> see new data roughly every
> statestore.transaction.buffer.max.bytes.
> > > >>>> After a failure the state store needs to restore up to
> > > >>>> statestore.transaction.buffer.max.bytes.
> > > >>>>
> > > >>>> Is this correct?
> > > >>>>
> > > >>>> What about atomically updating the state store from the
> transaction
> > > >>>> buffer every commit interval and writing the checkpoint (thus,
> > > flushing
> > > >>>> the memtable) every configured amount of data and/or number of
> > commit
> > > >>>> intervals? In such a way, we would have the same delay for records
> > > >>>> appearing in output topics and IQ because both would appear when
> the
> > > >>>> Kafka transaction is committed. However, after a failure the state
> > > store
> > > >>>> still needs to restore up to
> statestore.transaction.buffer.max.bytes
> > > and
> > > >>>> it might restore data that is already in the state store because
> the
> > > >>>> checkpoint lags behind the last stable offset (i.e. the last
> > committed
> > > >>>> offset) of the changelog topics. Restoring data that is already in
> > the
> > > >>>> state store is idempotent, so eos should not violated.
> > > >>>> This solution needs at least one new config to specify when a
> > > checkpoint
> > > >>>> should be written.
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>> A small correction to your previous e-mail that does not change
> > > anything
> > > >>>> you said: Under alos the default commit interval is 30 seconds,
> not
> > > five
> > > >>>> seconds.
> > > >>>>
> > > >>>>
> > > >>>> Best,
> > > >>>> Bruno
> > > >>>>
> > > >>>>
> > > >>>> On 01.07.23 12:37, Nick Telford wrote:
> > > >>>>> Hi everyone,
> > > >>>>>
> > > >>>>> I've begun performance testing my branch on our staging
> > environment,
> > > >>>>> putting it through its paces in our non-trivial application. I'm
> > > >>>> already
> > > >>>>> observing the same increased flush rate that we saw the last time
> > we
> > > >>>>> attempted to use a version of this KIP, but this time, I think I
> > know
> > > >>>> why.
> > > >>>>>
> > > >>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end of
> > the
> > > >>>> Task
> > > >>>>> commit process, has the following behaviour:
> > > >>>>>
> > > >>>>>      - Under ALOS: checkpoint the state stores. This includes
> > > >>>>>      flushing memtables in RocksDB. This is acceptable because
> the
> > > >>>> default
> > > >>>>>      commit.interval.ms is 5 seconds, so forcibly flushing
> > memtables
> > > >>>> every 5
> > > >>>>>      seconds is acceptable for most applications.
> > > >>>>>      - Under EOS: checkpointing is not done, *unless* it's being
> > > >>>> forced, due
> > > >>>>>      to e.g. the Task closing or being revoked. This means that
> > under
> > > >>>> normal
> > > >>>>>      processing conditions, the state stores will not be
> > > checkpointed,
> > > >>>> and will
> > > >>>>>      not have memtables flushed at all , unless RocksDB decides
> to
> > > >>>> flush them on
> > > >>>>>      its own. Checkpointing stores and force-flushing their
> > memtables
> > > >>>> is only
> > > >>>>>      done when a Task is being closed.
> > > >>>>>
> > > >>>>> Under EOS, KIP-892 needs to checkpoint stores on at least *some*
> > > normal
> > > >>>>> Task commits, in order to write the RocksDB transaction buffers
> to
> > > the
> > > >>>>> state stores, and to ensure the offsets are synced to disk to
> > prevent
> > > >>>>> restores from getting out of hand. Consequently, my current
> > > >>>> implementation
> > > >>>>> calls maybeCheckpoint on *every* Task commit, which is far too
> > > >>>> frequent.
> > > >>>>> This causes checkpoints every 10,000 records, which is a change
> in
> > > >>>> flush
> > > >>>>> behaviour, potentially causing performance problems for some
> > > >>>> applications.
> > > >>>>>
> > > >>>>> I'm looking into possible solutions, and I'm currently leaning
> > > towards
> > > >>>>> using the statestore.transaction.buffer.max.bytes configuration
> to
> > > >>>>> checkpoint Tasks once we are likely to exceed it. This would
> > > >>>> complement the
> > > >>>>> existing "early Task commit" functionality that this
> configuration
> > > >>>>> provides, in the following way:
> > > >>>>>
> > > >>>>>      - Currently, we use statestore.transaction.buffer.max.bytes
> to
> > > >>>> force an
> > > >>>>>      early Task commit if processing more records would cause our
> > > state
> > > >>>> store
> > > >>>>>      transactions to exceed the memory assigned to them.
> > > >>>>>      - New functionality: when a Task *does* commit, we will not
> > > >>>> checkpoint
> > > >>>>>      the stores (and hence flush the transaction buffers) unless
> we
> > > >>>> expect to
> > > >>>>>      cross the statestore.transaction.buffer.max.bytes threshold
> > > before
> > > >>>> the next
> > > >>>>>      commit
> > > >>>>>
> > > >>>>> I'm also open to suggestions.
> > > >>>>>
> > > >>>>> Regards,
> > > >>>>> Nick
> > > >>>>>
> > > >>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <
> nick.telford@gmail.com
> > >
> > > >>>> wrote:
> > > >>>>>
> > > >>>>>> Hi Bruno!
> > > >>>>>>
> > > >>>>>> 3.
> > > >>>>>> By "less predictable for users", I meant in terms of
> understanding
> > > the
> > > >>>>>> performance profile under various circumstances. The more
> complex
> > > the
> > > >>>>>> solution, the more difficult it would be for users to understand
> > the
> > > >>>>>> performance they see. For example, spilling records to disk when
> > the
> > > >>>>>> transaction buffer reaches a threshold would, I expect, reduce
> > write
> > > >>>>>> throughput. This reduction in write throughput could be
> > unexpected,
> > > >>>> and
> > > >>>>>> potentially difficult to diagnose/understand for users.
> > > >>>>>> At the moment, I think the "early commit" concept is relatively
> > > >>>>>> straightforward; it's easy to document, and conceptually fairly
> > > >>>> obvious to
> > > >>>>>> users. We could probably add a metric to make it easier to
> > > understand
> > > >>>> when
> > > >>>>>> it happens though.
> > > >>>>>>
> > > >>>>>> 3. (the second one)
> > > >>>>>> The IsolationLevel is *essentially* an indirect way of telling
> > > >>>> StateStores
> > > >>>>>> whether they should be transactional. READ_COMMITTED essentially
> > > >>>> requires
> > > >>>>>> transactions, because it dictates that two threads calling
> > > >>>>>> `newTransaction()` should not see writes from the other
> > transaction
> > > >>>> until
> > > >>>>>> they have been committed. With READ_UNCOMMITTED, all bets are
> off,
> > > and
> > > >>>>>> stores can allow threads to observe written records at any time,
> > > >>>> which is
> > > >>>>>> essentially "no transactions". That said, StateStores are free
> to
> > > >>>> implement
> > > >>>>>> these guarantees however they can, which is a bit more relaxed
> > than
> > > >>>>>> dictating "you must use transactions". For example, with RocksDB
> > we
> > > >>>> would
> > > >>>>>> implement these as READ_COMMITTED == WBWI-based "transactions",
> > > >>>>>> READ_UNCOMMITTED == direct writes to the database. But with
> other
> > > >>>> storage
> > > >>>>>> engines, it might be preferable to *always* use transactions,
> even
> > > >>>> when
> > > >>>>>> unnecessary; or there may be storage engines that don't provide
> > > >>>>>> transactions, but the isolation guarantees can be met using a
> > > >>>> different
> > > >>>>>> technique.
> > > >>>>>> My idea was to try to keep the StateStore interface as loosely
> > > coupled
> > > >>>>>> from the Streams engine as possible, to give implementers more
> > > >>>> freedom, and
> > > >>>>>> reduce the amount of internal knowledge required.
> > > >>>>>> That said, I understand that "IsolationLevel" might not be the
> > right
> > > >>>>>> abstraction, and we can always make it much more explicit if
> > > >>>> required, e.g.
> > > >>>>>> boolean transactional()
> > > >>>>>>
> > > >>>>>> 7-8.
> > > >>>>>> I can make these changes either later today or tomorrow.
> > > >>>>>>
> > > >>>>>> Small update:
> > > >>>>>> I've rebased my branch on trunk and fixed a bunch of issues that
> > > >>>> needed
> > > >>>>>> addressing. Currently, all the tests pass, which is promising,
> but
> > > it
> > > >>>> will
> > > >>>>>> need to undergo some performance testing. I haven't (yet) worked
> > on
> > > >>>>>> removing the `newTransaction()` stuff, but I would expect that,
> > > >>>>>> behaviourally, it should make no difference. The branch is
> > available
> > > >>>> at
> > > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone
> is
> > > >>>>>> interested in taking an early look.
> > > >>>>>>
> > > >>>>>> Regards,
> > > >>>>>> Nick
> > > >>>>>>
> > > >>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <cadonna@apache.org
> >
> > > >>>> wrote:
> > > >>>>>>
> > > >>>>>>> Hi Nick,
> > > >>>>>>>
> > > >>>>>>> 1.
> > > >>>>>>> Yeah, I agree with you. That was actually also my point. I
> > > understood
> > > >>>>>>> that John was proposing the ingestion path as a way to avoid
> the
> > > >>>> early
> > > >>>>>>> commits. Probably, I misinterpreted the intent.
> > > >>>>>>>
> > > >>>>>>> 2.
> > > >>>>>>> I agree with John here, that actually it is public API. My
> > question
> > > >>>> is
> > > >>>>>>> how this usage pattern affects normal processing.
> > > >>>>>>>
> > > >>>>>>> 3.
> > > >>>>>>> My concern is that checking for the size of the transaction
> > buffer
> > > >>>> and
> > > >>>>>>> maybe triggering an early commit affects the whole processing
> of
> > > >>>> Kafka
> > > >>>>>>> Streams. The transactionality of a state store is not confined
> to
> > > the
> > > >>>>>>> state store itself, but spills over and changes the behavior of
> > > other
> > > >>>>>>> parts of the system. I agree with you that it is a decent
> > > >>>> compromise. I
> > > >>>>>>> just wanted to analyse the downsides and list the options to
> > > overcome
> > > >>>>>>> them. I also agree with you that all options seem quite heavy
> > > >>>> compared
> > > >>>>>>> with your KIP. I do not understand what you mean with "less
> > > >>>> predictable
> > > >>>>>>> for users", though.
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> I found the discussions about the alternatives really
> > interesting.
> > > >>>> But I
> > > >>>>>>> also think that your plan sounds good and we should continue
> with
> > > it!
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> Some comments on your reply to my e-mail on June 20th:
> > > >>>>>>>
> > > >>>>>>> 3.
> > > >>>>>>> Ah, now, I understand the reasoning behind putting isolation
> > level
> > > in
> > > >>>>>>> the state store context. Thanks! Should that also be a way to
> > give
> > > >>>> the
> > > >>>>>>> the state store the opportunity to decide whether to turn on
> > > >>>>>>> transactions or not?
> > > >>>>>>> With my comment, I was more concerned about how do you know if
> a
> > > >>>>>>> checkpoint file needs to be written under EOS, if you do not
> > have a
> > > >>>> way
> > > >>>>>>> to know if the state store is transactional or not. If a state
> > > store
> > > >>>> is
> > > >>>>>>> transactional, the checkpoint file can be written during normal
> > > >>>>>>> processing under EOS. If the state store is not transactional,
> > the
> > > >>>>>>> checkpoint file must not be written under EOS.
> > > >>>>>>>
> > > >>>>>>> 7.
> > > >>>>>>> My point was about not only considering the bytes in memory in
> > > config
> > > >>>>>>> statestore.uncommitted.max.bytes, but also bytes that might be
> > > >>>> spilled
> > > >>>>>>> on disk. Basically, I was wondering whether you should remove
> the
> > > >>>>>>> "memory" in "Maximum number of memory bytes to be used to
> > > >>>>>>> buffer uncommitted state-store records." My thinking was that
> > even
> > > >>>> if a
> > > >>>>>>> state store spills uncommitted bytes to disk, limiting the
> > overall
> > > >>>> bytes
> > > >>>>>>> might make sense. Thinking about it again and considering the
> > > recent
> > > >>>>>>> discussions, it does not make too much sense anymore.
> > > >>>>>>> I like the name statestore.transaction.buffer.max.bytes that
> you
> > > >>>> proposed.
> > > >>>>>>>
> > > >>>>>>> 8.
> > > >>>>>>> A high-level description (without implementation details) of
> how
> > > >>>> Kafka
> > > >>>>>>> Streams will manage the commit of changelog transactions, state
> > > store
> > > >>>>>>> transactions and checkpointing would be great. Would be great
> if
> > > you
> > > >>>>>>> could also add some sentences about the behavior in case of a
> > > >>>> failure.
> > > >>>>>>> For instance how does a transactional state store recover
> after a
> > > >>>>>>> failure or what happens with the transaction buffer, etc. (that
> > is
> > > >>>> what
> > > >>>>>>> I meant by "fail-over" in point 9.)
> > > >>>>>>>
> > > >>>>>>> Best,
> > > >>>>>>> Bruno
> > > >>>>>>>
> > > >>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > > >>>>>>>> Hi Bruno,
> > > >>>>>>>>
> > > >>>>>>>> 1.
> > > >>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
> > > >>>> transactions
> > > >>>>>>>> have, whereby exceeding (or likely to exceed) configured
> memory
> > > >>>> needs to
> > > >>>>>>>> trigger an early commit?
> > > >>>>>>>>
> > > >>>>>>>> 2.
> > > >>>>>>>> This is one of my big concerns. Ultimately, any approach based
> > on
> > > >>>>>>> cracking
> > > >>>>>>>> open RocksDB internals and using it in ways it's not really
> > > designed
> > > >>>>>>> for is
> > > >>>>>>>> likely to have some unforseen performance or consistency
> issues.
> > > >>>>>>>>
> > > >>>>>>>> 3.
> > > >>>>>>>> What's your motivation for removing these early commits? While
> > not
> > > >>>>>>> ideal, I
> > > >>>>>>>> think they're a decent compromise to ensure consistency whilst
> > > >>>>>>> maintaining
> > > >>>>>>>> good and predictable performance.
> > > >>>>>>>> All 3 of your suggested ideas seem *very* complicated, and
> might
> > > >>>>>>> actually
> > > >>>>>>>> make behaviour less predictable for users as a consequence.
> > > >>>>>>>>
> > > >>>>>>>> I'm a bit concerned that the scope of this KIP is growing a
> bit
> > > out
> > > >>>> of
> > > >>>>>>>> control. While it's good to discuss ideas for future
> > > improvements, I
> > > >>>>>>> think
> > > >>>>>>>> it's important to narrow the scope down to a design that
> > achieves
> > > >>>> the
> > > >>>>>>> most
> > > >>>>>>>> pressing objectives (constant sized restorations during dirty
> > > >>>>>>>> close/unexpected errors). Any design that this KIP produces
> can
> > > >>>>>>> ultimately
> > > >>>>>>>> be changed in the future, especially if the bulk of it is
> > internal
> > > >>>>>>>> behaviour.
> > > >>>>>>>>
> > > >>>>>>>> I'm going to spend some time next week trying to re-work the
> > > >>>> original
> > > >>>>>>>> WriteBatchWithIndex design to remove the newTransaction()
> > method,
> > > >>>> such
> > > >>>>>>> that
> > > >>>>>>>> it's just an implementation detail of RocksDBStore. That way,
> if
> > > we
> > > >>>>>>> want to
> > > >>>>>>>> replace WBWI with something in the future, like the SST file
> > > >>>> management
> > > >>>>>>>> outlined by John, then we can do so with little/no API
> changes.
> > > >>>>>>>>
> > > >>>>>>>> Regards,
> > > >>>>>>>>
> > > >>>>>>>> Nick
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>
> > > >>>>
> > > >>>
> > > >
> > >
> >
>

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

Posted by Colt McNealy <co...@littlehorse.io>.
Howdy folks,

First I wanted to say fantastic work and thank you to Nick. I built your
branch (https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0) and did
some testing on our Streams app with Kafka 3.5.0, your `kip-892-3.5.0`
branch, and your `kip-892-3.5.0` branch built with Speedb OSS 2.3.0.1. And
it worked! Including the global store (we don't have any segmented stores,
unfortunately).

The test I ran involved running 3,000 workflows with 100 tasks each, and
roughly 650MB state total.

With Streams 3.5.0, I indeed verified that unclean shutdown caused a fresh
restore from scratch. I also benchmarked my application at:
- Running the benchmark took 211 seconds
- 1,421 tasks per second on one partition
- 8 seconds to restore the state (650MB or so)

With KIP 892, I verified that unclean shutdown does not cause a fresh
restore (!!!!). I got the following benchmark results:
- Benchmark took 216 seconds
- 1,401 tasks per second on one partition
- 11 seconds to restore the state

I ran the restorations many times to ensure that there was no rounding
error or noise; the results were remarkably consistent. Additionally, I ran
the restorations with KIP-892 built with Speedb OSS. The restoration time
consistently came out as 10 seconds, which was an improvement from the 11
seconds observed with RocksDB + KIP-892.

My application is bottlenecked mostly by serialization and deserialization,
so improving the performance of the state store doesn't really impact our
throughput that much. And the processing performance (benchmark time,
tasks/second) are pretty close in KIP-892 vs Streams 3.5.0. However, at
larger state store sizes, RocksDB performance begins to degrade, so that
might not be true once we pass 20GB per partition.

-- QUESTION: Because we observed a significant (30% or so) and reproducible
slowdown during restoration, it seems like KIP-892 uses the checkpointing
behavior during restoration as well? If so, I would argue that this might
not be necessary, because everything we write is already committed, so we
don't need to change the behavior during restoration or standby tasks.
Perhaps we could write the offsets to RocksDB on every batch (or even every
5 seconds or so).

-- Note: This was a very small-scale test, with <1GB of state (as I didn't
have time to spend hours building up state). In the past I have noted that
RocksDB performance degrades significantly after 25GB of state in one
store. Future work involves determining the performance impact of KIP-892
relative to trunk at larger scale, since it's possible that the relative
behaviors are far different (i.e. relative to trunk, 892's processing and
restoration throughput might be much better or much worse).

-- Note: For those who want to replicate the tests, you can find the branch
of our streams app here:
https://github.com/littlehorse-enterprises/littlehorse/tree/minor/testing-streams-forks
. The example I ran was `examples/hundred-tasks`, and I ran the server with
`./local-dev/do-server.sh one-partition`. The `STREAMS_TESTS.md` file 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.

Thanks Nick!

Ride well,
Colt McNealy

*Founder, LittleHorse.dev*


On Thu, Aug 24, 2023 at 3:19 AM Nick Telford <ni...@gmail.com> wrote:

> Hi Bruno,
>
> Thanks for taking the time to review the KIP. I'm back from leave now and
> intend to move this forwards as quickly as I can.
>
> Addressing your points:
>
> 1.
> Because flush() is part of the StateStore API, it's exposed to custom
> Processors, which might be making calls to flush(). This was actually the
> case in a few integration tests.
> To maintain as much compatibility as possible, I'd prefer not to make this
> an UnsupportedOperationException, as it will cause previously working
> Processors to start throwing exceptions at runtime.
> I agree that it doesn't make sense for it to proxy commit(), though, as
> that would cause it to violate the "StateStores commit only when the Task
> commits" rule.
> Instead, I think we should make this a no-op. That way, existing user
> Processors will continue to work as-before, without violation of store
> consistency that would be caused by premature flush/commit of StateStore
> data to disk.
> What do you think?
>
> 2.
> As stated in the JavaDoc, when a StateStore implementation is
> transactional, but is unable to estimate the uncommitted memory usage, the
> method will return -1.
> The intention here is to permit third-party implementations that may not be
> able to estimate memory usage.
>
> Yes, it will be 0 when nothing has been written to the store yet. I thought
> that was implied by "This method will return an approximation of the memory
> would be freed by the next call to {@link #commit(Map)}" and "@return The
> approximate size of all records awaiting {@link #commit(Map)}", however, I
> can add it explicitly to the JavaDoc if you think this is unclear?
>
> 3.
> I realise this is probably the most contentious point in my design, and I'm
> open to changing it if I'm unable to convince you of the benefits.
> Nevertheless, here's my argument:
> The Interactive Query (IQ) API(s) are directly provided StateStores to
> query, and it may be important for users to programmatically know which
> mode the StateStore is operating under. If we simply provide an
> "eosEnabled" boolean (as used throughout the internal streams engine), or
> similar, then users will need to understand the operation and consequences
> of each available processing mode and how it pertains to their StateStore.
>
> Interactive Query users aren't the only people that care about the
> processing.mode/IsolationLevel of a StateStore: implementers of custom
> StateStores also need to understand the behaviour expected of their
> implementation. KIP-892 introduces some assumptions into the Streams Engine
> about how StateStores operate under each processing mode, and it's
> important that custom implementations adhere to those assumptions in order
> to maintain the consistency guarantees.
>
> IsolationLevels provide a high-level contract on the behaviour of the
> StateStore: a user knows that under READ_COMMITTED, they will see writes
> only after the Task has committed, and under READ_UNCOMMITTED they will see
> writes immediately. No understanding of the details of each processing.mode
> is required, either for IQ users or StateStore implementers.
>
> An argument can be made that these contractual guarantees can simply be
> documented for the processing.mode (i.e. that exactly-once and
> exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves like
> READ_UNCOMMITTED), but there are several small issues with this I'd prefer
> to avoid:
>
>    - Where would we document these contracts, in a way that is difficult
>    for users/implementers to miss/ignore?
>    - It's not clear to users that the processing mode is communicating
>    an expectation of read isolation, unless they read the documentation.
> Users
>    rarely consult documentation unless they feel they need to, so it's
> likely
>    this detail would get missed by many users.
>    - It tightly couples processing modes to read isolation. Adding new
>    processing modes, or changing the read isolation of existing processing
>    modes would be difficult/impossible.
>
> Ultimately, the cost of introducing IsolationLevels is just a single
> method, since we re-use the existing IsolationLevel enum from Kafka. This
> gives us a clear place to document the contractual guarantees expected
> of/provided by StateStores, that is accessible both by the StateStore
> itself, and by IQ users.
>
> (Writing this I've just realised that the StateStore and IQ APIs actually
> don't provide access to StateStoreContext that IQ users would have direct
> access to... Perhaps StateStore should expose isolationLevel() itself too?)
>
> 4.
> Yeah, I'm not comfortable renaming the metrics in-place either, as it's a
> backwards incompatible change. My concern is that, if we leave the existing
> "flush" metrics in place, they will be confusing to users. Right now,
> "flush" metrics record explicit flushes to disk, but under KIP-892, even a
> commit() will not explicitly flush data to disk - RocksDB will decide on
> when to flush memtables to disk itself.
>
> If we keep the existing "flush" metrics, we'd have two options, which both
> seem pretty bad to me:
>
>    1. Have them record calls to commit(), which would be misleading, as
>    data is no longer explicitly "flushed" to disk by this call.
>    2. Have them record nothing at all, which is equivalent to removing the
>    metrics, except that users will see the metric still exists and so
> assume
>    that the metric is correct, and that there's a problem with their system
>    when there isn't.
>
> I agree that removing them is also a bad solution, and I'd like some
> guidance on the best path forward here.
>
> 5.
> Position files are updated on every write to a StateStore. Since our writes
> are now buffered until commit(), we can't update the Position file until
> commit() has been called, otherwise it would be inconsistent with the data
> in the event of a rollback. Consequently, we need to manage these offsets
> the same way we manage the checkpoint offsets, and ensure they're only
> written on commit().
>
> 6.
> Agreed, although I'm not exactly sure yet what tests to write. How explicit
> do we need to be here in the KIP?
>
> As for upgrade/downgrade: upgrade is designed to be seamless, and we should
> definitely add some tests around that. Downgrade, it transpires, isn't
> currently possible, as the extra column family for offset storage is
> incompatible with the pre-KIP-892 implementation: when you open a RocksDB
> database, you must open all available column families or receive an error.
> What currently happens on downgrade is that it attempts to open the store,
> throws an error about the offsets column family not being opened, which
> triggers a wipe and rebuild of the Task. Given that downgrades should be
> uncommon, I think this is acceptable behaviour, as the end-state is
> consistent, even if it results in an undesirable state restore.
>
> Should I document the upgrade/downgrade behaviour explicitly in the KIP?
>
> --
>
> Regards,
> Nick
>
>
> On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org> wrote:
>
> > Hi Nick!
> >
> > Thanks for the updates!
> >
> > 1.
> > Why does StateStore#flush() default to
> > StateStore#commit(Collections.emptyMap())?
> > Since calls to flush() will not exist anymore after this KIP is
> > released, I would rather throw an unsupported operation exception by
> > default.
> >
> >
> > 2.
> > When would a state store return -1 from
> > StateStore#approximateNumUncommittedBytes() while being transactional?
> >
> > Wouldn't StateStore#approximateNumUncommittedBytes() also return 0 if
> > the state store is transactional but nothing has been written to the
> > state store yet?
> >
> >
> > 3.
> > Sorry for bringing this up again. Does this KIP really need to introduce
> > StateStoreContext#isolationLevel()? StateStoreContext has already
> > appConfigs() which basically exposes the same information, i.e., if EOS
> > is enabled or not.
> > In one of your previous e-mails you wrote:
> >
> > "My idea was to try to keep the StateStore interface as loosely coupled
> > from the Streams engine as possible, to give implementers more freedom,
> > and reduce the amount of internal knowledge required."
> >
> > While I understand the intent, I doubt that it decreases the coupling of
> > a StateStore interface and the Streams engine. READ_COMMITTED only
> > applies to IQ but not to reads by processors. Thus, implementers need to
> > understand how Streams accesses the state stores.
> >
> > I would like to hear what others think about this.
> >
> >
> > 4.
> > Great exposing new metrics for transactional state stores! However, I
> > would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides unit
> > tests. Additionally, we probably need integration or system tests to
> > verify that upgrades and downgrades between transactional and
> > non-transactional state stores work as expected.
> >
> >
> > Best,
> > Bruno
> >
> >
> >
> >
> >
> > On 7/21/23 10:34 PM, Nick Telford wrote:
> > > One more thing: I noted John's suggestion in the KIP, under "Rejected
> > > Alternatives". I still think it's an idea worth pursuing, but I believe
> > > that it's out of the scope of this KIP, because it solves a different
> set
> > > of problems to this KIP, and the scope of this one has already grown
> > quite
> > > large!
> > >
> > > On Fri, 21 Jul 2023 at 21:33, Nick Telford <ni...@gmail.com>
> > wrote:
> > >
> > >> Hi everyone,
> > >>
> > >> I've updated the KIP (
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > )
> > >> with the latest changes; mostly bringing back "Atomic Checkpointing"
> > (for
> > >> what feels like the 10th time!). I think the one thing missing is some
> > >> changes to metrics (notably the store "flush" metrics will need to be
> > >> renamed to "commit").
> > >>
> > >> The reason I brought back Atomic Checkpointing was to decouple store
> > flush
> > >> from store commit. This is important, because with Transactional
> > >> StateStores, we now need to call "flush" on *every* Task commit, and
> not
> > >> just when the StateStore is closing, otherwise our transaction buffer
> > will
> > >> never be written and persisted, instead growing unbounded! I
> > experimented
> > >> with some simple solutions, like forcing a store flush whenever the
> > >> transaction buffer was likely to exceed its configured size, but this
> > was
> > >> brittle: it prevented the transaction buffer from being configured to
> be
> > >> unbounded, and it still would have required explicit flushes of
> RocksDB,
> > >> yielding sub-optimal performance and memory utilization.
> > >>
> > >> I deemed Atomic Checkpointing to be the "right" way to resolve this
> > >> problem. By ensuring that the changelog offsets that correspond to the
> > most
> > >> recently written records are always atomically written to the
> StateStore
> > >> (by writing them to the same transaction buffer), we can avoid
> forcibly
> > >> flushing the RocksDB memtables to disk, letting RocksDB flush them
> only
> > >> when necessary, without losing any of our consistency guarantees. See
> > the
> > >> updated KIP for more info.
> > >>
> > >> I have fully implemented these changes, although I'm still not
> entirely
> > >> happy with the implementation for segmented StateStores, so I plan to
> > >> refactor that. Despite that, all tests pass. If you'd like to try out
> or
> > >> review this highly experimental and incomplete branch, it's available
> > here:
> > >> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note: it's
> > built
> > >> against Kafka 3.5.0 so that I had a stable base to build and test it
> on,
> > >> and to enable easy apples-to-apples comparisons in a live
> environment. I
> > >> plan to rebase it against trunk once it's nearer completion and has
> been
> > >> proven on our main application.
> > >>
> > >> I would really appreciate help in reviewing and testing:
> > >> - Segmented (Versioned, Session and Window) stores
> > >> - Global stores
> > >>
> > >> As I do not currently use either of these, so my primary test
> > environment
> > >> doesn't test these areas.
> > >>
> > >> I'm going on Parental Leave starting next week for a few weeks, so
> will
> > >> not have time to move this forward until late August. That said, your
> > >> feedback is welcome and appreciated, I just won't be able to respond
> as
> > >> quickly as usual.
> > >>
> > >> Regards,
> > >> Nick
> > >>
> > >> On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com>
> > wrote:
> > >>
> > >>> Hi Bruno
> > >>>
> > >>> Yes, that's correct, although the impact on IQ is not something I had
> > >>> considered.
> > >>>
> > >>> What about atomically updating the state store from the transaction
> > >>>> buffer every commit interval and writing the checkpoint (thus,
> > flushing
> > >>>> the memtable) every configured amount of data and/or number of
> commit
> > >>>> intervals?
> > >>>>
> > >>>
> > >>> I'm not quite sure I follow. Are you suggesting that we add an
> > additional
> > >>> config for the max number of commit intervals between checkpoints?
> That
> > >>> way, we would checkpoint *either* when the transaction buffers are
> > nearly
> > >>> full, *OR* whenever a certain number of commit intervals have
> elapsed,
> > >>> whichever comes first?
> > >>>
> > >>> That certainly seems reasonable, although this re-ignites an earlier
> > >>> debate about whether a config should be measured in "number of commit
> > >>> intervals", instead of just an absolute time.
> > >>>
> > >>> FWIW, I realised that this issue is the reason I was pursuing the
> > Atomic
> > >>> Checkpoints, as it de-couples memtable flush from checkpointing,
> which
> > >>> enables us to just checkpoint on every commit without any performance
> > >>> impact. Atomic Checkpointing is definitely the "best" solution, but
> > I'm not
> > >>> sure if this is enough to bring it back into this KIP.
> > >>>
> > >>> I'm currently working on moving all the transactional logic directly
> > into
> > >>> RocksDBStore itself, which does away with the
> StateStore#newTransaction
> > >>> method, and reduces the number of new classes introduced,
> significantly
> > >>> reducing the complexity. If it works, and the complexity is
> drastically
> > >>> reduced, I may try bringing back Atomic Checkpoints into this KIP.
> > >>>
> > >>> Regards,
> > >>> Nick
> > >>>
> > >>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org>
> wrote:
> > >>>
> > >>>> Hi Nick,
> > >>>>
> > >>>> Thanks for the insights! Very interesting!
> > >>>>
> > >>>> As far as I understand, you want to atomically update the state
> store
> > >>>> from the transaction buffer, flush the memtable of a state store and
> > >>>> write the checkpoint not after the commit time elapsed but after the
> > >>>> transaction buffer reached a size that would lead to exceeding
> > >>>> statestore.transaction.buffer.max.bytes before the next commit
> > interval
> > >>>> ends.
> > >>>> That means, the Kafka transaction would commit every commit interval
> > but
> > >>>> the state store will only be atomically updated roughly every
> > >>>> statestore.transaction.buffer.max.bytes of data. Also IQ would then
> > only
> > >>>> see new data roughly every statestore.transaction.buffer.max.bytes.
> > >>>> After a failure the state store needs to restore up to
> > >>>> statestore.transaction.buffer.max.bytes.
> > >>>>
> > >>>> Is this correct?
> > >>>>
> > >>>> What about atomically updating the state store from the transaction
> > >>>> buffer every commit interval and writing the checkpoint (thus,
> > flushing
> > >>>> the memtable) every configured amount of data and/or number of
> commit
> > >>>> intervals? In such a way, we would have the same delay for records
> > >>>> appearing in output topics and IQ because both would appear when the
> > >>>> Kafka transaction is committed. However, after a failure the state
> > store
> > >>>> still needs to restore up to statestore.transaction.buffer.max.bytes
> > and
> > >>>> it might restore data that is already in the state store because the
> > >>>> checkpoint lags behind the last stable offset (i.e. the last
> committed
> > >>>> offset) of the changelog topics. Restoring data that is already in
> the
> > >>>> state store is idempotent, so eos should not violated.
> > >>>> This solution needs at least one new config to specify when a
> > checkpoint
> > >>>> should be written.
> > >>>>
> > >>>>
> > >>>>
> > >>>> A small correction to your previous e-mail that does not change
> > anything
> > >>>> you said: Under alos the default commit interval is 30 seconds, not
> > five
> > >>>> seconds.
> > >>>>
> > >>>>
> > >>>> Best,
> > >>>> Bruno
> > >>>>
> > >>>>
> > >>>> On 01.07.23 12:37, Nick Telford wrote:
> > >>>>> Hi everyone,
> > >>>>>
> > >>>>> I've begun performance testing my branch on our staging
> environment,
> > >>>>> putting it through its paces in our non-trivial application. I'm
> > >>>> already
> > >>>>> observing the same increased flush rate that we saw the last time
> we
> > >>>>> attempted to use a version of this KIP, but this time, I think I
> know
> > >>>> why.
> > >>>>>
> > >>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end of
> the
> > >>>> Task
> > >>>>> commit process, has the following behaviour:
> > >>>>>
> > >>>>>      - Under ALOS: checkpoint the state stores. This includes
> > >>>>>      flushing memtables in RocksDB. This is acceptable because the
> > >>>> default
> > >>>>>      commit.interval.ms is 5 seconds, so forcibly flushing
> memtables
> > >>>> every 5
> > >>>>>      seconds is acceptable for most applications.
> > >>>>>      - Under EOS: checkpointing is not done, *unless* it's being
> > >>>> forced, due
> > >>>>>      to e.g. the Task closing or being revoked. This means that
> under
> > >>>> normal
> > >>>>>      processing conditions, the state stores will not be
> > checkpointed,
> > >>>> and will
> > >>>>>      not have memtables flushed at all , unless RocksDB decides to
> > >>>> flush them on
> > >>>>>      its own. Checkpointing stores and force-flushing their
> memtables
> > >>>> is only
> > >>>>>      done when a Task is being closed.
> > >>>>>
> > >>>>> Under EOS, KIP-892 needs to checkpoint stores on at least *some*
> > normal
> > >>>>> Task commits, in order to write the RocksDB transaction buffers to
> > the
> > >>>>> state stores, and to ensure the offsets are synced to disk to
> prevent
> > >>>>> restores from getting out of hand. Consequently, my current
> > >>>> implementation
> > >>>>> calls maybeCheckpoint on *every* Task commit, which is far too
> > >>>> frequent.
> > >>>>> This causes checkpoints every 10,000 records, which is a change in
> > >>>> flush
> > >>>>> behaviour, potentially causing performance problems for some
> > >>>> applications.
> > >>>>>
> > >>>>> I'm looking into possible solutions, and I'm currently leaning
> > towards
> > >>>>> using the statestore.transaction.buffer.max.bytes configuration to
> > >>>>> checkpoint Tasks once we are likely to exceed it. This would
> > >>>> complement the
> > >>>>> existing "early Task commit" functionality that this configuration
> > >>>>> provides, in the following way:
> > >>>>>
> > >>>>>      - Currently, we use statestore.transaction.buffer.max.bytes to
> > >>>> force an
> > >>>>>      early Task commit if processing more records would cause our
> > state
> > >>>> store
> > >>>>>      transactions to exceed the memory assigned to them.
> > >>>>>      - New functionality: when a Task *does* commit, we will not
> > >>>> checkpoint
> > >>>>>      the stores (and hence flush the transaction buffers) unless we
> > >>>> expect to
> > >>>>>      cross the statestore.transaction.buffer.max.bytes threshold
> > before
> > >>>> the next
> > >>>>>      commit
> > >>>>>
> > >>>>> I'm also open to suggestions.
> > >>>>>
> > >>>>> Regards,
> > >>>>> Nick
> > >>>>>
> > >>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <nick.telford@gmail.com
> >
> > >>>> wrote:
> > >>>>>
> > >>>>>> Hi Bruno!
> > >>>>>>
> > >>>>>> 3.
> > >>>>>> By "less predictable for users", I meant in terms of understanding
> > the
> > >>>>>> performance profile under various circumstances. The more complex
> > the
> > >>>>>> solution, the more difficult it would be for users to understand
> the
> > >>>>>> performance they see. For example, spilling records to disk when
> the
> > >>>>>> transaction buffer reaches a threshold would, I expect, reduce
> write
> > >>>>>> throughput. This reduction in write throughput could be
> unexpected,
> > >>>> and
> > >>>>>> potentially difficult to diagnose/understand for users.
> > >>>>>> At the moment, I think the "early commit" concept is relatively
> > >>>>>> straightforward; it's easy to document, and conceptually fairly
> > >>>> obvious to
> > >>>>>> users. We could probably add a metric to make it easier to
> > understand
> > >>>> when
> > >>>>>> it happens though.
> > >>>>>>
> > >>>>>> 3. (the second one)
> > >>>>>> The IsolationLevel is *essentially* an indirect way of telling
> > >>>> StateStores
> > >>>>>> whether they should be transactional. READ_COMMITTED essentially
> > >>>> requires
> > >>>>>> transactions, because it dictates that two threads calling
> > >>>>>> `newTransaction()` should not see writes from the other
> transaction
> > >>>> until
> > >>>>>> they have been committed. With READ_UNCOMMITTED, all bets are off,
> > and
> > >>>>>> stores can allow threads to observe written records at any time,
> > >>>> which is
> > >>>>>> essentially "no transactions". That said, StateStores are free to
> > >>>> implement
> > >>>>>> these guarantees however they can, which is a bit more relaxed
> than
> > >>>>>> dictating "you must use transactions". For example, with RocksDB
> we
> > >>>> would
> > >>>>>> implement these as READ_COMMITTED == WBWI-based "transactions",
> > >>>>>> READ_UNCOMMITTED == direct writes to the database. But with other
> > >>>> storage
> > >>>>>> engines, it might be preferable to *always* use transactions, even
> > >>>> when
> > >>>>>> unnecessary; or there may be storage engines that don't provide
> > >>>>>> transactions, but the isolation guarantees can be met using a
> > >>>> different
> > >>>>>> technique.
> > >>>>>> My idea was to try to keep the StateStore interface as loosely
> > coupled
> > >>>>>> from the Streams engine as possible, to give implementers more
> > >>>> freedom, and
> > >>>>>> reduce the amount of internal knowledge required.
> > >>>>>> That said, I understand that "IsolationLevel" might not be the
> right
> > >>>>>> abstraction, and we can always make it much more explicit if
> > >>>> required, e.g.
> > >>>>>> boolean transactional()
> > >>>>>>
> > >>>>>> 7-8.
> > >>>>>> I can make these changes either later today or tomorrow.
> > >>>>>>
> > >>>>>> Small update:
> > >>>>>> I've rebased my branch on trunk and fixed a bunch of issues that
> > >>>> needed
> > >>>>>> addressing. Currently, all the tests pass, which is promising, but
> > it
> > >>>> will
> > >>>>>> need to undergo some performance testing. I haven't (yet) worked
> on
> > >>>>>> removing the `newTransaction()` stuff, but I would expect that,
> > >>>>>> behaviourally, it should make no difference. The branch is
> available
> > >>>> at
> > >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
> > >>>>>> interested in taking an early look.
> > >>>>>>
> > >>>>>> Regards,
> > >>>>>> Nick
> > >>>>>>
> > >>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org>
> > >>>> wrote:
> > >>>>>>
> > >>>>>>> Hi Nick,
> > >>>>>>>
> > >>>>>>> 1.
> > >>>>>>> Yeah, I agree with you. That was actually also my point. I
> > understood
> > >>>>>>> that John was proposing the ingestion path as a way to avoid the
> > >>>> early
> > >>>>>>> commits. Probably, I misinterpreted the intent.
> > >>>>>>>
> > >>>>>>> 2.
> > >>>>>>> I agree with John here, that actually it is public API. My
> question
> > >>>> is
> > >>>>>>> how this usage pattern affects normal processing.
> > >>>>>>>
> > >>>>>>> 3.
> > >>>>>>> My concern is that checking for the size of the transaction
> buffer
> > >>>> and
> > >>>>>>> maybe triggering an early commit affects the whole processing of
> > >>>> Kafka
> > >>>>>>> Streams. The transactionality of a state store is not confined to
> > the
> > >>>>>>> state store itself, but spills over and changes the behavior of
> > other
> > >>>>>>> parts of the system. I agree with you that it is a decent
> > >>>> compromise. I
> > >>>>>>> just wanted to analyse the downsides and list the options to
> > overcome
> > >>>>>>> them. I also agree with you that all options seem quite heavy
> > >>>> compared
> > >>>>>>> with your KIP. I do not understand what you mean with "less
> > >>>> predictable
> > >>>>>>> for users", though.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> I found the discussions about the alternatives really
> interesting.
> > >>>> But I
> > >>>>>>> also think that your plan sounds good and we should continue with
> > it!
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> Some comments on your reply to my e-mail on June 20th:
> > >>>>>>>
> > >>>>>>> 3.
> > >>>>>>> Ah, now, I understand the reasoning behind putting isolation
> level
> > in
> > >>>>>>> the state store context. Thanks! Should that also be a way to
> give
> > >>>> the
> > >>>>>>> the state store the opportunity to decide whether to turn on
> > >>>>>>> transactions or not?
> > >>>>>>> With my comment, I was more concerned about how do you know if a
> > >>>>>>> checkpoint file needs to be written under EOS, if you do not
> have a
> > >>>> way
> > >>>>>>> to know if the state store is transactional or not. If a state
> > store
> > >>>> is
> > >>>>>>> transactional, the checkpoint file can be written during normal
> > >>>>>>> processing under EOS. If the state store is not transactional,
> the
> > >>>>>>> checkpoint file must not be written under EOS.
> > >>>>>>>
> > >>>>>>> 7.
> > >>>>>>> My point was about not only considering the bytes in memory in
> > config
> > >>>>>>> statestore.uncommitted.max.bytes, but also bytes that might be
> > >>>> spilled
> > >>>>>>> on disk. Basically, I was wondering whether you should remove the
> > >>>>>>> "memory" in "Maximum number of memory bytes to be used to
> > >>>>>>> buffer uncommitted state-store records." My thinking was that
> even
> > >>>> if a
> > >>>>>>> state store spills uncommitted bytes to disk, limiting the
> overall
> > >>>> bytes
> > >>>>>>> might make sense. Thinking about it again and considering the
> > recent
> > >>>>>>> discussions, it does not make too much sense anymore.
> > >>>>>>> I like the name statestore.transaction.buffer.max.bytes that you
> > >>>> proposed.
> > >>>>>>>
> > >>>>>>> 8.
> > >>>>>>> A high-level description (without implementation details) of how
> > >>>> Kafka
> > >>>>>>> Streams will manage the commit of changelog transactions, state
> > store
> > >>>>>>> transactions and checkpointing would be great. Would be great if
> > you
> > >>>>>>> could also add some sentences about the behavior in case of a
> > >>>> failure.
> > >>>>>>> For instance how does a transactional state store recover after a
> > >>>>>>> failure or what happens with the transaction buffer, etc. (that
> is
> > >>>> what
> > >>>>>>> I meant by "fail-over" in point 9.)
> > >>>>>>>
> > >>>>>>> Best,
> > >>>>>>> Bruno
> > >>>>>>>
> > >>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> > >>>>>>>> Hi Bruno,
> > >>>>>>>>
> > >>>>>>>> 1.
> > >>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
> > >>>> transactions
> > >>>>>>>> have, whereby exceeding (or likely to exceed) configured memory
> > >>>> needs to
> > >>>>>>>> trigger an early commit?
> > >>>>>>>>
> > >>>>>>>> 2.
> > >>>>>>>> This is one of my big concerns. Ultimately, any approach based
> on
> > >>>>>>> cracking
> > >>>>>>>> open RocksDB internals and using it in ways it's not really
> > designed
> > >>>>>>> for is
> > >>>>>>>> likely to have some unforseen performance or consistency issues.
> > >>>>>>>>
> > >>>>>>>> 3.
> > >>>>>>>> What's your motivation for removing these early commits? While
> not
> > >>>>>>> ideal, I
> > >>>>>>>> think they're a decent compromise to ensure consistency whilst
> > >>>>>>> maintaining
> > >>>>>>>> good and predictable performance.
> > >>>>>>>> All 3 of your suggested ideas seem *very* complicated, and might
> > >>>>>>> actually
> > >>>>>>>> make behaviour less predictable for users as a consequence.
> > >>>>>>>>
> > >>>>>>>> I'm a bit concerned that the scope of this KIP is growing a bit
> > out
> > >>>> of
> > >>>>>>>> control. While it's good to discuss ideas for future
> > improvements, I
> > >>>>>>> think
> > >>>>>>>> it's important to narrow the scope down to a design that
> achieves
> > >>>> the
> > >>>>>>> most
> > >>>>>>>> pressing objectives (constant sized restorations during dirty
> > >>>>>>>> close/unexpected errors). Any design that this KIP produces can
> > >>>>>>> ultimately
> > >>>>>>>> be changed in the future, especially if the bulk of it is
> internal
> > >>>>>>>> behaviour.
> > >>>>>>>>
> > >>>>>>>> I'm going to spend some time next week trying to re-work the
> > >>>> original
> > >>>>>>>> WriteBatchWithIndex design to remove the newTransaction()
> method,
> > >>>> such
> > >>>>>>> that
> > >>>>>>>> it's just an implementation detail of RocksDBStore. That way, if
> > we
> > >>>>>>> want to
> > >>>>>>>> replace WBWI with something in the future, like the SST file
> > >>>> management
> > >>>>>>>> outlined by John, then we can do so with little/no API changes.
> > >>>>>>>>
> > >>>>>>>> Regards,
> > >>>>>>>>
> > >>>>>>>> Nick
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>
> > >>>
> > >
> >
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Bruno,

Thanks for taking the time to review the KIP. I'm back from leave now and
intend to move this forwards as quickly as I can.

Addressing your points:

1.
Because flush() is part of the StateStore API, it's exposed to custom
Processors, which might be making calls to flush(). This was actually the
case in a few integration tests.
To maintain as much compatibility as possible, I'd prefer not to make this
an UnsupportedOperationException, as it will cause previously working
Processors to start throwing exceptions at runtime.
I agree that it doesn't make sense for it to proxy commit(), though, as
that would cause it to violate the "StateStores commit only when the Task
commits" rule.
Instead, I think we should make this a no-op. That way, existing user
Processors will continue to work as-before, without violation of store
consistency that would be caused by premature flush/commit of StateStore
data to disk.
What do you think?

2.
As stated in the JavaDoc, when a StateStore implementation is
transactional, but is unable to estimate the uncommitted memory usage, the
method will return -1.
The intention here is to permit third-party implementations that may not be
able to estimate memory usage.

Yes, it will be 0 when nothing has been written to the store yet. I thought
that was implied by "This method will return an approximation of the memory
would be freed by the next call to {@link #commit(Map)}" and "@return The
approximate size of all records awaiting {@link #commit(Map)}", however, I
can add it explicitly to the JavaDoc if you think this is unclear?

3.
I realise this is probably the most contentious point in my design, and I'm
open to changing it if I'm unable to convince you of the benefits.
Nevertheless, here's my argument:
The Interactive Query (IQ) API(s) are directly provided StateStores to
query, and it may be important for users to programmatically know which
mode the StateStore is operating under. If we simply provide an
"eosEnabled" boolean (as used throughout the internal streams engine), or
similar, then users will need to understand the operation and consequences
of each available processing mode and how it pertains to their StateStore.

Interactive Query users aren't the only people that care about the
processing.mode/IsolationLevel of a StateStore: implementers of custom
StateStores also need to understand the behaviour expected of their
implementation. KIP-892 introduces some assumptions into the Streams Engine
about how StateStores operate under each processing mode, and it's
important that custom implementations adhere to those assumptions in order
to maintain the consistency guarantees.

IsolationLevels provide a high-level contract on the behaviour of the
StateStore: a user knows that under READ_COMMITTED, they will see writes
only after the Task has committed, and under READ_UNCOMMITTED they will see
writes immediately. No understanding of the details of each processing.mode
is required, either for IQ users or StateStore implementers.

An argument can be made that these contractual guarantees can simply be
documented for the processing.mode (i.e. that exactly-once and
exactly-once-v2 behave like READ_COMMITTED and at-least-once behaves like
READ_UNCOMMITTED), but there are several small issues with this I'd prefer
to avoid:

   - Where would we document these contracts, in a way that is difficult
   for users/implementers to miss/ignore?
   - It's not clear to users that the processing mode is communicating
   an expectation of read isolation, unless they read the documentation. Users
   rarely consult documentation unless they feel they need to, so it's likely
   this detail would get missed by many users.
   - It tightly couples processing modes to read isolation. Adding new
   processing modes, or changing the read isolation of existing processing
   modes would be difficult/impossible.

Ultimately, the cost of introducing IsolationLevels is just a single
method, since we re-use the existing IsolationLevel enum from Kafka. This
gives us a clear place to document the contractual guarantees expected
of/provided by StateStores, that is accessible both by the StateStore
itself, and by IQ users.

(Writing this I've just realised that the StateStore and IQ APIs actually
don't provide access to StateStoreContext that IQ users would have direct
access to... Perhaps StateStore should expose isolationLevel() itself too?)

4.
Yeah, I'm not comfortable renaming the metrics in-place either, as it's a
backwards incompatible change. My concern is that, if we leave the existing
"flush" metrics in place, they will be confusing to users. Right now,
"flush" metrics record explicit flushes to disk, but under KIP-892, even a
commit() will not explicitly flush data to disk - RocksDB will decide on
when to flush memtables to disk itself.

If we keep the existing "flush" metrics, we'd have two options, which both
seem pretty bad to me:

   1. Have them record calls to commit(), which would be misleading, as
   data is no longer explicitly "flushed" to disk by this call.
   2. Have them record nothing at all, which is equivalent to removing the
   metrics, except that users will see the metric still exists and so assume
   that the metric is correct, and that there's a problem with their system
   when there isn't.

I agree that removing them is also a bad solution, and I'd like some
guidance on the best path forward here.

5.
Position files are updated on every write to a StateStore. Since our writes
are now buffered until commit(), we can't update the Position file until
commit() has been called, otherwise it would be inconsistent with the data
in the event of a rollback. Consequently, we need to manage these offsets
the same way we manage the checkpoint offsets, and ensure they're only
written on commit().

6.
Agreed, although I'm not exactly sure yet what tests to write. How explicit
do we need to be here in the KIP?

As for upgrade/downgrade: upgrade is designed to be seamless, and we should
definitely add some tests around that. Downgrade, it transpires, isn't
currently possible, as the extra column family for offset storage is
incompatible with the pre-KIP-892 implementation: when you open a RocksDB
database, you must open all available column families or receive an error.
What currently happens on downgrade is that it attempts to open the store,
throws an error about the offsets column family not being opened, which
triggers a wipe and rebuild of the Task. Given that downgrades should be
uncommon, I think this is acceptable behaviour, as the end-state is
consistent, even if it results in an undesirable state restore.

Should I document the upgrade/downgrade behaviour explicitly in the KIP?

--

Regards,
Nick


On Mon, 14 Aug 2023 at 22:31, Bruno Cadonna <ca...@apache.org> wrote:

> Hi Nick!
>
> Thanks for the updates!
>
> 1.
> Why does StateStore#flush() default to
> StateStore#commit(Collections.emptyMap())?
> Since calls to flush() will not exist anymore after this KIP is
> released, I would rather throw an unsupported operation exception by
> default.
>
>
> 2.
> When would a state store return -1 from
> StateStore#approximateNumUncommittedBytes() while being transactional?
>
> Wouldn't StateStore#approximateNumUncommittedBytes() also return 0 if
> the state store is transactional but nothing has been written to the
> state store yet?
>
>
> 3.
> Sorry for bringing this up again. Does this KIP really need to introduce
> StateStoreContext#isolationLevel()? StateStoreContext has already
> appConfigs() which basically exposes the same information, i.e., if EOS
> is enabled or not.
> In one of your previous e-mails you wrote:
>
> "My idea was to try to keep the StateStore interface as loosely coupled
> from the Streams engine as possible, to give implementers more freedom,
> and reduce the amount of internal knowledge required."
>
> While I understand the intent, I doubt that it decreases the coupling of
> a StateStore interface and the Streams engine. READ_COMMITTED only
> applies to IQ but not to reads by processors. Thus, implementers need to
> understand how Streams accesses the state stores.
>
> I would like to hear what others think about this.
>
>
> 4.
> Great exposing new metrics for transactional state stores! However, I
> would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides unit
> tests. Additionally, we probably need integration or system tests to
> verify that upgrades and downgrades between transactional and
> non-transactional state stores work as expected.
>
>
> Best,
> Bruno
>
>
>
>
>
> On 7/21/23 10:34 PM, Nick Telford wrote:
> > One more thing: I noted John's suggestion in the KIP, under "Rejected
> > Alternatives". I still think it's an idea worth pursuing, but I believe
> > that it's out of the scope of this KIP, because it solves a different set
> > of problems to this KIP, and the scope of this one has already grown
> quite
> > large!
> >
> > On Fri, 21 Jul 2023 at 21:33, Nick Telford <ni...@gmail.com>
> wrote:
> >
> >> Hi everyone,
> >>
> >> I've updated the KIP (
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> )
> >> with the latest changes; mostly bringing back "Atomic Checkpointing"
> (for
> >> what feels like the 10th time!). I think the one thing missing is some
> >> changes to metrics (notably the store "flush" metrics will need to be
> >> renamed to "commit").
> >>
> >> The reason I brought back Atomic Checkpointing was to decouple store
> flush
> >> from store commit. This is important, because with Transactional
> >> StateStores, we now need to call "flush" on *every* Task commit, and not
> >> just when the StateStore is closing, otherwise our transaction buffer
> will
> >> never be written and persisted, instead growing unbounded! I
> experimented
> >> with some simple solutions, like forcing a store flush whenever the
> >> transaction buffer was likely to exceed its configured size, but this
> was
> >> brittle: it prevented the transaction buffer from being configured to be
> >> unbounded, and it still would have required explicit flushes of RocksDB,
> >> yielding sub-optimal performance and memory utilization.
> >>
> >> I deemed Atomic Checkpointing to be the "right" way to resolve this
> >> problem. By ensuring that the changelog offsets that correspond to the
> most
> >> recently written records are always atomically written to the StateStore
> >> (by writing them to the same transaction buffer), we can avoid forcibly
> >> flushing the RocksDB memtables to disk, letting RocksDB flush them only
> >> when necessary, without losing any of our consistency guarantees. See
> the
> >> updated KIP for more info.
> >>
> >> I have fully implemented these changes, although I'm still not entirely
> >> happy with the implementation for segmented StateStores, so I plan to
> >> refactor that. Despite that, all tests pass. If you'd like to try out or
> >> review this highly experimental and incomplete branch, it's available
> here:
> >> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note: it's
> built
> >> against Kafka 3.5.0 so that I had a stable base to build and test it on,
> >> and to enable easy apples-to-apples comparisons in a live environment. I
> >> plan to rebase it against trunk once it's nearer completion and has been
> >> proven on our main application.
> >>
> >> I would really appreciate help in reviewing and testing:
> >> - Segmented (Versioned, Session and Window) stores
> >> - Global stores
> >>
> >> As I do not currently use either of these, so my primary test
> environment
> >> doesn't test these areas.
> >>
> >> I'm going on Parental Leave starting next week for a few weeks, so will
> >> not have time to move this forward until late August. That said, your
> >> feedback is welcome and appreciated, I just won't be able to respond as
> >> quickly as usual.
> >>
> >> Regards,
> >> Nick
> >>
> >> On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com>
> wrote:
> >>
> >>> Hi Bruno
> >>>
> >>> Yes, that's correct, although the impact on IQ is not something I had
> >>> considered.
> >>>
> >>> What about atomically updating the state store from the transaction
> >>>> buffer every commit interval and writing the checkpoint (thus,
> flushing
> >>>> the memtable) every configured amount of data and/or number of commit
> >>>> intervals?
> >>>>
> >>>
> >>> I'm not quite sure I follow. Are you suggesting that we add an
> additional
> >>> config for the max number of commit intervals between checkpoints? That
> >>> way, we would checkpoint *either* when the transaction buffers are
> nearly
> >>> full, *OR* whenever a certain number of commit intervals have elapsed,
> >>> whichever comes first?
> >>>
> >>> That certainly seems reasonable, although this re-ignites an earlier
> >>> debate about whether a config should be measured in "number of commit
> >>> intervals", instead of just an absolute time.
> >>>
> >>> FWIW, I realised that this issue is the reason I was pursuing the
> Atomic
> >>> Checkpoints, as it de-couples memtable flush from checkpointing, which
> >>> enables us to just checkpoint on every commit without any performance
> >>> impact. Atomic Checkpointing is definitely the "best" solution, but
> I'm not
> >>> sure if this is enough to bring it back into this KIP.
> >>>
> >>> I'm currently working on moving all the transactional logic directly
> into
> >>> RocksDBStore itself, which does away with the StateStore#newTransaction
> >>> method, and reduces the number of new classes introduced, significantly
> >>> reducing the complexity. If it works, and the complexity is drastically
> >>> reduced, I may try bringing back Atomic Checkpoints into this KIP.
> >>>
> >>> Regards,
> >>> Nick
> >>>
> >>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org> wrote:
> >>>
> >>>> Hi Nick,
> >>>>
> >>>> Thanks for the insights! Very interesting!
> >>>>
> >>>> As far as I understand, you want to atomically update the state store
> >>>> from the transaction buffer, flush the memtable of a state store and
> >>>> write the checkpoint not after the commit time elapsed but after the
> >>>> transaction buffer reached a size that would lead to exceeding
> >>>> statestore.transaction.buffer.max.bytes before the next commit
> interval
> >>>> ends.
> >>>> That means, the Kafka transaction would commit every commit interval
> but
> >>>> the state store will only be atomically updated roughly every
> >>>> statestore.transaction.buffer.max.bytes of data. Also IQ would then
> only
> >>>> see new data roughly every statestore.transaction.buffer.max.bytes.
> >>>> After a failure the state store needs to restore up to
> >>>> statestore.transaction.buffer.max.bytes.
> >>>>
> >>>> Is this correct?
> >>>>
> >>>> What about atomically updating the state store from the transaction
> >>>> buffer every commit interval and writing the checkpoint (thus,
> flushing
> >>>> the memtable) every configured amount of data and/or number of commit
> >>>> intervals? In such a way, we would have the same delay for records
> >>>> appearing in output topics and IQ because both would appear when the
> >>>> Kafka transaction is committed. However, after a failure the state
> store
> >>>> still needs to restore up to statestore.transaction.buffer.max.bytes
> and
> >>>> it might restore data that is already in the state store because the
> >>>> checkpoint lags behind the last stable offset (i.e. the last committed
> >>>> offset) of the changelog topics. Restoring data that is already in the
> >>>> state store is idempotent, so eos should not violated.
> >>>> This solution needs at least one new config to specify when a
> checkpoint
> >>>> should be written.
> >>>>
> >>>>
> >>>>
> >>>> A small correction to your previous e-mail that does not change
> anything
> >>>> you said: Under alos the default commit interval is 30 seconds, not
> five
> >>>> seconds.
> >>>>
> >>>>
> >>>> Best,
> >>>> Bruno
> >>>>
> >>>>
> >>>> On 01.07.23 12:37, Nick Telford wrote:
> >>>>> Hi everyone,
> >>>>>
> >>>>> I've begun performance testing my branch on our staging environment,
> >>>>> putting it through its paces in our non-trivial application. I'm
> >>>> already
> >>>>> observing the same increased flush rate that we saw the last time we
> >>>>> attempted to use a version of this KIP, but this time, I think I know
> >>>> why.
> >>>>>
> >>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end of the
> >>>> Task
> >>>>> commit process, has the following behaviour:
> >>>>>
> >>>>>      - Under ALOS: checkpoint the state stores. This includes
> >>>>>      flushing memtables in RocksDB. This is acceptable because the
> >>>> default
> >>>>>      commit.interval.ms is 5 seconds, so forcibly flushing memtables
> >>>> every 5
> >>>>>      seconds is acceptable for most applications.
> >>>>>      - Under EOS: checkpointing is not done, *unless* it's being
> >>>> forced, due
> >>>>>      to e.g. the Task closing or being revoked. This means that under
> >>>> normal
> >>>>>      processing conditions, the state stores will not be
> checkpointed,
> >>>> and will
> >>>>>      not have memtables flushed at all , unless RocksDB decides to
> >>>> flush them on
> >>>>>      its own. Checkpointing stores and force-flushing their memtables
> >>>> is only
> >>>>>      done when a Task is being closed.
> >>>>>
> >>>>> Under EOS, KIP-892 needs to checkpoint stores on at least *some*
> normal
> >>>>> Task commits, in order to write the RocksDB transaction buffers to
> the
> >>>>> state stores, and to ensure the offsets are synced to disk to prevent
> >>>>> restores from getting out of hand. Consequently, my current
> >>>> implementation
> >>>>> calls maybeCheckpoint on *every* Task commit, which is far too
> >>>> frequent.
> >>>>> This causes checkpoints every 10,000 records, which is a change in
> >>>> flush
> >>>>> behaviour, potentially causing performance problems for some
> >>>> applications.
> >>>>>
> >>>>> I'm looking into possible solutions, and I'm currently leaning
> towards
> >>>>> using the statestore.transaction.buffer.max.bytes configuration to
> >>>>> checkpoint Tasks once we are likely to exceed it. This would
> >>>> complement the
> >>>>> existing "early Task commit" functionality that this configuration
> >>>>> provides, in the following way:
> >>>>>
> >>>>>      - Currently, we use statestore.transaction.buffer.max.bytes to
> >>>> force an
> >>>>>      early Task commit if processing more records would cause our
> state
> >>>> store
> >>>>>      transactions to exceed the memory assigned to them.
> >>>>>      - New functionality: when a Task *does* commit, we will not
> >>>> checkpoint
> >>>>>      the stores (and hence flush the transaction buffers) unless we
> >>>> expect to
> >>>>>      cross the statestore.transaction.buffer.max.bytes threshold
> before
> >>>> the next
> >>>>>      commit
> >>>>>
> >>>>> I'm also open to suggestions.
> >>>>>
> >>>>> Regards,
> >>>>> Nick
> >>>>>
> >>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <ni...@gmail.com>
> >>>> wrote:
> >>>>>
> >>>>>> Hi Bruno!
> >>>>>>
> >>>>>> 3.
> >>>>>> By "less predictable for users", I meant in terms of understanding
> the
> >>>>>> performance profile under various circumstances. The more complex
> the
> >>>>>> solution, the more difficult it would be for users to understand the
> >>>>>> performance they see. For example, spilling records to disk when the
> >>>>>> transaction buffer reaches a threshold would, I expect, reduce write
> >>>>>> throughput. This reduction in write throughput could be unexpected,
> >>>> and
> >>>>>> potentially difficult to diagnose/understand for users.
> >>>>>> At the moment, I think the "early commit" concept is relatively
> >>>>>> straightforward; it's easy to document, and conceptually fairly
> >>>> obvious to
> >>>>>> users. We could probably add a metric to make it easier to
> understand
> >>>> when
> >>>>>> it happens though.
> >>>>>>
> >>>>>> 3. (the second one)
> >>>>>> The IsolationLevel is *essentially* an indirect way of telling
> >>>> StateStores
> >>>>>> whether they should be transactional. READ_COMMITTED essentially
> >>>> requires
> >>>>>> transactions, because it dictates that two threads calling
> >>>>>> `newTransaction()` should not see writes from the other transaction
> >>>> until
> >>>>>> they have been committed. With READ_UNCOMMITTED, all bets are off,
> and
> >>>>>> stores can allow threads to observe written records at any time,
> >>>> which is
> >>>>>> essentially "no transactions". That said, StateStores are free to
> >>>> implement
> >>>>>> these guarantees however they can, which is a bit more relaxed than
> >>>>>> dictating "you must use transactions". For example, with RocksDB we
> >>>> would
> >>>>>> implement these as READ_COMMITTED == WBWI-based "transactions",
> >>>>>> READ_UNCOMMITTED == direct writes to the database. But with other
> >>>> storage
> >>>>>> engines, it might be preferable to *always* use transactions, even
> >>>> when
> >>>>>> unnecessary; or there may be storage engines that don't provide
> >>>>>> transactions, but the isolation guarantees can be met using a
> >>>> different
> >>>>>> technique.
> >>>>>> My idea was to try to keep the StateStore interface as loosely
> coupled
> >>>>>> from the Streams engine as possible, to give implementers more
> >>>> freedom, and
> >>>>>> reduce the amount of internal knowledge required.
> >>>>>> That said, I understand that "IsolationLevel" might not be the right
> >>>>>> abstraction, and we can always make it much more explicit if
> >>>> required, e.g.
> >>>>>> boolean transactional()
> >>>>>>
> >>>>>> 7-8.
> >>>>>> I can make these changes either later today or tomorrow.
> >>>>>>
> >>>>>> Small update:
> >>>>>> I've rebased my branch on trunk and fixed a bunch of issues that
> >>>> needed
> >>>>>> addressing. Currently, all the tests pass, which is promising, but
> it
> >>>> will
> >>>>>> need to undergo some performance testing. I haven't (yet) worked on
> >>>>>> removing the `newTransaction()` stuff, but I would expect that,
> >>>>>> behaviourally, it should make no difference. The branch is available
> >>>> at
> >>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
> >>>>>> interested in taking an early look.
> >>>>>>
> >>>>>> Regards,
> >>>>>> Nick
> >>>>>>
> >>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org>
> >>>> wrote:
> >>>>>>
> >>>>>>> Hi Nick,
> >>>>>>>
> >>>>>>> 1.
> >>>>>>> Yeah, I agree with you. That was actually also my point. I
> understood
> >>>>>>> that John was proposing the ingestion path as a way to avoid the
> >>>> early
> >>>>>>> commits. Probably, I misinterpreted the intent.
> >>>>>>>
> >>>>>>> 2.
> >>>>>>> I agree with John here, that actually it is public API. My question
> >>>> is
> >>>>>>> how this usage pattern affects normal processing.
> >>>>>>>
> >>>>>>> 3.
> >>>>>>> My concern is that checking for the size of the transaction buffer
> >>>> and
> >>>>>>> maybe triggering an early commit affects the whole processing of
> >>>> Kafka
> >>>>>>> Streams. The transactionality of a state store is not confined to
> the
> >>>>>>> state store itself, but spills over and changes the behavior of
> other
> >>>>>>> parts of the system. I agree with you that it is a decent
> >>>> compromise. I
> >>>>>>> just wanted to analyse the downsides and list the options to
> overcome
> >>>>>>> them. I also agree with you that all options seem quite heavy
> >>>> compared
> >>>>>>> with your KIP. I do not understand what you mean with "less
> >>>> predictable
> >>>>>>> for users", though.
> >>>>>>>
> >>>>>>>
> >>>>>>> I found the discussions about the alternatives really interesting.
> >>>> But I
> >>>>>>> also think that your plan sounds good and we should continue with
> it!
> >>>>>>>
> >>>>>>>
> >>>>>>> Some comments on your reply to my e-mail on June 20th:
> >>>>>>>
> >>>>>>> 3.
> >>>>>>> Ah, now, I understand the reasoning behind putting isolation level
> in
> >>>>>>> the state store context. Thanks! Should that also be a way to give
> >>>> the
> >>>>>>> the state store the opportunity to decide whether to turn on
> >>>>>>> transactions or not?
> >>>>>>> With my comment, I was more concerned about how do you know if a
> >>>>>>> checkpoint file needs to be written under EOS, if you do not have a
> >>>> way
> >>>>>>> to know if the state store is transactional or not. If a state
> store
> >>>> is
> >>>>>>> transactional, the checkpoint file can be written during normal
> >>>>>>> processing under EOS. If the state store is not transactional, the
> >>>>>>> checkpoint file must not be written under EOS.
> >>>>>>>
> >>>>>>> 7.
> >>>>>>> My point was about not only considering the bytes in memory in
> config
> >>>>>>> statestore.uncommitted.max.bytes, but also bytes that might be
> >>>> spilled
> >>>>>>> on disk. Basically, I was wondering whether you should remove the
> >>>>>>> "memory" in "Maximum number of memory bytes to be used to
> >>>>>>> buffer uncommitted state-store records." My thinking was that even
> >>>> if a
> >>>>>>> state store spills uncommitted bytes to disk, limiting the overall
> >>>> bytes
> >>>>>>> might make sense. Thinking about it again and considering the
> recent
> >>>>>>> discussions, it does not make too much sense anymore.
> >>>>>>> I like the name statestore.transaction.buffer.max.bytes that you
> >>>> proposed.
> >>>>>>>
> >>>>>>> 8.
> >>>>>>> A high-level description (without implementation details) of how
> >>>> Kafka
> >>>>>>> Streams will manage the commit of changelog transactions, state
> store
> >>>>>>> transactions and checkpointing would be great. Would be great if
> you
> >>>>>>> could also add some sentences about the behavior in case of a
> >>>> failure.
> >>>>>>> For instance how does a transactional state store recover after a
> >>>>>>> failure or what happens with the transaction buffer, etc. (that is
> >>>> what
> >>>>>>> I meant by "fail-over" in point 9.)
> >>>>>>>
> >>>>>>> Best,
> >>>>>>> Bruno
> >>>>>>>
> >>>>>>> On 21.06.23 18:50, Nick Telford wrote:
> >>>>>>>> Hi Bruno,
> >>>>>>>>
> >>>>>>>> 1.
> >>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
> >>>> transactions
> >>>>>>>> have, whereby exceeding (or likely to exceed) configured memory
> >>>> needs to
> >>>>>>>> trigger an early commit?
> >>>>>>>>
> >>>>>>>> 2.
> >>>>>>>> This is one of my big concerns. Ultimately, any approach based on
> >>>>>>> cracking
> >>>>>>>> open RocksDB internals and using it in ways it's not really
> designed
> >>>>>>> for is
> >>>>>>>> likely to have some unforseen performance or consistency issues.
> >>>>>>>>
> >>>>>>>> 3.
> >>>>>>>> What's your motivation for removing these early commits? While not
> >>>>>>> ideal, I
> >>>>>>>> think they're a decent compromise to ensure consistency whilst
> >>>>>>> maintaining
> >>>>>>>> good and predictable performance.
> >>>>>>>> All 3 of your suggested ideas seem *very* complicated, and might
> >>>>>>> actually
> >>>>>>>> make behaviour less predictable for users as a consequence.
> >>>>>>>>
> >>>>>>>> I'm a bit concerned that the scope of this KIP is growing a bit
> out
> >>>> of
> >>>>>>>> control. While it's good to discuss ideas for future
> improvements, I
> >>>>>>> think
> >>>>>>>> it's important to narrow the scope down to a design that achieves
> >>>> the
> >>>>>>> most
> >>>>>>>> pressing objectives (constant sized restorations during dirty
> >>>>>>>> close/unexpected errors). Any design that this KIP produces can
> >>>>>>> ultimately
> >>>>>>>> be changed in the future, especially if the bulk of it is internal
> >>>>>>>> behaviour.
> >>>>>>>>
> >>>>>>>> I'm going to spend some time next week trying to re-work the
> >>>> original
> >>>>>>>> WriteBatchWithIndex design to remove the newTransaction() method,
> >>>> such
> >>>>>>> that
> >>>>>>>> it's just an implementation detail of RocksDBStore. That way, if
> we
> >>>>>>> want to
> >>>>>>>> replace WBWI with something in the future, like the SST file
> >>>> management
> >>>>>>>> outlined by John, then we can do so with little/no API changes.
> >>>>>>>>
> >>>>>>>> Regards,
> >>>>>>>>
> >>>>>>>> Nick
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >
>

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

Posted by Bruno Cadonna <ca...@apache.org>.
Hi Nick!

Thanks for the updates!

1.
Why does StateStore#flush() default to 
StateStore#commit(Collections.emptyMap())?
Since calls to flush() will not exist anymore after this KIP is 
released, I would rather throw an unsupported operation exception by 
default.


2.
When would a state store return -1 from 
StateStore#approximateNumUncommittedBytes() while being transactional?

Wouldn't StateStore#approximateNumUncommittedBytes() also return 0 if 
the state store is transactional but nothing has been written to the 
state store yet?


3.
Sorry for bringing this up again. Does this KIP really need to introduce 
StateStoreContext#isolationLevel()? StateStoreContext has already 
appConfigs() which basically exposes the same information, i.e., if EOS 
is enabled or not.
In one of your previous e-mails you wrote:

"My idea was to try to keep the StateStore interface as loosely coupled
from the Streams engine as possible, to give implementers more freedom, 
and reduce the amount of internal knowledge required."

While I understand the intent, I doubt that it decreases the coupling of 
a StateStore interface and the Streams engine. READ_COMMITTED only 
applies to IQ but not to reads by processors. Thus, implementers need to 
understand how Streams accesses the state stores.

I would like to hear what others think about this.


4.
Great exposing new metrics for transactional state stores! However, I 
would prefer to add new metrics and deprecate (in the docs) the 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 also need to adapt/add integration tests besides unit 
tests. Additionally, we probably need integration or system tests to 
verify that upgrades and downgrades between transactional and 
non-transactional state stores work as expected.


Best,
Bruno





On 7/21/23 10:34 PM, Nick Telford wrote:
> One more thing: I noted John's suggestion in the KIP, under "Rejected
> Alternatives". I still think it's an idea worth pursuing, but I believe
> that it's out of the scope of this KIP, because it solves a different set
> of problems to this KIP, and the scope of this one has already grown quite
> large!
> 
> On Fri, 21 Jul 2023 at 21:33, Nick Telford <ni...@gmail.com> wrote:
> 
>> Hi everyone,
>>
>> I've updated the KIP (
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores)
>> with the latest changes; mostly bringing back "Atomic Checkpointing" (for
>> what feels like the 10th time!). I think the one thing missing is some
>> changes to metrics (notably the store "flush" metrics will need to be
>> renamed to "commit").
>>
>> The reason I brought back Atomic Checkpointing was to decouple store flush
>> from store commit. This is important, because with Transactional
>> StateStores, we now need to call "flush" on *every* Task commit, and not
>> just when the StateStore is closing, otherwise our transaction buffer will
>> never be written and persisted, instead growing unbounded! I experimented
>> with some simple solutions, like forcing a store flush whenever the
>> transaction buffer was likely to exceed its configured size, but this was
>> brittle: it prevented the transaction buffer from being configured to be
>> unbounded, and it still would have required explicit flushes of RocksDB,
>> yielding sub-optimal performance and memory utilization.
>>
>> I deemed Atomic Checkpointing to be the "right" way to resolve this
>> problem. By ensuring that the changelog offsets that correspond to the most
>> recently written records are always atomically written to the StateStore
>> (by writing them to the same transaction buffer), we can avoid forcibly
>> flushing the RocksDB memtables to disk, letting RocksDB flush them only
>> when necessary, without losing any of our consistency guarantees. See the
>> updated KIP for more info.
>>
>> I have fully implemented these changes, although I'm still not entirely
>> happy with the implementation for segmented StateStores, so I plan to
>> refactor that. Despite that, all tests pass. If you'd like to try out or
>> review this highly experimental and incomplete branch, it's available here:
>> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note: it's built
>> against Kafka 3.5.0 so that I had a stable base to build and test it on,
>> and to enable easy apples-to-apples comparisons in a live environment. I
>> plan to rebase it against trunk once it's nearer completion and has been
>> proven on our main application.
>>
>> I would really appreciate help in reviewing and testing:
>> - Segmented (Versioned, Session and Window) stores
>> - Global stores
>>
>> As I do not currently use either of these, so my primary test environment
>> doesn't test these areas.
>>
>> I'm going on Parental Leave starting next week for a few weeks, so will
>> not have time to move this forward until late August. That said, your
>> feedback is welcome and appreciated, I just won't be able to respond as
>> quickly as usual.
>>
>> Regards,
>> Nick
>>
>> On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com> wrote:
>>
>>> Hi Bruno
>>>
>>> Yes, that's correct, although the impact on IQ is not something I had
>>> considered.
>>>
>>> What about atomically updating the state store from the transaction
>>>> buffer every commit interval and writing the checkpoint (thus, flushing
>>>> the memtable) every configured amount of data and/or number of commit
>>>> intervals?
>>>>
>>>
>>> I'm not quite sure I follow. Are you suggesting that we add an additional
>>> config for the max number of commit intervals between checkpoints? That
>>> way, we would checkpoint *either* when the transaction buffers are nearly
>>> full, *OR* whenever a certain number of commit intervals have elapsed,
>>> whichever comes first?
>>>
>>> That certainly seems reasonable, although this re-ignites an earlier
>>> debate about whether a config should be measured in "number of commit
>>> intervals", instead of just an absolute time.
>>>
>>> FWIW, I realised that this issue is the reason I was pursuing the Atomic
>>> Checkpoints, as it de-couples memtable flush from checkpointing, which
>>> enables us to just checkpoint on every commit without any performance
>>> impact. Atomic Checkpointing is definitely the "best" solution, but I'm not
>>> sure if this is enough to bring it back into this KIP.
>>>
>>> I'm currently working on moving all the transactional logic directly into
>>> RocksDBStore itself, which does away with the StateStore#newTransaction
>>> method, and reduces the number of new classes introduced, significantly
>>> reducing the complexity. If it works, and the complexity is drastically
>>> reduced, I may try bringing back Atomic Checkpoints into this KIP.
>>>
>>> Regards,
>>> Nick
>>>
>>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org> wrote:
>>>
>>>> Hi Nick,
>>>>
>>>> Thanks for the insights! Very interesting!
>>>>
>>>> As far as I understand, you want to atomically update the state store
>>>> from the transaction buffer, flush the memtable of a state store and
>>>> write the checkpoint not after the commit time elapsed but after the
>>>> transaction buffer reached a size that would lead to exceeding
>>>> statestore.transaction.buffer.max.bytes before the next commit interval
>>>> ends.
>>>> That means, the Kafka transaction would commit every commit interval but
>>>> the state store will only be atomically updated roughly every
>>>> statestore.transaction.buffer.max.bytes of data. Also IQ would then only
>>>> see new data roughly every statestore.transaction.buffer.max.bytes.
>>>> After a failure the state store needs to restore up to
>>>> statestore.transaction.buffer.max.bytes.
>>>>
>>>> Is this correct?
>>>>
>>>> What about atomically updating the state store from the transaction
>>>> buffer every commit interval and writing the checkpoint (thus, flushing
>>>> the memtable) every configured amount of data and/or number of commit
>>>> intervals? In such a way, we would have the same delay for records
>>>> appearing in output topics and IQ because both would appear when the
>>>> Kafka transaction is committed. However, after a failure the state store
>>>> still needs to restore up to statestore.transaction.buffer.max.bytes and
>>>> it might restore data that is already in the state store because the
>>>> checkpoint lags behind the last stable offset (i.e. the last committed
>>>> offset) of the changelog topics. Restoring data that is already in the
>>>> state store is idempotent, so eos should not violated.
>>>> This solution needs at least one new config to specify when a checkpoint
>>>> should be written.
>>>>
>>>>
>>>>
>>>> A small correction to your previous e-mail that does not change anything
>>>> you said: Under alos the default commit interval is 30 seconds, not five
>>>> seconds.
>>>>
>>>>
>>>> Best,
>>>> Bruno
>>>>
>>>>
>>>> On 01.07.23 12:37, Nick Telford wrote:
>>>>> Hi everyone,
>>>>>
>>>>> I've begun performance testing my branch on our staging environment,
>>>>> putting it through its paces in our non-trivial application. I'm
>>>> already
>>>>> observing the same increased flush rate that we saw the last time we
>>>>> attempted to use a version of this KIP, but this time, I think I know
>>>> why.
>>>>>
>>>>> Pre-KIP-892, StreamTask#postCommit, which is called at the end of the
>>>> Task
>>>>> commit process, has the following behaviour:
>>>>>
>>>>>      - Under ALOS: checkpoint the state stores. This includes
>>>>>      flushing memtables in RocksDB. This is acceptable because the
>>>> default
>>>>>      commit.interval.ms is 5 seconds, so forcibly flushing memtables
>>>> every 5
>>>>>      seconds is acceptable for most applications.
>>>>>      - Under EOS: checkpointing is not done, *unless* it's being
>>>> forced, due
>>>>>      to e.g. the Task closing or being revoked. This means that under
>>>> normal
>>>>>      processing conditions, the state stores will not be checkpointed,
>>>> and will
>>>>>      not have memtables flushed at all , unless RocksDB decides to
>>>> flush them on
>>>>>      its own. Checkpointing stores and force-flushing their memtables
>>>> is only
>>>>>      done when a Task is being closed.
>>>>>
>>>>> Under EOS, KIP-892 needs to checkpoint stores on at least *some* normal
>>>>> Task commits, in order to write the RocksDB transaction buffers to the
>>>>> state stores, and to ensure the offsets are synced to disk to prevent
>>>>> restores from getting out of hand. Consequently, my current
>>>> implementation
>>>>> calls maybeCheckpoint on *every* Task commit, which is far too
>>>> frequent.
>>>>> This causes checkpoints every 10,000 records, which is a change in
>>>> flush
>>>>> behaviour, potentially causing performance problems for some
>>>> applications.
>>>>>
>>>>> I'm looking into possible solutions, and I'm currently leaning towards
>>>>> using the statestore.transaction.buffer.max.bytes configuration to
>>>>> checkpoint Tasks once we are likely to exceed it. This would
>>>> complement the
>>>>> existing "early Task commit" functionality that this configuration
>>>>> provides, in the following way:
>>>>>
>>>>>      - Currently, we use statestore.transaction.buffer.max.bytes to
>>>> force an
>>>>>      early Task commit if processing more records would cause our state
>>>> store
>>>>>      transactions to exceed the memory assigned to them.
>>>>>      - New functionality: when a Task *does* commit, we will not
>>>> checkpoint
>>>>>      the stores (and hence flush the transaction buffers) unless we
>>>> expect to
>>>>>      cross the statestore.transaction.buffer.max.bytes threshold before
>>>> the next
>>>>>      commit
>>>>>
>>>>> I'm also open to suggestions.
>>>>>
>>>>> Regards,
>>>>> Nick
>>>>>
>>>>> On Thu, 22 Jun 2023 at 14:06, Nick Telford <ni...@gmail.com>
>>>> wrote:
>>>>>
>>>>>> Hi Bruno!
>>>>>>
>>>>>> 3.
>>>>>> By "less predictable for users", I meant in terms of understanding the
>>>>>> performance profile under various circumstances. The more complex the
>>>>>> solution, the more difficult it would be for users to understand the
>>>>>> performance they see. For example, spilling records to disk when the
>>>>>> transaction buffer reaches a threshold would, I expect, reduce write
>>>>>> throughput. This reduction in write throughput could be unexpected,
>>>> and
>>>>>> potentially difficult to diagnose/understand for users.
>>>>>> At the moment, I think the "early commit" concept is relatively
>>>>>> straightforward; it's easy to document, and conceptually fairly
>>>> obvious to
>>>>>> users. We could probably add a metric to make it easier to understand
>>>> when
>>>>>> it happens though.
>>>>>>
>>>>>> 3. (the second one)
>>>>>> The IsolationLevel is *essentially* an indirect way of telling
>>>> StateStores
>>>>>> whether they should be transactional. READ_COMMITTED essentially
>>>> requires
>>>>>> transactions, because it dictates that two threads calling
>>>>>> `newTransaction()` should not see writes from the other transaction
>>>> until
>>>>>> they have been committed. With READ_UNCOMMITTED, all bets are off, and
>>>>>> stores can allow threads to observe written records at any time,
>>>> which is
>>>>>> essentially "no transactions". That said, StateStores are free to
>>>> implement
>>>>>> these guarantees however they can, which is a bit more relaxed than
>>>>>> dictating "you must use transactions". For example, with RocksDB we
>>>> would
>>>>>> implement these as READ_COMMITTED == WBWI-based "transactions",
>>>>>> READ_UNCOMMITTED == direct writes to the database. But with other
>>>> storage
>>>>>> engines, it might be preferable to *always* use transactions, even
>>>> when
>>>>>> unnecessary; or there may be storage engines that don't provide
>>>>>> transactions, but the isolation guarantees can be met using a
>>>> different
>>>>>> technique.
>>>>>> My idea was to try to keep the StateStore interface as loosely coupled
>>>>>> from the Streams engine as possible, to give implementers more
>>>> freedom, and
>>>>>> reduce the amount of internal knowledge required.
>>>>>> That said, I understand that "IsolationLevel" might not be the right
>>>>>> abstraction, and we can always make it much more explicit if
>>>> required, e.g.
>>>>>> boolean transactional()
>>>>>>
>>>>>> 7-8.
>>>>>> I can make these changes either later today or tomorrow.
>>>>>>
>>>>>> Small update:
>>>>>> I've rebased my branch on trunk and fixed a bunch of issues that
>>>> needed
>>>>>> addressing. Currently, all the tests pass, which is promising, but it
>>>> will
>>>>>> need to undergo some performance testing. I haven't (yet) worked on
>>>>>> removing the `newTransaction()` stuff, but I would expect that,
>>>>>> behaviourally, it should make no difference. The branch is available
>>>> at
>>>>>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
>>>>>> interested in taking an early look.
>>>>>>
>>>>>> Regards,
>>>>>> Nick
>>>>>>
>>>>>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org>
>>>> wrote:
>>>>>>
>>>>>>> Hi Nick,
>>>>>>>
>>>>>>> 1.
>>>>>>> Yeah, I agree with you. That was actually also my point. I understood
>>>>>>> that John was proposing the ingestion path as a way to avoid the
>>>> early
>>>>>>> commits. Probably, I misinterpreted the intent.
>>>>>>>
>>>>>>> 2.
>>>>>>> I agree with John here, that actually it is public API. My question
>>>> is
>>>>>>> how this usage pattern affects normal processing.
>>>>>>>
>>>>>>> 3.
>>>>>>> My concern is that checking for the size of the transaction buffer
>>>> and
>>>>>>> maybe triggering an early commit affects the whole processing of
>>>> Kafka
>>>>>>> Streams. The transactionality of a state store is not confined to the
>>>>>>> state store itself, but spills over and changes the behavior of other
>>>>>>> parts of the system. I agree with you that it is a decent
>>>> compromise. I
>>>>>>> just wanted to analyse the downsides and list the options to overcome
>>>>>>> them. I also agree with you that all options seem quite heavy
>>>> compared
>>>>>>> with your KIP. I do not understand what you mean with "less
>>>> predictable
>>>>>>> for users", though.
>>>>>>>
>>>>>>>
>>>>>>> I found the discussions about the alternatives really interesting.
>>>> But I
>>>>>>> also think that your plan sounds good and we should continue with it!
>>>>>>>
>>>>>>>
>>>>>>> Some comments on your reply to my e-mail on June 20th:
>>>>>>>
>>>>>>> 3.
>>>>>>> Ah, now, I understand the reasoning behind putting isolation level in
>>>>>>> the state store context. Thanks! Should that also be a way to give
>>>> the
>>>>>>> the state store the opportunity to decide whether to turn on
>>>>>>> transactions or not?
>>>>>>> With my comment, I was more concerned about how do you know if a
>>>>>>> checkpoint file needs to be written under EOS, if you do not have a
>>>> way
>>>>>>> to know if the state store is transactional or not. If a state store
>>>> is
>>>>>>> transactional, the checkpoint file can be written during normal
>>>>>>> processing under EOS. If the state store is not transactional, the
>>>>>>> checkpoint file must not be written under EOS.
>>>>>>>
>>>>>>> 7.
>>>>>>> My point was about not only considering the bytes in memory in config
>>>>>>> statestore.uncommitted.max.bytes, but also bytes that might be
>>>> spilled
>>>>>>> on disk. Basically, I was wondering whether you should remove the
>>>>>>> "memory" in "Maximum number of memory bytes to be used to
>>>>>>> buffer uncommitted state-store records." My thinking was that even
>>>> if a
>>>>>>> state store spills uncommitted bytes to disk, limiting the overall
>>>> bytes
>>>>>>> might make sense. Thinking about it again and considering the recent
>>>>>>> discussions, it does not make too much sense anymore.
>>>>>>> I like the name statestore.transaction.buffer.max.bytes that you
>>>> proposed.
>>>>>>>
>>>>>>> 8.
>>>>>>> A high-level description (without implementation details) of how
>>>> Kafka
>>>>>>> Streams will manage the commit of changelog transactions, state store
>>>>>>> transactions and checkpointing would be great. Would be great if you
>>>>>>> could also add some sentences about the behavior in case of a
>>>> failure.
>>>>>>> For instance how does a transactional state store recover after a
>>>>>>> failure or what happens with the transaction buffer, etc. (that is
>>>> what
>>>>>>> I meant by "fail-over" in point 9.)
>>>>>>>
>>>>>>> Best,
>>>>>>> Bruno
>>>>>>>
>>>>>>> On 21.06.23 18:50, Nick Telford wrote:
>>>>>>>> Hi Bruno,
>>>>>>>>
>>>>>>>> 1.
>>>>>>>> Isn't this exactly the same issue that WriteBatchWithIndex
>>>> transactions
>>>>>>>> have, whereby exceeding (or likely to exceed) configured memory
>>>> needs to
>>>>>>>> trigger an early commit?
>>>>>>>>
>>>>>>>> 2.
>>>>>>>> This is one of my big concerns. Ultimately, any approach based on
>>>>>>> cracking
>>>>>>>> open RocksDB internals and using it in ways it's not really designed
>>>>>>> for is
>>>>>>>> likely to have some unforseen performance or consistency issues.
>>>>>>>>
>>>>>>>> 3.
>>>>>>>> What's your motivation for removing these early commits? While not
>>>>>>> ideal, I
>>>>>>>> think they're a decent compromise to ensure consistency whilst
>>>>>>> maintaining
>>>>>>>> good and predictable performance.
>>>>>>>> All 3 of your suggested ideas seem *very* complicated, and might
>>>>>>> actually
>>>>>>>> make behaviour less predictable for users as a consequence.
>>>>>>>>
>>>>>>>> I'm a bit concerned that the scope of this KIP is growing a bit out
>>>> of
>>>>>>>> control. While it's good to discuss ideas for future improvements, I
>>>>>>> think
>>>>>>>> it's important to narrow the scope down to a design that achieves
>>>> the
>>>>>>> most
>>>>>>>> pressing objectives (constant sized restorations during dirty
>>>>>>>> close/unexpected errors). Any design that this KIP produces can
>>>>>>> ultimately
>>>>>>>> be changed in the future, especially if the bulk of it is internal
>>>>>>>> behaviour.
>>>>>>>>
>>>>>>>> I'm going to spend some time next week trying to re-work the
>>>> original
>>>>>>>> WriteBatchWithIndex design to remove the newTransaction() method,
>>>> such
>>>>>>> that
>>>>>>>> it's just an implementation detail of RocksDBStore. That way, if we
>>>>>>> want to
>>>>>>>> replace WBWI with something in the future, like the SST file
>>>> management
>>>>>>>> outlined by John, then we can do so with little/no API changes.
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>>
>>>>>>>> Nick
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
> 

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

Posted by John Roesler <vv...@apache.org>.
No worries, I should have included a ";)" to let you know it was mostly 
tongue-in-cheek.

Thanks,
-John

On 6/21/23 12:34, Nick Telford wrote:
> Sorry John, I didn't mean to mis-characterize it like that. I was mostly
> referring to disabling memtables. AFAIK the SstFileWriter API is primarily
> designed for bulk ingest, e.g. for bootstrapping a database from a backup,
> rather than during normal operation of an online database. That said, I was
> overly alarmist in my phrasing.
> 
> My concern is only that, while the concept seems quite reasonable, there
> are no doubt hidden issues lurking.
> 
> On Wed, 21 Jun 2023 at 18:25, John Roesler <vv...@apache.org> wrote:
> 
>> Thanks Nick,
>>
>> That sounds good to me.
>>
>> I can't let (2) slide, though.. Writing and ingesting SST files is not a
>> RocksDB internal, but rather a supported usage pattern on public APIs.
>> Regardless, I think your overall preference is fine with me, especially
>> if we can internalize this change within the store implementation itself.
>>
>> Thanks,
>> -John
>>
>> On 6/21/23 11:50, Nick Telford wrote:
>>> Hi Bruno,
>>>
>>> 1.
>>> Isn't this exactly the same issue that WriteBatchWithIndex transactions
>>> have, whereby exceeding (or likely to exceed) configured memory needs to
>>> trigger an early commit?
>>>
>>> 2.
>>> This is one of my big concerns. Ultimately, any approach based on
>> cracking
>>> open RocksDB internals and using it in ways it's not really designed for
>> is
>>> likely to have some unforseen performance or consistency issues.
>>>
>>> 3.
>>> What's your motivation for removing these early commits? While not
>> ideal, I
>>> think they're a decent compromise to ensure consistency whilst
>> maintaining
>>> good and predictable performance.
>>> All 3 of your suggested ideas seem *very* complicated, and might actually
>>> make behaviour less predictable for users as a consequence.
>>>
>>> I'm a bit concerned that the scope of this KIP is growing a bit out of
>>> control. While it's good to discuss ideas for future improvements, I
>> think
>>> it's important to narrow the scope down to a design that achieves the
>> most
>>> pressing objectives (constant sized restorations during dirty
>>> close/unexpected errors). Any design that this KIP produces can
>> ultimately
>>> be changed in the future, especially if the bulk of it is internal
>>> behaviour.
>>>
>>> I'm going to spend some time next week trying to re-work the original
>>> WriteBatchWithIndex design to remove the newTransaction() method, such
>> that
>>> it's just an implementation detail of RocksDBStore. That way, if we want
>> to
>>> replace WBWI with something in the future, like the SST file management
>>> outlined by John, then we can do so with little/no API changes.
>>>
>>> Regards,
>>>
>>> Nick
>>>
>>
> 

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

Posted by Nick Telford <ni...@gmail.com>.
Sorry John, I didn't mean to mis-characterize it like that. I was mostly
referring to disabling memtables. AFAIK the SstFileWriter API is primarily
designed for bulk ingest, e.g. for bootstrapping a database from a backup,
rather than during normal operation of an online database. That said, I was
overly alarmist in my phrasing.

My concern is only that, while the concept seems quite reasonable, there
are no doubt hidden issues lurking.

On Wed, 21 Jun 2023 at 18:25, John Roesler <vv...@apache.org> wrote:

> Thanks Nick,
>
> That sounds good to me.
>
> I can't let (2) slide, though.. Writing and ingesting SST files is not a
> RocksDB internal, but rather a supported usage pattern on public APIs.
> Regardless, I think your overall preference is fine with me, especially
> if we can internalize this change within the store implementation itself.
>
> Thanks,
> -John
>
> On 6/21/23 11:50, Nick Telford wrote:
> > Hi Bruno,
> >
> > 1.
> > Isn't this exactly the same issue that WriteBatchWithIndex transactions
> > have, whereby exceeding (or likely to exceed) configured memory needs to
> > trigger an early commit?
> >
> > 2.
> > This is one of my big concerns. Ultimately, any approach based on
> cracking
> > open RocksDB internals and using it in ways it's not really designed for
> is
> > likely to have some unforseen performance or consistency issues.
> >
> > 3.
> > What's your motivation for removing these early commits? While not
> ideal, I
> > think they're a decent compromise to ensure consistency whilst
> maintaining
> > good and predictable performance.
> > All 3 of your suggested ideas seem *very* complicated, and might actually
> > make behaviour less predictable for users as a consequence.
> >
> > I'm a bit concerned that the scope of this KIP is growing a bit out of
> > control. While it's good to discuss ideas for future improvements, I
> think
> > it's important to narrow the scope down to a design that achieves the
> most
> > pressing objectives (constant sized restorations during dirty
> > close/unexpected errors). Any design that this KIP produces can
> ultimately
> > be changed in the future, especially if the bulk of it is internal
> > behaviour.
> >
> > I'm going to spend some time next week trying to re-work the original
> > WriteBatchWithIndex design to remove the newTransaction() method, such
> that
> > it's just an implementation detail of RocksDBStore. That way, if we want
> to
> > replace WBWI with something in the future, like the SST file management
> > outlined by John, then we can do so with little/no API changes.
> >
> > Regards,
> >
> > Nick
> >
>

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

Posted by John Roesler <vv...@apache.org>.
Thanks Nick,

That sounds good to me.

I can't let (2) slide, though.. Writing and ingesting SST files is not a 
RocksDB internal, but rather a supported usage pattern on public APIs. 
Regardless, I think your overall preference is fine with me, especially 
if we can internalize this change within the store implementation itself.

Thanks,
-John

On 6/21/23 11:50, Nick Telford wrote:
> Hi Bruno,
> 
> 1.
> Isn't this exactly the same issue that WriteBatchWithIndex transactions
> have, whereby exceeding (or likely to exceed) configured memory needs to
> trigger an early commit?
> 
> 2.
> This is one of my big concerns. Ultimately, any approach based on cracking
> open RocksDB internals and using it in ways it's not really designed for is
> likely to have some unforseen performance or consistency issues.
> 
> 3.
> What's your motivation for removing these early commits? While not ideal, I
> think they're a decent compromise to ensure consistency whilst maintaining
> good and predictable performance.
> All 3 of your suggested ideas seem *very* complicated, and might actually
> make behaviour less predictable for users as a consequence.
> 
> I'm a bit concerned that the scope of this KIP is growing a bit out of
> control. While it's good to discuss ideas for future improvements, I think
> it's important to narrow the scope down to a design that achieves the most
> pressing objectives (constant sized restorations during dirty
> close/unexpected errors). Any design that this KIP produces can ultimately
> be changed in the future, especially if the bulk of it is internal
> behaviour.
> 
> I'm going to spend some time next week trying to re-work the original
> WriteBatchWithIndex design to remove the newTransaction() method, such that
> it's just an implementation detail of RocksDBStore. That way, if we want to
> replace WBWI with something in the future, like the SST file management
> outlined by John, then we can do so with little/no API changes.
> 
> Regards,
> 
> Nick
> 

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

Posted by Nick Telford <ni...@gmail.com>.
One more thing: I noted John's suggestion in the KIP, under "Rejected
Alternatives". I still think it's an idea worth pursuing, but I believe
that it's out of the scope of this KIP, because it solves a different set
of problems to this KIP, and the scope of this one has already grown quite
large!

On Fri, 21 Jul 2023 at 21:33, Nick Telford <ni...@gmail.com> wrote:

> Hi everyone,
>
> I've updated the KIP (
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores)
> with the latest changes; mostly bringing back "Atomic Checkpointing" (for
> what feels like the 10th time!). I think the one thing missing is some
> changes to metrics (notably the store "flush" metrics will need to be
> renamed to "commit").
>
> The reason I brought back Atomic Checkpointing was to decouple store flush
> from store commit. This is important, because with Transactional
> StateStores, we now need to call "flush" on *every* Task commit, and not
> just when the StateStore is closing, otherwise our transaction buffer will
> never be written and persisted, instead growing unbounded! I experimented
> with some simple solutions, like forcing a store flush whenever the
> transaction buffer was likely to exceed its configured size, but this was
> brittle: it prevented the transaction buffer from being configured to be
> unbounded, and it still would have required explicit flushes of RocksDB,
> yielding sub-optimal performance and memory utilization.
>
> I deemed Atomic Checkpointing to be the "right" way to resolve this
> problem. By ensuring that the changelog offsets that correspond to the most
> recently written records are always atomically written to the StateStore
> (by writing them to the same transaction buffer), we can avoid forcibly
> flushing the RocksDB memtables to disk, letting RocksDB flush them only
> when necessary, without losing any of our consistency guarantees. See the
> updated KIP for more info.
>
> I have fully implemented these changes, although I'm still not entirely
> happy with the implementation for segmented StateStores, so I plan to
> refactor that. Despite that, all tests pass. If you'd like to try out or
> review this highly experimental and incomplete branch, it's available here:
> https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note: it's built
> against Kafka 3.5.0 so that I had a stable base to build and test it on,
> and to enable easy apples-to-apples comparisons in a live environment. I
> plan to rebase it against trunk once it's nearer completion and has been
> proven on our main application.
>
> I would really appreciate help in reviewing and testing:
> - Segmented (Versioned, Session and Window) stores
> - Global stores
>
> As I do not currently use either of these, so my primary test environment
> doesn't test these areas.
>
> I'm going on Parental Leave starting next week for a few weeks, so will
> not have time to move this forward until late August. That said, your
> feedback is welcome and appreciated, I just won't be able to respond as
> quickly as usual.
>
> Regards,
> Nick
>
> On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com> wrote:
>
>> Hi Bruno
>>
>> Yes, that's correct, although the impact on IQ is not something I had
>> considered.
>>
>> What about atomically updating the state store from the transaction
>>> buffer every commit interval and writing the checkpoint (thus, flushing
>>> the memtable) every configured amount of data and/or number of commit
>>> intervals?
>>>
>>
>> I'm not quite sure I follow. Are you suggesting that we add an additional
>> config for the max number of commit intervals between checkpoints? That
>> way, we would checkpoint *either* when the transaction buffers are nearly
>> full, *OR* whenever a certain number of commit intervals have elapsed,
>> whichever comes first?
>>
>> That certainly seems reasonable, although this re-ignites an earlier
>> debate about whether a config should be measured in "number of commit
>> intervals", instead of just an absolute time.
>>
>> FWIW, I realised that this issue is the reason I was pursuing the Atomic
>> Checkpoints, as it de-couples memtable flush from checkpointing, which
>> enables us to just checkpoint on every commit without any performance
>> impact. Atomic Checkpointing is definitely the "best" solution, but I'm not
>> sure if this is enough to bring it back into this KIP.
>>
>> I'm currently working on moving all the transactional logic directly into
>> RocksDBStore itself, which does away with the StateStore#newTransaction
>> method, and reduces the number of new classes introduced, significantly
>> reducing the complexity. If it works, and the complexity is drastically
>> reduced, I may try bringing back Atomic Checkpoints into this KIP.
>>
>> Regards,
>> Nick
>>
>> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org> wrote:
>>
>>> Hi Nick,
>>>
>>> Thanks for the insights! Very interesting!
>>>
>>> As far as I understand, you want to atomically update the state store
>>> from the transaction buffer, flush the memtable of a state store and
>>> write the checkpoint not after the commit time elapsed but after the
>>> transaction buffer reached a size that would lead to exceeding
>>> statestore.transaction.buffer.max.bytes before the next commit interval
>>> ends.
>>> That means, the Kafka transaction would commit every commit interval but
>>> the state store will only be atomically updated roughly every
>>> statestore.transaction.buffer.max.bytes of data. Also IQ would then only
>>> see new data roughly every statestore.transaction.buffer.max.bytes.
>>> After a failure the state store needs to restore up to
>>> statestore.transaction.buffer.max.bytes.
>>>
>>> Is this correct?
>>>
>>> What about atomically updating the state store from the transaction
>>> buffer every commit interval and writing the checkpoint (thus, flushing
>>> the memtable) every configured amount of data and/or number of commit
>>> intervals? In such a way, we would have the same delay for records
>>> appearing in output topics and IQ because both would appear when the
>>> Kafka transaction is committed. However, after a failure the state store
>>> still needs to restore up to statestore.transaction.buffer.max.bytes and
>>> it might restore data that is already in the state store because the
>>> checkpoint lags behind the last stable offset (i.e. the last committed
>>> offset) of the changelog topics. Restoring data that is already in the
>>> state store is idempotent, so eos should not violated.
>>> This solution needs at least one new config to specify when a checkpoint
>>> should be written.
>>>
>>>
>>>
>>> A small correction to your previous e-mail that does not change anything
>>> you said: Under alos the default commit interval is 30 seconds, not five
>>> seconds.
>>>
>>>
>>> Best,
>>> Bruno
>>>
>>>
>>> On 01.07.23 12:37, Nick Telford wrote:
>>> > Hi everyone,
>>> >
>>> > I've begun performance testing my branch on our staging environment,
>>> > putting it through its paces in our non-trivial application. I'm
>>> already
>>> > observing the same increased flush rate that we saw the last time we
>>> > attempted to use a version of this KIP, but this time, I think I know
>>> why.
>>> >
>>> > Pre-KIP-892, StreamTask#postCommit, which is called at the end of the
>>> Task
>>> > commit process, has the following behaviour:
>>> >
>>> >     - Under ALOS: checkpoint the state stores. This includes
>>> >     flushing memtables in RocksDB. This is acceptable because the
>>> default
>>> >     commit.interval.ms is 5 seconds, so forcibly flushing memtables
>>> every 5
>>> >     seconds is acceptable for most applications.
>>> >     - Under EOS: checkpointing is not done, *unless* it's being
>>> forced, due
>>> >     to e.g. the Task closing or being revoked. This means that under
>>> normal
>>> >     processing conditions, the state stores will not be checkpointed,
>>> and will
>>> >     not have memtables flushed at all , unless RocksDB decides to
>>> flush them on
>>> >     its own. Checkpointing stores and force-flushing their memtables
>>> is only
>>> >     done when a Task is being closed.
>>> >
>>> > Under EOS, KIP-892 needs to checkpoint stores on at least *some* normal
>>> > Task commits, in order to write the RocksDB transaction buffers to the
>>> > state stores, and to ensure the offsets are synced to disk to prevent
>>> > restores from getting out of hand. Consequently, my current
>>> implementation
>>> > calls maybeCheckpoint on *every* Task commit, which is far too
>>> frequent.
>>> > This causes checkpoints every 10,000 records, which is a change in
>>> flush
>>> > behaviour, potentially causing performance problems for some
>>> applications.
>>> >
>>> > I'm looking into possible solutions, and I'm currently leaning towards
>>> > using the statestore.transaction.buffer.max.bytes configuration to
>>> > checkpoint Tasks once we are likely to exceed it. This would
>>> complement the
>>> > existing "early Task commit" functionality that this configuration
>>> > provides, in the following way:
>>> >
>>> >     - Currently, we use statestore.transaction.buffer.max.bytes to
>>> force an
>>> >     early Task commit if processing more records would cause our state
>>> store
>>> >     transactions to exceed the memory assigned to them.
>>> >     - New functionality: when a Task *does* commit, we will not
>>> checkpoint
>>> >     the stores (and hence flush the transaction buffers) unless we
>>> expect to
>>> >     cross the statestore.transaction.buffer.max.bytes threshold before
>>> the next
>>> >     commit
>>> >
>>> > I'm also open to suggestions.
>>> >
>>> > Regards,
>>> > Nick
>>> >
>>> > On Thu, 22 Jun 2023 at 14:06, Nick Telford <ni...@gmail.com>
>>> wrote:
>>> >
>>> >> Hi Bruno!
>>> >>
>>> >> 3.
>>> >> By "less predictable for users", I meant in terms of understanding the
>>> >> performance profile under various circumstances. The more complex the
>>> >> solution, the more difficult it would be for users to understand the
>>> >> performance they see. For example, spilling records to disk when the
>>> >> transaction buffer reaches a threshold would, I expect, reduce write
>>> >> throughput. This reduction in write throughput could be unexpected,
>>> and
>>> >> potentially difficult to diagnose/understand for users.
>>> >> At the moment, I think the "early commit" concept is relatively
>>> >> straightforward; it's easy to document, and conceptually fairly
>>> obvious to
>>> >> users. We could probably add a metric to make it easier to understand
>>> when
>>> >> it happens though.
>>> >>
>>> >> 3. (the second one)
>>> >> The IsolationLevel is *essentially* an indirect way of telling
>>> StateStores
>>> >> whether they should be transactional. READ_COMMITTED essentially
>>> requires
>>> >> transactions, because it dictates that two threads calling
>>> >> `newTransaction()` should not see writes from the other transaction
>>> until
>>> >> they have been committed. With READ_UNCOMMITTED, all bets are off, and
>>> >> stores can allow threads to observe written records at any time,
>>> which is
>>> >> essentially "no transactions". That said, StateStores are free to
>>> implement
>>> >> these guarantees however they can, which is a bit more relaxed than
>>> >> dictating "you must use transactions". For example, with RocksDB we
>>> would
>>> >> implement these as READ_COMMITTED == WBWI-based "transactions",
>>> >> READ_UNCOMMITTED == direct writes to the database. But with other
>>> storage
>>> >> engines, it might be preferable to *always* use transactions, even
>>> when
>>> >> unnecessary; or there may be storage engines that don't provide
>>> >> transactions, but the isolation guarantees can be met using a
>>> different
>>> >> technique.
>>> >> My idea was to try to keep the StateStore interface as loosely coupled
>>> >> from the Streams engine as possible, to give implementers more
>>> freedom, and
>>> >> reduce the amount of internal knowledge required.
>>> >> That said, I understand that "IsolationLevel" might not be the right
>>> >> abstraction, and we can always make it much more explicit if
>>> required, e.g.
>>> >> boolean transactional()
>>> >>
>>> >> 7-8.
>>> >> I can make these changes either later today or tomorrow.
>>> >>
>>> >> Small update:
>>> >> I've rebased my branch on trunk and fixed a bunch of issues that
>>> needed
>>> >> addressing. Currently, all the tests pass, which is promising, but it
>>> will
>>> >> need to undergo some performance testing. I haven't (yet) worked on
>>> >> removing the `newTransaction()` stuff, but I would expect that,
>>> >> behaviourally, it should make no difference. The branch is available
>>> at
>>> >> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
>>> >> interested in taking an early look.
>>> >>
>>> >> Regards,
>>> >> Nick
>>> >>
>>> >> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org>
>>> wrote:
>>> >>
>>> >>> Hi Nick,
>>> >>>
>>> >>> 1.
>>> >>> Yeah, I agree with you. That was actually also my point. I understood
>>> >>> that John was proposing the ingestion path as a way to avoid the
>>> early
>>> >>> commits. Probably, I misinterpreted the intent.
>>> >>>
>>> >>> 2.
>>> >>> I agree with John here, that actually it is public API. My question
>>> is
>>> >>> how this usage pattern affects normal processing.
>>> >>>
>>> >>> 3.
>>> >>> My concern is that checking for the size of the transaction buffer
>>> and
>>> >>> maybe triggering an early commit affects the whole processing of
>>> Kafka
>>> >>> Streams. The transactionality of a state store is not confined to the
>>> >>> state store itself, but spills over and changes the behavior of other
>>> >>> parts of the system. I agree with you that it is a decent
>>> compromise. I
>>> >>> just wanted to analyse the downsides and list the options to overcome
>>> >>> them. I also agree with you that all options seem quite heavy
>>> compared
>>> >>> with your KIP. I do not understand what you mean with "less
>>> predictable
>>> >>> for users", though.
>>> >>>
>>> >>>
>>> >>> I found the discussions about the alternatives really interesting.
>>> But I
>>> >>> also think that your plan sounds good and we should continue with it!
>>> >>>
>>> >>>
>>> >>> Some comments on your reply to my e-mail on June 20th:
>>> >>>
>>> >>> 3.
>>> >>> Ah, now, I understand the reasoning behind putting isolation level in
>>> >>> the state store context. Thanks! Should that also be a way to give
>>> the
>>> >>> the state store the opportunity to decide whether to turn on
>>> >>> transactions or not?
>>> >>> With my comment, I was more concerned about how do you know if a
>>> >>> checkpoint file needs to be written under EOS, if you do not have a
>>> way
>>> >>> to know if the state store is transactional or not. If a state store
>>> is
>>> >>> transactional, the checkpoint file can be written during normal
>>> >>> processing under EOS. If the state store is not transactional, the
>>> >>> checkpoint file must not be written under EOS.
>>> >>>
>>> >>> 7.
>>> >>> My point was about not only considering the bytes in memory in config
>>> >>> statestore.uncommitted.max.bytes, but also bytes that might be
>>> spilled
>>> >>> on disk. Basically, I was wondering whether you should remove the
>>> >>> "memory" in "Maximum number of memory bytes to be used to
>>> >>> buffer uncommitted state-store records." My thinking was that even
>>> if a
>>> >>> state store spills uncommitted bytes to disk, limiting the overall
>>> bytes
>>> >>> might make sense. Thinking about it again and considering the recent
>>> >>> discussions, it does not make too much sense anymore.
>>> >>> I like the name statestore.transaction.buffer.max.bytes that you
>>> proposed.
>>> >>>
>>> >>> 8.
>>> >>> A high-level description (without implementation details) of how
>>> Kafka
>>> >>> Streams will manage the commit of changelog transactions, state store
>>> >>> transactions and checkpointing would be great. Would be great if you
>>> >>> could also add some sentences about the behavior in case of a
>>> failure.
>>> >>> For instance how does a transactional state store recover after a
>>> >>> failure or what happens with the transaction buffer, etc. (that is
>>> what
>>> >>> I meant by "fail-over" in point 9.)
>>> >>>
>>> >>> Best,
>>> >>> Bruno
>>> >>>
>>> >>> On 21.06.23 18:50, Nick Telford wrote:
>>> >>>> Hi Bruno,
>>> >>>>
>>> >>>> 1.
>>> >>>> Isn't this exactly the same issue that WriteBatchWithIndex
>>> transactions
>>> >>>> have, whereby exceeding (or likely to exceed) configured memory
>>> needs to
>>> >>>> trigger an early commit?
>>> >>>>
>>> >>>> 2.
>>> >>>> This is one of my big concerns. Ultimately, any approach based on
>>> >>> cracking
>>> >>>> open RocksDB internals and using it in ways it's not really designed
>>> >>> for is
>>> >>>> likely to have some unforseen performance or consistency issues.
>>> >>>>
>>> >>>> 3.
>>> >>>> What's your motivation for removing these early commits? While not
>>> >>> ideal, I
>>> >>>> think they're a decent compromise to ensure consistency whilst
>>> >>> maintaining
>>> >>>> good and predictable performance.
>>> >>>> All 3 of your suggested ideas seem *very* complicated, and might
>>> >>> actually
>>> >>>> make behaviour less predictable for users as a consequence.
>>> >>>>
>>> >>>> I'm a bit concerned that the scope of this KIP is growing a bit out
>>> of
>>> >>>> control. While it's good to discuss ideas for future improvements, I
>>> >>> think
>>> >>>> it's important to narrow the scope down to a design that achieves
>>> the
>>> >>> most
>>> >>>> pressing objectives (constant sized restorations during dirty
>>> >>>> close/unexpected errors). Any design that this KIP produces can
>>> >>> ultimately
>>> >>>> be changed in the future, especially if the bulk of it is internal
>>> >>>> behaviour.
>>> >>>>
>>> >>>> I'm going to spend some time next week trying to re-work the
>>> original
>>> >>>> WriteBatchWithIndex design to remove the newTransaction() method,
>>> such
>>> >>> that
>>> >>>> it's just an implementation detail of RocksDBStore. That way, if we
>>> >>> want to
>>> >>>> replace WBWI with something in the future, like the SST file
>>> management
>>> >>>> outlined by John, then we can do so with little/no API changes.
>>> >>>>
>>> >>>> Regards,
>>> >>>>
>>> >>>> Nick
>>> >>>>
>>> >>>
>>> >>
>>> >
>>>
>>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi everyone,

I've updated the KIP (
https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores)
with the latest changes; mostly bringing back "Atomic Checkpointing" (for
what feels like the 10th time!). I think the one thing missing is some
changes to metrics (notably the store "flush" metrics will need to be
renamed to "commit").

The reason I brought back Atomic Checkpointing was to decouple store flush
from store commit. This is important, because with Transactional
StateStores, we now need to call "flush" on *every* Task commit, and not
just when the StateStore is closing, otherwise our transaction buffer will
never be written and persisted, instead growing unbounded! I experimented
with some simple solutions, like forcing a store flush whenever the
transaction buffer was likely to exceed its configured size, but this was
brittle: it prevented the transaction buffer from being configured to be
unbounded, and it still would have required explicit flushes of RocksDB,
yielding sub-optimal performance and memory utilization.

I deemed Atomic Checkpointing to be the "right" way to resolve this
problem. By ensuring that the changelog offsets that correspond to the most
recently written records are always atomically written to the StateStore
(by writing them to the same transaction buffer), we can avoid forcibly
flushing the RocksDB memtables to disk, letting RocksDB flush them only
when necessary, without losing any of our consistency guarantees. See the
updated KIP for more info.

I have fully implemented these changes, although I'm still not entirely
happy with the implementation for segmented StateStores, so I plan to
refactor that. Despite that, all tests pass. If you'd like to try out or
review this highly experimental and incomplete branch, it's available here:
https://github.com/nicktelford/kafka/tree/KIP-892-3.5.0. Note: it's built
against Kafka 3.5.0 so that I had a stable base to build and test it on,
and to enable easy apples-to-apples comparisons in a live environment. I
plan to rebase it against trunk once it's nearer completion and has been
proven on our main application.

I would really appreciate help in reviewing and testing:
- Segmented (Versioned, Session and Window) stores
- Global stores

As I do not currently use either of these, so my primary test environment
doesn't test these areas.

I'm going on Parental Leave starting next week for a few weeks, so will not
have time to move this forward until late August. That said, your feedback
is welcome and appreciated, I just won't be able to respond as quickly as
usual.

Regards,
Nick

On Mon, 3 Jul 2023 at 16:23, Nick Telford <ni...@gmail.com> wrote:

> Hi Bruno
>
> Yes, that's correct, although the impact on IQ is not something I had
> considered.
>
> What about atomically updating the state store from the transaction
>> buffer every commit interval and writing the checkpoint (thus, flushing
>> the memtable) every configured amount of data and/or number of commit
>> intervals?
>>
>
> I'm not quite sure I follow. Are you suggesting that we add an additional
> config for the max number of commit intervals between checkpoints? That
> way, we would checkpoint *either* when the transaction buffers are nearly
> full, *OR* whenever a certain number of commit intervals have elapsed,
> whichever comes first?
>
> That certainly seems reasonable, although this re-ignites an earlier
> debate about whether a config should be measured in "number of commit
> intervals", instead of just an absolute time.
>
> FWIW, I realised that this issue is the reason I was pursuing the Atomic
> Checkpoints, as it de-couples memtable flush from checkpointing, which
> enables us to just checkpoint on every commit without any performance
> impact. Atomic Checkpointing is definitely the "best" solution, but I'm not
> sure if this is enough to bring it back into this KIP.
>
> I'm currently working on moving all the transactional logic directly into
> RocksDBStore itself, which does away with the StateStore#newTransaction
> method, and reduces the number of new classes introduced, significantly
> reducing the complexity. If it works, and the complexity is drastically
> reduced, I may try bringing back Atomic Checkpoints into this KIP.
>
> Regards,
> Nick
>
> On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org> wrote:
>
>> Hi Nick,
>>
>> Thanks for the insights! Very interesting!
>>
>> As far as I understand, you want to atomically update the state store
>> from the transaction buffer, flush the memtable of a state store and
>> write the checkpoint not after the commit time elapsed but after the
>> transaction buffer reached a size that would lead to exceeding
>> statestore.transaction.buffer.max.bytes before the next commit interval
>> ends.
>> That means, the Kafka transaction would commit every commit interval but
>> the state store will only be atomically updated roughly every
>> statestore.transaction.buffer.max.bytes of data. Also IQ would then only
>> see new data roughly every statestore.transaction.buffer.max.bytes.
>> After a failure the state store needs to restore up to
>> statestore.transaction.buffer.max.bytes.
>>
>> Is this correct?
>>
>> What about atomically updating the state store from the transaction
>> buffer every commit interval and writing the checkpoint (thus, flushing
>> the memtable) every configured amount of data and/or number of commit
>> intervals? In such a way, we would have the same delay for records
>> appearing in output topics and IQ because both would appear when the
>> Kafka transaction is committed. However, after a failure the state store
>> still needs to restore up to statestore.transaction.buffer.max.bytes and
>> it might restore data that is already in the state store because the
>> checkpoint lags behind the last stable offset (i.e. the last committed
>> offset) of the changelog topics. Restoring data that is already in the
>> state store is idempotent, so eos should not violated.
>> This solution needs at least one new config to specify when a checkpoint
>> should be written.
>>
>>
>>
>> A small correction to your previous e-mail that does not change anything
>> you said: Under alos the default commit interval is 30 seconds, not five
>> seconds.
>>
>>
>> Best,
>> Bruno
>>
>>
>> On 01.07.23 12:37, Nick Telford wrote:
>> > Hi everyone,
>> >
>> > I've begun performance testing my branch on our staging environment,
>> > putting it through its paces in our non-trivial application. I'm already
>> > observing the same increased flush rate that we saw the last time we
>> > attempted to use a version of this KIP, but this time, I think I know
>> why.
>> >
>> > Pre-KIP-892, StreamTask#postCommit, which is called at the end of the
>> Task
>> > commit process, has the following behaviour:
>> >
>> >     - Under ALOS: checkpoint the state stores. This includes
>> >     flushing memtables in RocksDB. This is acceptable because the
>> default
>> >     commit.interval.ms is 5 seconds, so forcibly flushing memtables
>> every 5
>> >     seconds is acceptable for most applications.
>> >     - Under EOS: checkpointing is not done, *unless* it's being forced,
>> due
>> >     to e.g. the Task closing or being revoked. This means that under
>> normal
>> >     processing conditions, the state stores will not be checkpointed,
>> and will
>> >     not have memtables flushed at all , unless RocksDB decides to flush
>> them on
>> >     its own. Checkpointing stores and force-flushing their memtables is
>> only
>> >     done when a Task is being closed.
>> >
>> > Under EOS, KIP-892 needs to checkpoint stores on at least *some* normal
>> > Task commits, in order to write the RocksDB transaction buffers to the
>> > state stores, and to ensure the offsets are synced to disk to prevent
>> > restores from getting out of hand. Consequently, my current
>> implementation
>> > calls maybeCheckpoint on *every* Task commit, which is far too frequent.
>> > This causes checkpoints every 10,000 records, which is a change in flush
>> > behaviour, potentially causing performance problems for some
>> applications.
>> >
>> > I'm looking into possible solutions, and I'm currently leaning towards
>> > using the statestore.transaction.buffer.max.bytes configuration to
>> > checkpoint Tasks once we are likely to exceed it. This would complement
>> the
>> > existing "early Task commit" functionality that this configuration
>> > provides, in the following way:
>> >
>> >     - Currently, we use statestore.transaction.buffer.max.bytes to
>> force an
>> >     early Task commit if processing more records would cause our state
>> store
>> >     transactions to exceed the memory assigned to them.
>> >     - New functionality: when a Task *does* commit, we will not
>> checkpoint
>> >     the stores (and hence flush the transaction buffers) unless we
>> expect to
>> >     cross the statestore.transaction.buffer.max.bytes threshold before
>> the next
>> >     commit
>> >
>> > I'm also open to suggestions.
>> >
>> > Regards,
>> > Nick
>> >
>> > On Thu, 22 Jun 2023 at 14:06, Nick Telford <ni...@gmail.com>
>> wrote:
>> >
>> >> Hi Bruno!
>> >>
>> >> 3.
>> >> By "less predictable for users", I meant in terms of understanding the
>> >> performance profile under various circumstances. The more complex the
>> >> solution, the more difficult it would be for users to understand the
>> >> performance they see. For example, spilling records to disk when the
>> >> transaction buffer reaches a threshold would, I expect, reduce write
>> >> throughput. This reduction in write throughput could be unexpected, and
>> >> potentially difficult to diagnose/understand for users.
>> >> At the moment, I think the "early commit" concept is relatively
>> >> straightforward; it's easy to document, and conceptually fairly
>> obvious to
>> >> users. We could probably add a metric to make it easier to understand
>> when
>> >> it happens though.
>> >>
>> >> 3. (the second one)
>> >> The IsolationLevel is *essentially* an indirect way of telling
>> StateStores
>> >> whether they should be transactional. READ_COMMITTED essentially
>> requires
>> >> transactions, because it dictates that two threads calling
>> >> `newTransaction()` should not see writes from the other transaction
>> until
>> >> they have been committed. With READ_UNCOMMITTED, all bets are off, and
>> >> stores can allow threads to observe written records at any time, which
>> is
>> >> essentially "no transactions". That said, StateStores are free to
>> implement
>> >> these guarantees however they can, which is a bit more relaxed than
>> >> dictating "you must use transactions". For example, with RocksDB we
>> would
>> >> implement these as READ_COMMITTED == WBWI-based "transactions",
>> >> READ_UNCOMMITTED == direct writes to the database. But with other
>> storage
>> >> engines, it might be preferable to *always* use transactions, even when
>> >> unnecessary; or there may be storage engines that don't provide
>> >> transactions, but the isolation guarantees can be met using a different
>> >> technique.
>> >> My idea was to try to keep the StateStore interface as loosely coupled
>> >> from the Streams engine as possible, to give implementers more
>> freedom, and
>> >> reduce the amount of internal knowledge required.
>> >> That said, I understand that "IsolationLevel" might not be the right
>> >> abstraction, and we can always make it much more explicit if required,
>> e.g.
>> >> boolean transactional()
>> >>
>> >> 7-8.
>> >> I can make these changes either later today or tomorrow.
>> >>
>> >> Small update:
>> >> I've rebased my branch on trunk and fixed a bunch of issues that needed
>> >> addressing. Currently, all the tests pass, which is promising, but it
>> will
>> >> need to undergo some performance testing. I haven't (yet) worked on
>> >> removing the `newTransaction()` stuff, but I would expect that,
>> >> behaviourally, it should make no difference. The branch is available at
>> >> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
>> >> interested in taking an early look.
>> >>
>> >> Regards,
>> >> Nick
>> >>
>> >> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org>
>> wrote:
>> >>
>> >>> Hi Nick,
>> >>>
>> >>> 1.
>> >>> Yeah, I agree with you. That was actually also my point. I understood
>> >>> that John was proposing the ingestion path as a way to avoid the early
>> >>> commits. Probably, I misinterpreted the intent.
>> >>>
>> >>> 2.
>> >>> I agree with John here, that actually it is public API. My question is
>> >>> how this usage pattern affects normal processing.
>> >>>
>> >>> 3.
>> >>> My concern is that checking for the size of the transaction buffer and
>> >>> maybe triggering an early commit affects the whole processing of Kafka
>> >>> Streams. The transactionality of a state store is not confined to the
>> >>> state store itself, but spills over and changes the behavior of other
>> >>> parts of the system. I agree with you that it is a decent compromise.
>> I
>> >>> just wanted to analyse the downsides and list the options to overcome
>> >>> them. I also agree with you that all options seem quite heavy compared
>> >>> with your KIP. I do not understand what you mean with "less
>> predictable
>> >>> for users", though.
>> >>>
>> >>>
>> >>> I found the discussions about the alternatives really interesting.
>> But I
>> >>> also think that your plan sounds good and we should continue with it!
>> >>>
>> >>>
>> >>> Some comments on your reply to my e-mail on June 20th:
>> >>>
>> >>> 3.
>> >>> Ah, now, I understand the reasoning behind putting isolation level in
>> >>> the state store context. Thanks! Should that also be a way to give the
>> >>> the state store the opportunity to decide whether to turn on
>> >>> transactions or not?
>> >>> With my comment, I was more concerned about how do you know if a
>> >>> checkpoint file needs to be written under EOS, if you do not have a
>> way
>> >>> to know if the state store is transactional or not. If a state store
>> is
>> >>> transactional, the checkpoint file can be written during normal
>> >>> processing under EOS. If the state store is not transactional, the
>> >>> checkpoint file must not be written under EOS.
>> >>>
>> >>> 7.
>> >>> My point was about not only considering the bytes in memory in config
>> >>> statestore.uncommitted.max.bytes, but also bytes that might be spilled
>> >>> on disk. Basically, I was wondering whether you should remove the
>> >>> "memory" in "Maximum number of memory bytes to be used to
>> >>> buffer uncommitted state-store records." My thinking was that even if
>> a
>> >>> state store spills uncommitted bytes to disk, limiting the overall
>> bytes
>> >>> might make sense. Thinking about it again and considering the recent
>> >>> discussions, it does not make too much sense anymore.
>> >>> I like the name statestore.transaction.buffer.max.bytes that you
>> proposed.
>> >>>
>> >>> 8.
>> >>> A high-level description (without implementation details) of how Kafka
>> >>> Streams will manage the commit of changelog transactions, state store
>> >>> transactions and checkpointing would be great. Would be great if you
>> >>> could also add some sentences about the behavior in case of a failure.
>> >>> For instance how does a transactional state store recover after a
>> >>> failure or what happens with the transaction buffer, etc. (that is
>> what
>> >>> I meant by "fail-over" in point 9.)
>> >>>
>> >>> Best,
>> >>> Bruno
>> >>>
>> >>> On 21.06.23 18:50, Nick Telford wrote:
>> >>>> Hi Bruno,
>> >>>>
>> >>>> 1.
>> >>>> Isn't this exactly the same issue that WriteBatchWithIndex
>> transactions
>> >>>> have, whereby exceeding (or likely to exceed) configured memory
>> needs to
>> >>>> trigger an early commit?
>> >>>>
>> >>>> 2.
>> >>>> This is one of my big concerns. Ultimately, any approach based on
>> >>> cracking
>> >>>> open RocksDB internals and using it in ways it's not really designed
>> >>> for is
>> >>>> likely to have some unforseen performance or consistency issues.
>> >>>>
>> >>>> 3.
>> >>>> What's your motivation for removing these early commits? While not
>> >>> ideal, I
>> >>>> think they're a decent compromise to ensure consistency whilst
>> >>> maintaining
>> >>>> good and predictable performance.
>> >>>> All 3 of your suggested ideas seem *very* complicated, and might
>> >>> actually
>> >>>> make behaviour less predictable for users as a consequence.
>> >>>>
>> >>>> I'm a bit concerned that the scope of this KIP is growing a bit out
>> of
>> >>>> control. While it's good to discuss ideas for future improvements, I
>> >>> think
>> >>>> it's important to narrow the scope down to a design that achieves the
>> >>> most
>> >>>> pressing objectives (constant sized restorations during dirty
>> >>>> close/unexpected errors). Any design that this KIP produces can
>> >>> ultimately
>> >>>> be changed in the future, especially if the bulk of it is internal
>> >>>> behaviour.
>> >>>>
>> >>>> I'm going to spend some time next week trying to re-work the original
>> >>>> WriteBatchWithIndex design to remove the newTransaction() method,
>> such
>> >>> that
>> >>>> it's just an implementation detail of RocksDBStore. That way, if we
>> >>> want to
>> >>>> replace WBWI with something in the future, like the SST file
>> management
>> >>>> outlined by John, then we can do so with little/no API changes.
>> >>>>
>> >>>> Regards,
>> >>>>
>> >>>> Nick
>> >>>>
>> >>>
>> >>
>> >
>>
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Bruno

Yes, that's correct, although the impact on IQ is not something I had
considered.

What about atomically updating the state store from the transaction
> buffer every commit interval and writing the checkpoint (thus, flushing
> the memtable) every configured amount of data and/or number of commit
> intervals?
>

I'm not quite sure I follow. Are you suggesting that we add an additional
config for the max number of commit intervals between checkpoints? That
way, we would checkpoint *either* when the transaction buffers are nearly
full, *OR* whenever a certain number of commit intervals have elapsed,
whichever comes first?

That certainly seems reasonable, although this re-ignites an earlier debate
about whether a config should be measured in "number of commit intervals",
instead of just an absolute time.

FWIW, I realised that this issue is the reason I was pursuing the Atomic
Checkpoints, as it de-couples memtable flush from checkpointing, which
enables us to just checkpoint on every commit without any performance
impact. Atomic Checkpointing is definitely the "best" solution, but I'm not
sure if this is enough to bring it back into this KIP.

I'm currently working on moving all the transactional logic directly into
RocksDBStore itself, which does away with the StateStore#newTransaction
method, and reduces the number of new classes introduced, significantly
reducing the complexity. If it works, and the complexity is drastically
reduced, I may try bringing back Atomic Checkpoints into this KIP.

Regards,
Nick

On Mon, 3 Jul 2023 at 15:27, Bruno Cadonna <ca...@apache.org> wrote:

> Hi Nick,
>
> Thanks for the insights! Very interesting!
>
> As far as I understand, you want to atomically update the state store
> from the transaction buffer, flush the memtable of a state store and
> write the checkpoint not after the commit time elapsed but after the
> transaction buffer reached a size that would lead to exceeding
> statestore.transaction.buffer.max.bytes before the next commit interval
> ends.
> That means, the Kafka transaction would commit every commit interval but
> the state store will only be atomically updated roughly every
> statestore.transaction.buffer.max.bytes of data. Also IQ would then only
> see new data roughly every statestore.transaction.buffer.max.bytes.
> After a failure the state store needs to restore up to
> statestore.transaction.buffer.max.bytes.
>
> Is this correct?
>
> What about atomically updating the state store from the transaction
> buffer every commit interval and writing the checkpoint (thus, flushing
> the memtable) every configured amount of data and/or number of commit
> intervals? In such a way, we would have the same delay for records
> appearing in output topics and IQ because both would appear when the
> Kafka transaction is committed. However, after a failure the state store
> still needs to restore up to statestore.transaction.buffer.max.bytes and
> it might restore data that is already in the state store because the
> checkpoint lags behind the last stable offset (i.e. the last committed
> offset) of the changelog topics. Restoring data that is already in the
> state store is idempotent, so eos should not violated.
> This solution needs at least one new config to specify when a checkpoint
> should be written.
>
>
>
> A small correction to your previous e-mail that does not change anything
> you said: Under alos the default commit interval is 30 seconds, not five
> seconds.
>
>
> Best,
> Bruno
>
>
> On 01.07.23 12:37, Nick Telford wrote:
> > Hi everyone,
> >
> > I've begun performance testing my branch on our staging environment,
> > putting it through its paces in our non-trivial application. I'm already
> > observing the same increased flush rate that we saw the last time we
> > attempted to use a version of this KIP, but this time, I think I know
> why.
> >
> > Pre-KIP-892, StreamTask#postCommit, which is called at the end of the
> Task
> > commit process, has the following behaviour:
> >
> >     - Under ALOS: checkpoint the state stores. This includes
> >     flushing memtables in RocksDB. This is acceptable because the default
> >     commit.interval.ms is 5 seconds, so forcibly flushing memtables
> every 5
> >     seconds is acceptable for most applications.
> >     - Under EOS: checkpointing is not done, *unless* it's being forced,
> due
> >     to e.g. the Task closing or being revoked. This means that under
> normal
> >     processing conditions, the state stores will not be checkpointed,
> and will
> >     not have memtables flushed at all , unless RocksDB decides to flush
> them on
> >     its own. Checkpointing stores and force-flushing their memtables is
> only
> >     done when a Task is being closed.
> >
> > Under EOS, KIP-892 needs to checkpoint stores on at least *some* normal
> > Task commits, in order to write the RocksDB transaction buffers to the
> > state stores, and to ensure the offsets are synced to disk to prevent
> > restores from getting out of hand. Consequently, my current
> implementation
> > calls maybeCheckpoint on *every* Task commit, which is far too frequent.
> > This causes checkpoints every 10,000 records, which is a change in flush
> > behaviour, potentially causing performance problems for some
> applications.
> >
> > I'm looking into possible solutions, and I'm currently leaning towards
> > using the statestore.transaction.buffer.max.bytes configuration to
> > checkpoint Tasks once we are likely to exceed it. This would complement
> the
> > existing "early Task commit" functionality that this configuration
> > provides, in the following way:
> >
> >     - Currently, we use statestore.transaction.buffer.max.bytes to force
> an
> >     early Task commit if processing more records would cause our state
> store
> >     transactions to exceed the memory assigned to them.
> >     - New functionality: when a Task *does* commit, we will not
> checkpoint
> >     the stores (and hence flush the transaction buffers) unless we
> expect to
> >     cross the statestore.transaction.buffer.max.bytes threshold before
> the next
> >     commit
> >
> > I'm also open to suggestions.
> >
> > Regards,
> > Nick
> >
> > On Thu, 22 Jun 2023 at 14:06, Nick Telford <ni...@gmail.com>
> wrote:
> >
> >> Hi Bruno!
> >>
> >> 3.
> >> By "less predictable for users", I meant in terms of understanding the
> >> performance profile under various circumstances. The more complex the
> >> solution, the more difficult it would be for users to understand the
> >> performance they see. For example, spilling records to disk when the
> >> transaction buffer reaches a threshold would, I expect, reduce write
> >> throughput. This reduction in write throughput could be unexpected, and
> >> potentially difficult to diagnose/understand for users.
> >> At the moment, I think the "early commit" concept is relatively
> >> straightforward; it's easy to document, and conceptually fairly obvious
> to
> >> users. We could probably add a metric to make it easier to understand
> when
> >> it happens though.
> >>
> >> 3. (the second one)
> >> The IsolationLevel is *essentially* an indirect way of telling
> StateStores
> >> whether they should be transactional. READ_COMMITTED essentially
> requires
> >> transactions, because it dictates that two threads calling
> >> `newTransaction()` should not see writes from the other transaction
> until
> >> they have been committed. With READ_UNCOMMITTED, all bets are off, and
> >> stores can allow threads to observe written records at any time, which
> is
> >> essentially "no transactions". That said, StateStores are free to
> implement
> >> these guarantees however they can, which is a bit more relaxed than
> >> dictating "you must use transactions". For example, with RocksDB we
> would
> >> implement these as READ_COMMITTED == WBWI-based "transactions",
> >> READ_UNCOMMITTED == direct writes to the database. But with other
> storage
> >> engines, it might be preferable to *always* use transactions, even when
> >> unnecessary; or there may be storage engines that don't provide
> >> transactions, but the isolation guarantees can be met using a different
> >> technique.
> >> My idea was to try to keep the StateStore interface as loosely coupled
> >> from the Streams engine as possible, to give implementers more freedom,
> and
> >> reduce the amount of internal knowledge required.
> >> That said, I understand that "IsolationLevel" might not be the right
> >> abstraction, and we can always make it much more explicit if required,
> e.g.
> >> boolean transactional()
> >>
> >> 7-8.
> >> I can make these changes either later today or tomorrow.
> >>
> >> Small update:
> >> I've rebased my branch on trunk and fixed a bunch of issues that needed
> >> addressing. Currently, all the tests pass, which is promising, but it
> will
> >> need to undergo some performance testing. I haven't (yet) worked on
> >> removing the `newTransaction()` stuff, but I would expect that,
> >> behaviourally, it should make no difference. The branch is available at
> >> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
> >> interested in taking an early look.
> >>
> >> Regards,
> >> Nick
> >>
> >> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org> wrote:
> >>
> >>> Hi Nick,
> >>>
> >>> 1.
> >>> Yeah, I agree with you. That was actually also my point. I understood
> >>> that John was proposing the ingestion path as a way to avoid the early
> >>> commits. Probably, I misinterpreted the intent.
> >>>
> >>> 2.
> >>> I agree with John here, that actually it is public API. My question is
> >>> how this usage pattern affects normal processing.
> >>>
> >>> 3.
> >>> My concern is that checking for the size of the transaction buffer and
> >>> maybe triggering an early commit affects the whole processing of Kafka
> >>> Streams. The transactionality of a state store is not confined to the
> >>> state store itself, but spills over and changes the behavior of other
> >>> parts of the system. I agree with you that it is a decent compromise. I
> >>> just wanted to analyse the downsides and list the options to overcome
> >>> them. I also agree with you that all options seem quite heavy compared
> >>> with your KIP. I do not understand what you mean with "less predictable
> >>> for users", though.
> >>>
> >>>
> >>> I found the discussions about the alternatives really interesting. But
> I
> >>> also think that your plan sounds good and we should continue with it!
> >>>
> >>>
> >>> Some comments on your reply to my e-mail on June 20th:
> >>>
> >>> 3.
> >>> Ah, now, I understand the reasoning behind putting isolation level in
> >>> the state store context. Thanks! Should that also be a way to give the
> >>> the state store the opportunity to decide whether to turn on
> >>> transactions or not?
> >>> With my comment, I was more concerned about how do you know if a
> >>> checkpoint file needs to be written under EOS, if you do not have a way
> >>> to know if the state store is transactional or not. If a state store is
> >>> transactional, the checkpoint file can be written during normal
> >>> processing under EOS. If the state store is not transactional, the
> >>> checkpoint file must not be written under EOS.
> >>>
> >>> 7.
> >>> My point was about not only considering the bytes in memory in config
> >>> statestore.uncommitted.max.bytes, but also bytes that might be spilled
> >>> on disk. Basically, I was wondering whether you should remove the
> >>> "memory" in "Maximum number of memory bytes to be used to
> >>> buffer uncommitted state-store records." My thinking was that even if a
> >>> state store spills uncommitted bytes to disk, limiting the overall
> bytes
> >>> might make sense. Thinking about it again and considering the recent
> >>> discussions, it does not make too much sense anymore.
> >>> I like the name statestore.transaction.buffer.max.bytes that you
> proposed.
> >>>
> >>> 8.
> >>> A high-level description (without implementation details) of how Kafka
> >>> Streams will manage the commit of changelog transactions, state store
> >>> transactions and checkpointing would be great. Would be great if you
> >>> could also add some sentences about the behavior in case of a failure.
> >>> For instance how does a transactional state store recover after a
> >>> failure or what happens with the transaction buffer, etc. (that is what
> >>> I meant by "fail-over" in point 9.)
> >>>
> >>> Best,
> >>> Bruno
> >>>
> >>> On 21.06.23 18:50, Nick Telford wrote:
> >>>> Hi Bruno,
> >>>>
> >>>> 1.
> >>>> Isn't this exactly the same issue that WriteBatchWithIndex
> transactions
> >>>> have, whereby exceeding (or likely to exceed) configured memory needs
> to
> >>>> trigger an early commit?
> >>>>
> >>>> 2.
> >>>> This is one of my big concerns. Ultimately, any approach based on
> >>> cracking
> >>>> open RocksDB internals and using it in ways it's not really designed
> >>> for is
> >>>> likely to have some unforseen performance or consistency issues.
> >>>>
> >>>> 3.
> >>>> What's your motivation for removing these early commits? While not
> >>> ideal, I
> >>>> think they're a decent compromise to ensure consistency whilst
> >>> maintaining
> >>>> good and predictable performance.
> >>>> All 3 of your suggested ideas seem *very* complicated, and might
> >>> actually
> >>>> make behaviour less predictable for users as a consequence.
> >>>>
> >>>> I'm a bit concerned that the scope of this KIP is growing a bit out of
> >>>> control. While it's good to discuss ideas for future improvements, I
> >>> think
> >>>> it's important to narrow the scope down to a design that achieves the
> >>> most
> >>>> pressing objectives (constant sized restorations during dirty
> >>>> close/unexpected errors). Any design that this KIP produces can
> >>> ultimately
> >>>> be changed in the future, especially if the bulk of it is internal
> >>>> behaviour.
> >>>>
> >>>> I'm going to spend some time next week trying to re-work the original
> >>>> WriteBatchWithIndex design to remove the newTransaction() method, such
> >>> that
> >>>> it's just an implementation detail of RocksDBStore. That way, if we
> >>> want to
> >>>> replace WBWI with something in the future, like the SST file
> management
> >>>> outlined by John, then we can do so with little/no API changes.
> >>>>
> >>>> Regards,
> >>>>
> >>>> Nick
> >>>>
> >>>
> >>
> >
>

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

Posted by Bruno Cadonna <ca...@apache.org>.
Hi Nick,

Thanks for the insights! Very interesting!

As far as I understand, you want to atomically update the state store 
from the transaction buffer, flush the memtable of a state store and 
write the checkpoint not after the commit time elapsed but after the 
transaction buffer reached a size that would lead to exceeding 
statestore.transaction.buffer.max.bytes before the next commit interval 
ends.
That means, the Kafka transaction would commit every commit interval but 
the state store will only be atomically updated roughly every 
statestore.transaction.buffer.max.bytes of data. Also IQ would then only 
see new data roughly every statestore.transaction.buffer.max.bytes.
After a failure the state store needs to restore up to 
statestore.transaction.buffer.max.bytes.

Is this correct?

What about atomically updating the state store from the transaction 
buffer every commit interval and writing the checkpoint (thus, flushing 
the memtable) every configured amount of data and/or number of commit 
intervals? In such a way, we would have the same delay for records 
appearing in output topics and IQ because both would appear when the 
Kafka transaction is committed. However, after a failure the state store 
still needs to restore up to statestore.transaction.buffer.max.bytes and 
it might restore data that is already in the state store because the 
checkpoint lags behind the last stable offset (i.e. the last committed 
offset) of the changelog topics. Restoring data that is already in the 
state store is idempotent, so eos should not violated.
This solution needs at least one new config to specify when a checkpoint 
should be written.



A small correction to your previous e-mail that does not change anything 
you said: Under alos the default commit interval is 30 seconds, not five 
seconds.


Best,
Bruno


On 01.07.23 12:37, Nick Telford wrote:
> Hi everyone,
> 
> I've begun performance testing my branch on our staging environment,
> putting it through its paces in our non-trivial application. I'm already
> observing the same increased flush rate that we saw the last time we
> attempted to use a version of this KIP, but this time, I think I know why.
> 
> Pre-KIP-892, StreamTask#postCommit, which is called at the end of the Task
> commit process, has the following behaviour:
> 
>     - Under ALOS: checkpoint the state stores. This includes
>     flushing memtables in RocksDB. This is acceptable because the default
>     commit.interval.ms is 5 seconds, so forcibly flushing memtables every 5
>     seconds is acceptable for most applications.
>     - Under EOS: checkpointing is not done, *unless* it's being forced, due
>     to e.g. the Task closing or being revoked. This means that under normal
>     processing conditions, the state stores will not be checkpointed, and will
>     not have memtables flushed at all , unless RocksDB decides to flush them on
>     its own. Checkpointing stores and force-flushing their memtables is only
>     done when a Task is being closed.
> 
> Under EOS, KIP-892 needs to checkpoint stores on at least *some* normal
> Task commits, in order to write the RocksDB transaction buffers to the
> state stores, and to ensure the offsets are synced to disk to prevent
> restores from getting out of hand. Consequently, my current implementation
> calls maybeCheckpoint on *every* Task commit, which is far too frequent.
> This causes checkpoints every 10,000 records, which is a change in flush
> behaviour, potentially causing performance problems for some applications.
> 
> I'm looking into possible solutions, and I'm currently leaning towards
> using the statestore.transaction.buffer.max.bytes configuration to
> checkpoint Tasks once we are likely to exceed it. This would complement the
> existing "early Task commit" functionality that this configuration
> provides, in the following way:
> 
>     - Currently, we use statestore.transaction.buffer.max.bytes to force an
>     early Task commit if processing more records would cause our state store
>     transactions to exceed the memory assigned to them.
>     - New functionality: when a Task *does* commit, we will not checkpoint
>     the stores (and hence flush the transaction buffers) unless we expect to
>     cross the statestore.transaction.buffer.max.bytes threshold before the next
>     commit
> 
> I'm also open to suggestions.
> 
> Regards,
> Nick
> 
> On Thu, 22 Jun 2023 at 14:06, Nick Telford <ni...@gmail.com> wrote:
> 
>> Hi Bruno!
>>
>> 3.
>> By "less predictable for users", I meant in terms of understanding the
>> performance profile under various circumstances. The more complex the
>> solution, the more difficult it would be for users to understand the
>> performance they see. For example, spilling records to disk when the
>> transaction buffer reaches a threshold would, I expect, reduce write
>> throughput. This reduction in write throughput could be unexpected, and
>> potentially difficult to diagnose/understand for users.
>> At the moment, I think the "early commit" concept is relatively
>> straightforward; it's easy to document, and conceptually fairly obvious to
>> users. We could probably add a metric to make it easier to understand when
>> it happens though.
>>
>> 3. (the second one)
>> The IsolationLevel is *essentially* an indirect way of telling StateStores
>> whether they should be transactional. READ_COMMITTED essentially requires
>> transactions, because it dictates that two threads calling
>> `newTransaction()` should not see writes from the other transaction until
>> they have been committed. With READ_UNCOMMITTED, all bets are off, and
>> stores can allow threads to observe written records at any time, which is
>> essentially "no transactions". That said, StateStores are free to implement
>> these guarantees however they can, which is a bit more relaxed than
>> dictating "you must use transactions". For example, with RocksDB we would
>> implement these as READ_COMMITTED == WBWI-based "transactions",
>> READ_UNCOMMITTED == direct writes to the database. But with other storage
>> engines, it might be preferable to *always* use transactions, even when
>> unnecessary; or there may be storage engines that don't provide
>> transactions, but the isolation guarantees can be met using a different
>> technique.
>> My idea was to try to keep the StateStore interface as loosely coupled
>> from the Streams engine as possible, to give implementers more freedom, and
>> reduce the amount of internal knowledge required.
>> That said, I understand that "IsolationLevel" might not be the right
>> abstraction, and we can always make it much more explicit if required, e.g.
>> boolean transactional()
>>
>> 7-8.
>> I can make these changes either later today or tomorrow.
>>
>> Small update:
>> I've rebased my branch on trunk and fixed a bunch of issues that needed
>> addressing. Currently, all the tests pass, which is promising, but it will
>> need to undergo some performance testing. I haven't (yet) worked on
>> removing the `newTransaction()` stuff, but I would expect that,
>> behaviourally, it should make no difference. The branch is available at
>> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
>> interested in taking an early look.
>>
>> Regards,
>> Nick
>>
>> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org> wrote:
>>
>>> Hi Nick,
>>>
>>> 1.
>>> Yeah, I agree with you. That was actually also my point. I understood
>>> that John was proposing the ingestion path as a way to avoid the early
>>> commits. Probably, I misinterpreted the intent.
>>>
>>> 2.
>>> I agree with John here, that actually it is public API. My question is
>>> how this usage pattern affects normal processing.
>>>
>>> 3.
>>> My concern is that checking for the size of the transaction buffer and
>>> maybe triggering an early commit affects the whole processing of Kafka
>>> Streams. The transactionality of a state store is not confined to the
>>> state store itself, but spills over and changes the behavior of other
>>> parts of the system. I agree with you that it is a decent compromise. I
>>> just wanted to analyse the downsides and list the options to overcome
>>> them. I also agree with you that all options seem quite heavy compared
>>> with your KIP. I do not understand what you mean with "less predictable
>>> for users", though.
>>>
>>>
>>> I found the discussions about the alternatives really interesting. But I
>>> also think that your plan sounds good and we should continue with it!
>>>
>>>
>>> Some comments on your reply to my e-mail on June 20th:
>>>
>>> 3.
>>> Ah, now, I understand the reasoning behind putting isolation level in
>>> the state store context. Thanks! Should that also be a way to give the
>>> the state store the opportunity to decide whether to turn on
>>> transactions or not?
>>> With my comment, I was more concerned about how do you know if a
>>> checkpoint file needs to be written under EOS, if you do not have a way
>>> to know if the state store is transactional or not. If a state store is
>>> transactional, the checkpoint file can be written during normal
>>> processing under EOS. If the state store is not transactional, the
>>> checkpoint file must not be written under EOS.
>>>
>>> 7.
>>> My point was about not only considering the bytes in memory in config
>>> statestore.uncommitted.max.bytes, but also bytes that might be spilled
>>> on disk. Basically, I was wondering whether you should remove the
>>> "memory" in "Maximum number of memory bytes to be used to
>>> buffer uncommitted state-store records." My thinking was that even if a
>>> state store spills uncommitted bytes to disk, limiting the overall bytes
>>> might make sense. Thinking about it again and considering the recent
>>> discussions, it does not make too much sense anymore.
>>> I like the name statestore.transaction.buffer.max.bytes that you proposed.
>>>
>>> 8.
>>> A high-level description (without implementation details) of how Kafka
>>> Streams will manage the commit of changelog transactions, state store
>>> transactions and checkpointing would be great. Would be great if you
>>> could also add some sentences about the behavior in case of a failure.
>>> For instance how does a transactional state store recover after a
>>> failure or what happens with the transaction buffer, etc. (that is what
>>> I meant by "fail-over" in point 9.)
>>>
>>> Best,
>>> Bruno
>>>
>>> On 21.06.23 18:50, Nick Telford wrote:
>>>> Hi Bruno,
>>>>
>>>> 1.
>>>> Isn't this exactly the same issue that WriteBatchWithIndex transactions
>>>> have, whereby exceeding (or likely to exceed) configured memory needs to
>>>> trigger an early commit?
>>>>
>>>> 2.
>>>> This is one of my big concerns. Ultimately, any approach based on
>>> cracking
>>>> open RocksDB internals and using it in ways it's not really designed
>>> for is
>>>> likely to have some unforseen performance or consistency issues.
>>>>
>>>> 3.
>>>> What's your motivation for removing these early commits? While not
>>> ideal, I
>>>> think they're a decent compromise to ensure consistency whilst
>>> maintaining
>>>> good and predictable performance.
>>>> All 3 of your suggested ideas seem *very* complicated, and might
>>> actually
>>>> make behaviour less predictable for users as a consequence.
>>>>
>>>> I'm a bit concerned that the scope of this KIP is growing a bit out of
>>>> control. While it's good to discuss ideas for future improvements, I
>>> think
>>>> it's important to narrow the scope down to a design that achieves the
>>> most
>>>> pressing objectives (constant sized restorations during dirty
>>>> close/unexpected errors). Any design that this KIP produces can
>>> ultimately
>>>> be changed in the future, especially if the bulk of it is internal
>>>> behaviour.
>>>>
>>>> I'm going to spend some time next week trying to re-work the original
>>>> WriteBatchWithIndex design to remove the newTransaction() method, such
>>> that
>>>> it's just an implementation detail of RocksDBStore. That way, if we
>>> want to
>>>> replace WBWI with something in the future, like the SST file management
>>>> outlined by John, then we can do so with little/no API changes.
>>>>
>>>> Regards,
>>>>
>>>> Nick
>>>>
>>>
>>
> 

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

Posted by Nick Telford <ni...@gmail.com>.
Hi everyone,

I've begun performance testing my branch on our staging environment,
putting it through its paces in our non-trivial application. I'm already
observing the same increased flush rate that we saw the last time we
attempted to use a version of this KIP, but this time, I think I know why.

Pre-KIP-892, StreamTask#postCommit, which is called at the end of the Task
commit process, has the following behaviour:

   - Under ALOS: checkpoint the state stores. This includes
   flushing memtables in RocksDB. This is acceptable because the default
   commit.interval.ms is 5 seconds, so forcibly flushing memtables every 5
   seconds is acceptable for most applications.
   - Under EOS: checkpointing is not done, *unless* it's being forced, due
   to e.g. the Task closing or being revoked. This means that under normal
   processing conditions, the state stores will not be checkpointed, and will
   not have memtables flushed at all , unless RocksDB decides to flush them on
   its own. Checkpointing stores and force-flushing their memtables is only
   done when a Task is being closed.

Under EOS, KIP-892 needs to checkpoint stores on at least *some* normal
Task commits, in order to write the RocksDB transaction buffers to the
state stores, and to ensure the offsets are synced to disk to prevent
restores from getting out of hand. Consequently, my current implementation
calls maybeCheckpoint on *every* Task commit, which is far too frequent.
This causes checkpoints every 10,000 records, which is a change in flush
behaviour, potentially causing performance problems for some applications.

I'm looking into possible solutions, and I'm currently leaning towards
using the statestore.transaction.buffer.max.bytes configuration to
checkpoint Tasks once we are likely to exceed it. This would complement the
existing "early Task commit" functionality that this configuration
provides, in the following way:

   - Currently, we use statestore.transaction.buffer.max.bytes to force an
   early Task commit if processing more records would cause our state store
   transactions to exceed the memory assigned to them.
   - New functionality: when a Task *does* commit, we will not checkpoint
   the stores (and hence flush the transaction buffers) unless we expect to
   cross the statestore.transaction.buffer.max.bytes threshold before the next
   commit

I'm also open to suggestions.

Regards,
Nick

On Thu, 22 Jun 2023 at 14:06, Nick Telford <ni...@gmail.com> wrote:

> Hi Bruno!
>
> 3.
> By "less predictable for users", I meant in terms of understanding the
> performance profile under various circumstances. The more complex the
> solution, the more difficult it would be for users to understand the
> performance they see. For example, spilling records to disk when the
> transaction buffer reaches a threshold would, I expect, reduce write
> throughput. This reduction in write throughput could be unexpected, and
> potentially difficult to diagnose/understand for users.
> At the moment, I think the "early commit" concept is relatively
> straightforward; it's easy to document, and conceptually fairly obvious to
> users. We could probably add a metric to make it easier to understand when
> it happens though.
>
> 3. (the second one)
> The IsolationLevel is *essentially* an indirect way of telling StateStores
> whether they should be transactional. READ_COMMITTED essentially requires
> transactions, because it dictates that two threads calling
> `newTransaction()` should not see writes from the other transaction until
> they have been committed. With READ_UNCOMMITTED, all bets are off, and
> stores can allow threads to observe written records at any time, which is
> essentially "no transactions". That said, StateStores are free to implement
> these guarantees however they can, which is a bit more relaxed than
> dictating "you must use transactions". For example, with RocksDB we would
> implement these as READ_COMMITTED == WBWI-based "transactions",
> READ_UNCOMMITTED == direct writes to the database. But with other storage
> engines, it might be preferable to *always* use transactions, even when
> unnecessary; or there may be storage engines that don't provide
> transactions, but the isolation guarantees can be met using a different
> technique.
> My idea was to try to keep the StateStore interface as loosely coupled
> from the Streams engine as possible, to give implementers more freedom, and
> reduce the amount of internal knowledge required.
> That said, I understand that "IsolationLevel" might not be the right
> abstraction, and we can always make it much more explicit if required, e.g.
> boolean transactional()
>
> 7-8.
> I can make these changes either later today or tomorrow.
>
> Small update:
> I've rebased my branch on trunk and fixed a bunch of issues that needed
> addressing. Currently, all the tests pass, which is promising, but it will
> need to undergo some performance testing. I haven't (yet) worked on
> removing the `newTransaction()` stuff, but I would expect that,
> behaviourally, it should make no difference. The branch is available at
> https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is
> interested in taking an early look.
>
> Regards,
> Nick
>
> On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org> wrote:
>
>> Hi Nick,
>>
>> 1.
>> Yeah, I agree with you. That was actually also my point. I understood
>> that John was proposing the ingestion path as a way to avoid the early
>> commits. Probably, I misinterpreted the intent.
>>
>> 2.
>> I agree with John here, that actually it is public API. My question is
>> how this usage pattern affects normal processing.
>>
>> 3.
>> My concern is that checking for the size of the transaction buffer and
>> maybe triggering an early commit affects the whole processing of Kafka
>> Streams. The transactionality of a state store is not confined to the
>> state store itself, but spills over and changes the behavior of other
>> parts of the system. I agree with you that it is a decent compromise. I
>> just wanted to analyse the downsides and list the options to overcome
>> them. I also agree with you that all options seem quite heavy compared
>> with your KIP. I do not understand what you mean with "less predictable
>> for users", though.
>>
>>
>> I found the discussions about the alternatives really interesting. But I
>> also think that your plan sounds good and we should continue with it!
>>
>>
>> Some comments on your reply to my e-mail on June 20th:
>>
>> 3.
>> Ah, now, I understand the reasoning behind putting isolation level in
>> the state store context. Thanks! Should that also be a way to give the
>> the state store the opportunity to decide whether to turn on
>> transactions or not?
>> With my comment, I was more concerned about how do you know if a
>> checkpoint file needs to be written under EOS, if you do not have a way
>> to know if the state store is transactional or not. If a state store is
>> transactional, the checkpoint file can be written during normal
>> processing under EOS. If the state store is not transactional, the
>> checkpoint file must not be written under EOS.
>>
>> 7.
>> My point was about not only considering the bytes in memory in config
>> statestore.uncommitted.max.bytes, but also bytes that might be spilled
>> on disk. Basically, I was wondering whether you should remove the
>> "memory" in "Maximum number of memory bytes to be used to
>> buffer uncommitted state-store records." My thinking was that even if a
>> state store spills uncommitted bytes to disk, limiting the overall bytes
>> might make sense. Thinking about it again and considering the recent
>> discussions, it does not make too much sense anymore.
>> I like the name statestore.transaction.buffer.max.bytes that you proposed.
>>
>> 8.
>> A high-level description (without implementation details) of how Kafka
>> Streams will manage the commit of changelog transactions, state store
>> transactions and checkpointing would be great. Would be great if you
>> could also add some sentences about the behavior in case of a failure.
>> For instance how does a transactional state store recover after a
>> failure or what happens with the transaction buffer, etc. (that is what
>> I meant by "fail-over" in point 9.)
>>
>> Best,
>> Bruno
>>
>> On 21.06.23 18:50, Nick Telford wrote:
>> > Hi Bruno,
>> >
>> > 1.
>> > Isn't this exactly the same issue that WriteBatchWithIndex transactions
>> > have, whereby exceeding (or likely to exceed) configured memory needs to
>> > trigger an early commit?
>> >
>> > 2.
>> > This is one of my big concerns. Ultimately, any approach based on
>> cracking
>> > open RocksDB internals and using it in ways it's not really designed
>> for is
>> > likely to have some unforseen performance or consistency issues.
>> >
>> > 3.
>> > What's your motivation for removing these early commits? While not
>> ideal, I
>> > think they're a decent compromise to ensure consistency whilst
>> maintaining
>> > good and predictable performance.
>> > All 3 of your suggested ideas seem *very* complicated, and might
>> actually
>> > make behaviour less predictable for users as a consequence.
>> >
>> > I'm a bit concerned that the scope of this KIP is growing a bit out of
>> > control. While it's good to discuss ideas for future improvements, I
>> think
>> > it's important to narrow the scope down to a design that achieves the
>> most
>> > pressing objectives (constant sized restorations during dirty
>> > close/unexpected errors). Any design that this KIP produces can
>> ultimately
>> > be changed in the future, especially if the bulk of it is internal
>> > behaviour.
>> >
>> > I'm going to spend some time next week trying to re-work the original
>> > WriteBatchWithIndex design to remove the newTransaction() method, such
>> that
>> > it's just an implementation detail of RocksDBStore. That way, if we
>> want to
>> > replace WBWI with something in the future, like the SST file management
>> > outlined by John, then we can do so with little/no API changes.
>> >
>> > Regards,
>> >
>> > Nick
>> >
>>
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Bruno!

3.
By "less predictable for users", I meant in terms of understanding the
performance profile under various circumstances. The more complex the
solution, the more difficult it would be for users to understand the
performance they see. For example, spilling records to disk when the
transaction buffer reaches a threshold would, I expect, reduce write
throughput. This reduction in write throughput could be unexpected, and
potentially difficult to diagnose/understand for users.
At the moment, I think the "early commit" concept is relatively
straightforward; it's easy to document, and conceptually fairly obvious to
users. We could probably add a metric to make it easier to understand when
it happens though.

3. (the second one)
The IsolationLevel is *essentially* an indirect way of telling StateStores
whether they should be transactional. READ_COMMITTED essentially requires
transactions, because it dictates that two threads calling
`newTransaction()` should not see writes from the other transaction until
they have been committed. With READ_UNCOMMITTED, all bets are off, and
stores can allow threads to observe written records at any time, which is
essentially "no transactions". That said, StateStores are free to implement
these guarantees however they can, which is a bit more relaxed than
dictating "you must use transactions". For example, with RocksDB we would
implement these as READ_COMMITTED == WBWI-based "transactions",
READ_UNCOMMITTED == direct writes to the database. But with other storage
engines, it might be preferable to *always* use transactions, even when
unnecessary; or there may be storage engines that don't provide
transactions, but the isolation guarantees can be met using a different
technique.
My idea was to try to keep the StateStore interface as loosely coupled from
the Streams engine as possible, to give implementers more freedom, and
reduce the amount of internal knowledge required.
That said, I understand that "IsolationLevel" might not be the right
abstraction, and we can always make it much more explicit if required, e.g.
boolean transactional()

7-8.
I can make these changes either later today or tomorrow.

Small update:
I've rebased my branch on trunk and fixed a bunch of issues that needed
addressing. Currently, all the tests pass, which is promising, but it will
need to undergo some performance testing. I haven't (yet) worked on
removing the `newTransaction()` stuff, but I would expect that,
behaviourally, it should make no difference. The branch is available at
https://github.com/nicktelford/kafka/tree/KIP-892-c if anyone is interested
in taking an early look.

Regards,
Nick

On Thu, 22 Jun 2023 at 11:59, Bruno Cadonna <ca...@apache.org> wrote:

> Hi Nick,
>
> 1.
> Yeah, I agree with you. That was actually also my point. I understood
> that John was proposing the ingestion path as a way to avoid the early
> commits. Probably, I misinterpreted the intent.
>
> 2.
> I agree with John here, that actually it is public API. My question is
> how this usage pattern affects normal processing.
>
> 3.
> My concern is that checking for the size of the transaction buffer and
> maybe triggering an early commit affects the whole processing of Kafka
> Streams. The transactionality of a state store is not confined to the
> state store itself, but spills over and changes the behavior of other
> parts of the system. I agree with you that it is a decent compromise. I
> just wanted to analyse the downsides and list the options to overcome
> them. I also agree with you that all options seem quite heavy compared
> with your KIP. I do not understand what you mean with "less predictable
> for users", though.
>
>
> I found the discussions about the alternatives really interesting. But I
> also think that your plan sounds good and we should continue with it!
>
>
> Some comments on your reply to my e-mail on June 20th:
>
> 3.
> Ah, now, I understand the reasoning behind putting isolation level in
> the state store context. Thanks! Should that also be a way to give the
> the state store the opportunity to decide whether to turn on
> transactions or not?
> With my comment, I was more concerned about how do you know if a
> checkpoint file needs to be written under EOS, if you do not have a way
> to know if the state store is transactional or not. If a state store is
> transactional, the checkpoint file can be written during normal
> processing under EOS. If the state store is not transactional, the
> checkpoint file must not be written under EOS.
>
> 7.
> My point was about not only considering the bytes in memory in config
> statestore.uncommitted.max.bytes, but also bytes that might be spilled
> on disk. Basically, I was wondering whether you should remove the
> "memory" in "Maximum number of memory bytes to be used to
> buffer uncommitted state-store records." My thinking was that even if a
> state store spills uncommitted bytes to disk, limiting the overall bytes
> might make sense. Thinking about it again and considering the recent
> discussions, it does not make too much sense anymore.
> I like the name statestore.transaction.buffer.max.bytes that you proposed.
>
> 8.
> A high-level description (without implementation details) of how Kafka
> Streams will manage the commit of changelog transactions, state store
> transactions and checkpointing would be great. Would be great if you
> could also add some sentences about the behavior in case of a failure.
> For instance how does a transactional state store recover after a
> failure or what happens with the transaction buffer, etc. (that is what
> I meant by "fail-over" in point 9.)
>
> Best,
> Bruno
>
> On 21.06.23 18:50, Nick Telford wrote:
> > Hi Bruno,
> >
> > 1.
> > Isn't this exactly the same issue that WriteBatchWithIndex transactions
> > have, whereby exceeding (or likely to exceed) configured memory needs to
> > trigger an early commit?
> >
> > 2.
> > This is one of my big concerns. Ultimately, any approach based on
> cracking
> > open RocksDB internals and using it in ways it's not really designed for
> is
> > likely to have some unforseen performance or consistency issues.
> >
> > 3.
> > What's your motivation for removing these early commits? While not
> ideal, I
> > think they're a decent compromise to ensure consistency whilst
> maintaining
> > good and predictable performance.
> > All 3 of your suggested ideas seem *very* complicated, and might actually
> > make behaviour less predictable for users as a consequence.
> >
> > I'm a bit concerned that the scope of this KIP is growing a bit out of
> > control. While it's good to discuss ideas for future improvements, I
> think
> > it's important to narrow the scope down to a design that achieves the
> most
> > pressing objectives (constant sized restorations during dirty
> > close/unexpected errors). Any design that this KIP produces can
> ultimately
> > be changed in the future, especially if the bulk of it is internal
> > behaviour.
> >
> > I'm going to spend some time next week trying to re-work the original
> > WriteBatchWithIndex design to remove the newTransaction() method, such
> that
> > it's just an implementation detail of RocksDBStore. That way, if we want
> to
> > replace WBWI with something in the future, like the SST file management
> > outlined by John, then we can do so with little/no API changes.
> >
> > Regards,
> >
> > Nick
> >
>

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

Posted by Bruno Cadonna <ca...@apache.org>.
Hi Nick,

1.
Yeah, I agree with you. That was actually also my point. I understood 
that John was proposing the ingestion path as a way to avoid the early 
commits. Probably, I misinterpreted the intent.

2.
I agree with John here, that actually it is public API. My question is 
how this usage pattern affects normal processing.

3.
My concern is that checking for the size of the transaction buffer and 
maybe triggering an early commit affects the whole processing of Kafka 
Streams. The transactionality of a state store is not confined to the 
state store itself, but spills over and changes the behavior of other 
parts of the system. I agree with you that it is a decent compromise. I 
just wanted to analyse the downsides and list the options to overcome 
them. I also agree with you that all options seem quite heavy compared 
with your KIP. I do not understand what you mean with "less predictable 
for users", though.


I found the discussions about the alternatives really interesting. But I 
also think that your plan sounds good and we should continue with it!


Some comments on your reply to my e-mail on June 20th:

3.
Ah, now, I understand the reasoning behind putting isolation level in 
the state store context. Thanks! Should that also be a way to give the 
the state store the opportunity to decide whether to turn on 
transactions or not?
With my comment, I was more concerned about how do you know if a 
checkpoint file needs to be written under EOS, if you do not have a way 
to know if the state store is transactional or not. If a state store is 
transactional, the checkpoint file can be written during normal 
processing under EOS. If the state store is not transactional, the 
checkpoint file must not be written under EOS.

7.
My point was about not only considering the bytes in memory in config 
statestore.uncommitted.max.bytes, but also bytes that might be spilled 
on disk. Basically, I was wondering whether you should remove the 
"memory" in "Maximum number of memory bytes to be used to
buffer uncommitted state-store records." My thinking was that even if a 
state store spills uncommitted bytes to disk, limiting the overall bytes 
might make sense. Thinking about it again and considering the recent 
discussions, it does not make too much sense anymore.
I like the name statestore.transaction.buffer.max.bytes that you proposed.

8.
A high-level description (without implementation details) of how Kafka 
Streams will manage the commit of changelog transactions, state store 
transactions and checkpointing would be great. Would be great if you 
could also add some sentences about the behavior in case of a failure. 
For instance how does a transactional state store recover after a 
failure or what happens with the transaction buffer, etc. (that is what 
I meant by "fail-over" in point 9.)

Best,
Bruno

On 21.06.23 18:50, Nick Telford wrote:
> Hi Bruno,
> 
> 1.
> Isn't this exactly the same issue that WriteBatchWithIndex transactions
> have, whereby exceeding (or likely to exceed) configured memory needs to
> trigger an early commit?
> 
> 2.
> This is one of my big concerns. Ultimately, any approach based on cracking
> open RocksDB internals and using it in ways it's not really designed for is
> likely to have some unforseen performance or consistency issues.
> 
> 3.
> What's your motivation for removing these early commits? While not ideal, I
> think they're a decent compromise to ensure consistency whilst maintaining
> good and predictable performance.
> All 3 of your suggested ideas seem *very* complicated, and might actually
> make behaviour less predictable for users as a consequence.
> 
> I'm a bit concerned that the scope of this KIP is growing a bit out of
> control. While it's good to discuss ideas for future improvements, I think
> it's important to narrow the scope down to a design that achieves the most
> pressing objectives (constant sized restorations during dirty
> close/unexpected errors). Any design that this KIP produces can ultimately
> be changed in the future, especially if the bulk of it is internal
> behaviour.
> 
> I'm going to spend some time next week trying to re-work the original
> WriteBatchWithIndex design to remove the newTransaction() method, such that
> it's just an implementation detail of RocksDBStore. That way, if we want to
> replace WBWI with something in the future, like the SST file management
> outlined by John, then we can do so with little/no API changes.
> 
> Regards,
> 
> Nick
> 

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Bruno,

1.
Isn't this exactly the same issue that WriteBatchWithIndex transactions
have, whereby exceeding (or likely to exceed) configured memory needs to
trigger an early commit?

2.
This is one of my big concerns. Ultimately, any approach based on cracking
open RocksDB internals and using it in ways it's not really designed for is
likely to have some unforseen performance or consistency issues.

3.
What's your motivation for removing these early commits? While not ideal, I
think they're a decent compromise to ensure consistency whilst maintaining
good and predictable performance.
All 3 of your suggested ideas seem *very* complicated, and might actually
make behaviour less predictable for users as a consequence.

I'm a bit concerned that the scope of this KIP is growing a bit out of
control. While it's good to discuss ideas for future improvements, I think
it's important to narrow the scope down to a design that achieves the most
pressing objectives (constant sized restorations during dirty
close/unexpected errors). Any design that this KIP produces can ultimately
be changed in the future, especially if the bulk of it is internal
behaviour.

I'm going to spend some time next week trying to re-work the original
WriteBatchWithIndex design to remove the newTransaction() method, such that
it's just an implementation detail of RocksDBStore. That way, if we want to
replace WBWI with something in the future, like the SST file management
outlined by John, then we can do so with little/no API changes.

Regards,

Nick

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

Posted by Bruno Cadonna <ca...@apache.org>.
Hi John,
Hi Nick,

Thanks for the interesting ideas!

Here my comments.

1.
It is not clear to me what happens if the cache exceeds its configured 
size between two commits? Currently, the cache evicts its entries and 
writes dirty entries to the state store. Should the cache write those 
dirty entries to the SST files directly? But then, how would the task 
that owns the state store or an interactive query query those SST files? 
We cannot ingest the SST files into the state store yet, because we 
first need to commit the Kafka transaction containing those entries.
Nick, I think you had a similar question a couple of messages ago.

2.
I am wondering how ingesting external SST files affects compaction and 
whether Streams will experience more write stalls.

3.
I would really like to get rid of the extra commits triggered by 
exceeding the configured size of the transaction as proposed in Nick's 
KIP. Avoiding this extra commits would allow us to put all the 
transactional logic behind the state store interface without the need to 
check for memory sizes before starting to process a record.
However, at the moment, I do not see how to achieve this without the 
possibility to spill records to disk. John's (SST ingest) and Alex's 
(KIP-844) ideas go in that direction. Alex's idea has a performance 
issue and with John's idea it is not clear to me how to query records in 
SST files.

I see three options to improve on those two ideas:

a. After the Kafka transaction is committed ingest SST files produced by 
the temporary state store proposed by Alex.
That might improve performance since in this way we update the state 
store in batches. However, I am not sure how performant clearing the 
temporary state store is.

b. Implementing querying of SST files in Java.
This seems quite some work. However, the implementation does not need to 
be super performant because the data to query might not be too much and 
querying the SST files might also not happen too often.

c. Make WriteBatchWithIndex spill to disk when configured size is exceeded.
That would require a change in RocksDB. For that, we need to either fork 
RocksDB or to get this change in one of the next releases. The latter 
would also imply to upgrade the RocksDB version in Kafka Streams which 
might get us backwards compatibility issues.


Best,
Bruno


On 20.06.23 23:43, Nick Telford wrote:
> Here's what I'm thinking: based on Bruno's earlier feedback, I'm going to
> try to simplify my original design down such that it needs no/minimal
> changes to the public interface.
> 
> If that succeeds, then it should also be possible to transparently
> implement the "no memtables" solution as a performance optimization when
> the record cache is enabled. I consider this approach only an optimisation,
> because of the need to still support stores with the cache disabled.
> 
> For that reason, I think the "no memtables" approach would probably best be
> suited as a follow-up KIP, but that we keep it in mind during the design of
> this one.
> 
> What do you think?
> 
> Regards,
> Nick
> 
> 
> On Tue, 20 Jun 2023, 22:26 John Roesler, <vv...@apache.org> wrote:
> 
>> Oh, that's a good point.
>>
>> On the topic of a behavioral switch for disabled caches, the typical use
>> case for disabling the cache is to cause each individual update to
>> propagate down the topology, so another thought might be to just go
>> ahead and add the memory we would have used for the memtables to the
>> cache size, but if people did disable the cache entirely, then we could
>> still go ahead and forward the records on each write?
>>
>> I know that Guozhang was also proposing for a while to actually decouple
>> caching and forwarding, which might provide a way to side-step this
>> dilemma (i.e., we just always forward and only apply the cache to state
>> and changelog writes).
>>
>> By the way, I'm basing my statement on why you'd disable caches on
>> memory, but also on the guidance here:
>>
>> https://docs.confluent.io/platform/current/streams/developer-guide/memory-mgmt.html
>> . That doc also contains a section on how to bound the total memory
>> usage across RocksDB memtables, which points to another benefit of
>> disabling memtables and managing the write buffer ourselves (simplified
>> memory configuration).
>>
>> Thanks,
>> -John
>>
>> On 6/20/23 16:05, Nick Telford wrote:
>>> Potentially we could just go the memorable with Rocks WriteBatches route
>> if
>>> the cache is disabled?
>>>
>>> On Tue, 20 Jun 2023, 22:00 John Roesler, <jo...@vvcephei.org> wrote:
>>>
>>>> Touché!
>>>>
>>>> Ok, I agree that figuring out the case of a disabled cache would be
>>>> non-trivial. Ingesting single-record SST files will probably not be
>>>> performant, but benchmarking may prove different. Or maybe we can have
>>>> some reserved cache space on top of the user-configured cache, which we
>>>> would have reclaimed from the memtable space. Or some other, more
>>>> creative solution.
>>>>
>>>> Thanks,
>>>> -John
>>>>
>>>> On 6/20/23 15:30, Nick Telford wrote:
>>>>>> Note that users can disable the cache, which would still be
>>>>> ok, I think. We wouldn't ingest the SST files on every record, but just
>>>>> append to them and only ingest them on commit, when we're already
>>>>> waiting for acks and a RocksDB commit.
>>>>>
>>>>> In this case, how would uncommitted records be read by joins?
>>>>>
>>>>> On Tue, 20 Jun 2023, 20:51 John Roesler, <vv...@apache.org> wrote:
>>>>>
>>>>>> Ah, sorry Nick,
>>>>>>
>>>>>> I just meant the regular heap based cache that we maintain in
>> Streams. I
>>>>>> see that it's not called "RecordCache" (my mistake).
>>>>>>
>>>>>> The actual cache is ThreadCache:
>>>>>>
>>>>>>
>>>>
>> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
>>>>>>
>>>>>> Here's the example of how we use the cache in KeyValueStore:
>>>>>>
>>>>>>
>>>>
>> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
>>>>>>
>>>>>> It's basically just an on-heap Map of records that have not yet been
>>>>>> written to the changelog or flushed into the underlying store. It gets
>>>>>> flushed when the total cache size exceeds `cache.max.bytes.buffering`
>> or
>>>>>> the `commit.interval.ms` elapses.
>>>>>>
>>>>>> Speaking of those configs, another benefit to this idea is that we
>> would
>>>>>> no longer need to trigger extra commits based on the size of the
>> ongoing
>>>>>> transaction. Instead, we'd just preserve the existing cache-flush
>>>>>> behavior. Note that users can disable the cache, which would still be
>>>>>> ok, I think. We wouldn't ingest the SST files on every record, but
>> just
>>>>>> append to them and only ingest them on commit, when we're already
>>>>>> waiting for acks and a RocksDB commit.
>>>>>>
>>>>>> Thanks,
>>>>>> -John
>>>>>>
>>>>>> On 6/20/23 14:09, Nick Telford wrote:
>>>>>>> Hi John,
>>>>>>>
>>>>>>> By "RecordCache", do you mean the RocksDB "WriteBatch"? I can't find
>>>> any
>>>>>>> class called "RecordCache"...
>>>>>>>
>>>>>>> Cheers,
>>>>>>>
>>>>>>> Nick
>>>>>>>
>>>>>>> On Tue, 20 Jun 2023 at 19:42, John Roesler <vv...@apache.org>
>>>> wrote:
>>>>>>>
>>>>>>>> Hi Nick,
>>>>>>>>
>>>>>>>> Thanks for picking this up again!
>>>>>>>>
>>>>>>>> I did have one new thought over the intervening months, which I'd
>> like
>>>>>>>> your take on.
>>>>>>>>
>>>>>>>> What if, instead of using the RocksDB atomic write primitive at all,
>>>> we
>>>>>>>> instead just:
>>>>>>>> 1. disable memtables entirely
>>>>>>>> 2. directly write the RecordCache into SST files when we flush
>>>>>>>> 3. atomically ingest the SST file(s) into RocksDB when we get the
>> ACK
>>>>>>>> from the changelog (see
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>
>> https://github.com/EighteenZi/rocksdb_wiki/blob/master/Creating-and-Ingesting-SST-files.md
>>>>>>>> and
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>
>> https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/IngestExternalFileOptions.java
>>>>>>>> and
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>
>> https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L1413-L1429
>>>>>>>> )
>>>>>>>> 4. track the changelog offsets either in another CF or the same CF
>>>> with
>>>>>>>> a reserved key, either of which will make the changelog offset
>> update
>>>>>>>> atomic with the file ingestions
>>>>>>>>
>>>>>>>> I suspect this'll have a number of benefits:
>>>>>>>> * writes to RocksDB will always be atomic
>>>>>>>> * we don't fragment memory between the RecordCache and the memtables
>>>>>>>> * RecordCache gives far higher performance than memtable for reads
>> and
>>>>>>>> writes
>>>>>>>> * we don't need any new "transaction" concepts or memory bound
>> configs
>>>>>>>>
>>>>>>>> What do you think?
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> -John
>>>>>>>>
>>>>>>>> On 6/20/23 10:51, Nick Telford wrote:
>>>>>>>>> Hi Bruno,
>>>>>>>>>
>>>>>>>>> Thanks for reviewing the KIP. It's been a long road, I started
>>>> working
>>>>>> on
>>>>>>>>> this more than a year ago, and most of the time in the last 6
>> months
>>>>>> has
>>>>>>>>> been spent on the "Atomic Checkpointing" stuff that's been benched,
>>>> so
>>>>>>>> some
>>>>>>>>> of the reasoning behind some of my decisions have been lost, but
>> I'll
>>>>>> do
>>>>>>>> my
>>>>>>>>> best to reconstruct them.
>>>>>>>>>
>>>>>>>>> 1.
>>>>>>>>> IIRC, this was the initial approach I tried. I don't remember the
>>>> exact
>>>>>>>>> reasons I changed it to use a separate "view" of the StateStore
>> that
>>>>>>>>> encapsulates the transaction, but I believe it had something to do
>>>> with
>>>>>>>>> concurrent access to the StateStore from Interactive Query threads.
>>>>>> Reads
>>>>>>>>> from interactive queries need to be isolated from the currently
>>>> ongoing
>>>>>>>>> transaction, both for consistency (so interactive queries don't
>>>> observe
>>>>>>>>> changes that are subsequently rolled-back), but also to prevent
>>>>>> Iterators
>>>>>>>>> opened by an interactive query from being closed and invalidated by
>>>> the
>>>>>>>>> StreamThread when it commits the transaction, which causes your
>>>>>>>> interactive
>>>>>>>>> queries to crash.
>>>>>>>>>
>>>>>>>>> Another reason I believe I implemented it this way was a separation
>>>> of
>>>>>>>>> concerns. Recall that newTransaction() originally created an object
>>>> of
>>>>>>>> type
>>>>>>>>> Transaction, not StateStore. My intent was to improve the
>> type-safety
>>>>>> of
>>>>>>>>> the API, in an effort to ensure Transactions weren't used
>>>> incorrectly.
>>>>>>>>> Unfortunately, this didn't pan out, but newTransaction() remained.
>>>>>>>>>
>>>>>>>>> Finally, this had the added benefit that implementations could
>> easily
>>>>>> add
>>>>>>>>> support for transactions *without* re-writing their existing,
>>>>>>>>> non-transactional implementation. I think this can be a benefit
>> both
>>>>>> for
>>>>>>>>> implementers of custom StateStores, but also for anyone extending
>>>>>>>>> RocksDbStore, as they can rely on the existing access methods
>> working
>>>>>> how
>>>>>>>>> they expect them to.
>>>>>>>>>
>>>>>>>>> I'm not too happy with the way the current design has panned out,
>> so
>>>>>> I'm
>>>>>>>>> open to ideas on how to improve it. Key to this is finding some way
>>>> to
>>>>>>>>> ensure that reads from Interactive Query threads are properly
>>>> isolated
>>>>>>>> from
>>>>>>>>> the transaction, *without* the performance overhead of checking
>> which
>>>>>>>>> thread the method is being called from on every access.
>>>>>>>>>
>>>>>>>>> As for replacing flush() with commit() - I saw no reason to add
>> this
>>>>>>>>> complexity to the KIP, unless there was a need to add arguments to
>>>> the
>>>>>>>>> flush/commit method. This need arises with Atomic Checkpointing,
>> but
>>>>>> that
>>>>>>>>> will be implemented separately, in a future KIP. Do you see a need
>>>> for
>>>>>>>> some
>>>>>>>>> arguments to the flush/commit method that I've missed? Or were you
>>>>>> simply
>>>>>>>>> suggesting a rename?
>>>>>>>>>
>>>>>>>>> 2.
>>>>>>>>> This is simply due to the practical reason that isolationLevel() is
>>>>>>>> really
>>>>>>>>> a proxy for checking if the app is under EOS. The application
>>>>>>>> configuration
>>>>>>>>> is not provided to the constructor of StateStores, but it *is*
>>>> provided
>>>>>>>> to
>>>>>>>>> init(), via StateStoreContext. For this reason, it seemed somewhat
>>>>>>>> natural
>>>>>>>>> to add it to StateStoreContext. I think this makes sense, since the
>>>>>>>>> IsolationLevel of all StateStores in an application *must* be the
>>>> same,
>>>>>>>> and
>>>>>>>>> since those stores are all initialized with the same
>>>> StateStoreContext,
>>>>>>>> it
>>>>>>>>> seems natural for that context to carry the desired IsolationLevel
>> to
>>>>>>>> use.
>>>>>>>>>
>>>>>>>>> 3.
>>>>>>>>> Using IsolationLevel instead of just passing `boolean eosEnabled`,
>>>> like
>>>>>>>>> much of the internals was an attempt to logically de-couple the
>>>>>>>> StateStore
>>>>>>>>> API from the internals of Kafka Streams. Technically, StateStores
>>>> don't
>>>>>>>>> need to know/care what processing mode the KS app is using, all
>> they
>>>>>> need
>>>>>>>>> to know is the isolation level expected of them.
>>>>>>>>>
>>>>>>>>> Having formal definitions for the expectations of the two required
>>>>>>>>> IsolationLevels allow implementers to implement transactional
>> stores
>>>>>>>>> without having to dig through the internals of Kafka Streams and
>>>>>>>> understand
>>>>>>>>> exactly how they are used. The tight coupling between state stores
>>>> and
>>>>>>>>> internal behaviour has actually significantly hindered my progress
>> on
>>>>>>>> this
>>>>>>>>> KIP, and encouraged me to avoid increasing this logical coupling as
>>>>>> much
>>>>>>>> as
>>>>>>>>> possible.
>>>>>>>>>
>>>>>>>>> This also frees implementations to satisfy those requirements in
>> any
>>>>>> way
>>>>>>>>> they choose. Transactions might not be the only/available approach
>> to
>>>>>> an
>>>>>>>>> implementation, but they might have an alternative way to satisfy
>> the
>>>>>>>>> isolation requirements. I admit that this point is more about
>>>>>> semantics,
>>>>>>>>> but "transactional" would need to be formally defined in order for
>>>>>>>>> implementers to provide a valid implementation, and these
>>>>>> IsolationLevels
>>>>>>>>> provide that formal definition.
>>>>>>>>>
>>>>>>>>> 4.
>>>>>>>>> I can remove them. I added them only as I planned to include them
>> in
>>>>>> the
>>>>>>>>> org.apache.kafka.streams.state package, as a recommended base
>>>>>>>>> implementation for all StateStores, including those implemented by
>>>>>>>> users. I
>>>>>>>>> had assumed that anything in "public" packages, such as
>>>>>>>>> org.apache.kafka.streams.state, should be included in a KIP. Is
>> that
>>>>>>>> wrong?
>>>>>>>>>
>>>>>>>>> 5.
>>>>>>>>> RocksDB provides no way to measure the actual size of a
>>>>>>>>> WriteBatch(WithIndex), so we're limited to tracking the sum total
>> of
>>>>>> the
>>>>>>>>> size of keys + values that are written to the transaction. This
>>>>>> obviously
>>>>>>>>> under-estimates the actual memory usage, because WriteBatch
>> no-doubt
>>>>>>>>> includes some record overheads, and WriteBatchWithIndex has to
>>>> maintain
>>>>>>>> an
>>>>>>>>> index.
>>>>>>>>>
>>>>>>>>> Ideally, we could trivially add a method upstream to
>>>>>> WriteBatchInterface
>>>>>>>>> that provides the exact size of the batch, but that would require
>> an
>>>>>>>>> upgrade of RocksDB, which won't happen soon. So for the time being,
>>>>>> we're
>>>>>>>>> stuck with an approximation, so I felt that the new method should
>>>>>> reflect
>>>>>>>>> that.
>>>>>>>>>
>>>>>>>>> Would you prefer the new method name ignores this constraint and
>> that
>>>>>> we
>>>>>>>>> simply make the rocks measurement more accurate in the future?
>>>>>>>>>
>>>>>>>>> 6.
>>>>>>>>> Done
>>>>>>>>>
>>>>>>>>> 7.
>>>>>>>>> Very good point. The KIP already specifically calls out memory in
>> the
>>>>>>>>> documentation of the config: "Maximum number of memory bytes to be
>>>> used
>>>>>>>> to
>>>>>>>>> buffer uncommitted state-store records." - did you have something
>>>> else
>>>>>> in
>>>>>>>>> mind?
>>>>>>>>>
>>>>>>>>> Should we also make this clearer by renaming the config property
>>>>>> itself?
>>>>>>>>> Perhaps to something like statestore.transaction.buffer.max.bytes?
>>>>>>>>>
>>>>>>>>> 8.
>>>>>>>>> OK, I can remove this. The intent here was to describe how Streams
>>>>>> itself
>>>>>>>>> will manage transaction roll-over etc. Presumably that means we
>> also
>>>>>>>> don't
>>>>>>>>> need a description of how Streams will manage the commit of
>> changelog
>>>>>>>>> transactions, state store transactions and checkpointing?
>>>>>>>>>
>>>>>>>>> 9.
>>>>>>>>> What do you mean by fail-over? Do you mean failing over an Active
>>>> Task
>>>>>> to
>>>>>>>>> an instance already hosting a Standby Task?
>>>>>>>>>
>>>>>>>>> Thanks again and sorry for the essay of a response!
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>> Nick
>>>>>>>>>
>>>>>>>>> On Tue, 20 Jun 2023 at 10:49, Bruno Cadonna <ca...@apache.org>
>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Nick,
>>>>>>>>>>
>>>>>>>>>> Thanks for the updates!
>>>>>>>>>>
>>>>>>>>>> I really appreciate that you simplified the KIP by removing some
>>>>>>>>>> aspects. As I have already told you, I think the removed aspects
>> are
>>>>>>>>>> also good ideas and we can discuss them on follow-up KIPs.
>>>>>>>>>>
>>>>>>>>>> Regarding the current KIP, I have the following feedback.
>>>>>>>>>>
>>>>>>>>>> 1.
>>>>>>>>>> Is there a good reason to add method newTransaction() to the
>>>>>> StateStore
>>>>>>>>>> interface? As far as I understand, the idea is that users of a
>> state
>>>>>>>>>> store (transactional or not) call this method at start-up and
>> after
>>>>>> each
>>>>>>>>>> commit. Since the call to newTransaction() is done in any case
>> and I
>>>>>>>>>> think it would simplify the caller code if we just start a new
>>>>>>>>>> transaction after a commit in the implementation?
>>>>>>>>>> As far as I understand, you plan to commit the transaction in the
>>>>>>>>>> flush() method. I find the idea to replace flush() with commit()
>>>>>>>>>> presented in KIP-844 an elegant solution.
>>>>>>>>>>
>>>>>>>>>> 2.
>>>>>>>>>> Why is the method to query the isolation level added to the state
>>>>>> store
>>>>>>>>>> context?
>>>>>>>>>>
>>>>>>>>>> 3.
>>>>>>>>>> Do we need all the isolation level definitions? I think it is good
>>>> to
>>>>>>>>>> know the guarantees of the transactionality of the state store.
>>>>>>>>>> However, currently, Streams guarantees that there will only be one
>>>>>>>>>> transaction that writes to the state store. Only the stream thread
>>>>>> that
>>>>>>>>>> executes the active task that owns the state store will write to
>> the
>>>>>>>>>> state store. I think it should be enough to know if the state
>> store
>>>> is
>>>>>>>>>> transactional or not. So my proposal would be to just add a method
>>>> on
>>>>>>>>>> the state store interface the returns if a state store is
>>>>>> transactional
>>>>>>>>>> or not by returning a boolean or an enum.
>>>>>>>>>>
>>>>>>>>>> 4.
>>>>>>>>>> I am wondering why AbstractTransaction and
>>>> AbstractTransactionalStore
>>>>>>>>>> are part of the KIP. They look like implementation details that
>>>> should
>>>>>>>>>> not be exposed in the public API.
>>>>>>>>>>
>>>>>>>>>> 5.
>>>>>>>>>> Why does StateStore#approximateNumUncommittedBytes() return an
>>>>>>>>>> approximate number of bytes?
>>>>>>>>>>
>>>>>>>>>> 6.
>>>>>>>>>> RocksDB is just one implementation of the state stores in Streams.
>>>>>>>>>> However, the issues regarding OOM errors might also apply to other
>>>>>>>>>> custom implementations. So in the KIP I would extract that part
>> from
>>>>>>>>>> section "RocksDB Transaction". I would also move section "RocksDB
>>>>>>>>>> Transaction" to the end of section "Proposed Changes" and handle
>> it
>>>> as
>>>>>>>>>> an example implementation for a state store.
>>>>>>>>>>
>>>>>>>>>> 7.
>>>>>>>>>> Should statestore.uncommitted.max.bytes only limit the uncommitted
>>>>>> bytes
>>>>>>>>>> or the uncommitted bytes that reside in memory? In future, other
>>>>>>>>>> transactional state store implementations might implement a buffer
>>>> for
>>>>>>>>>> uncommitted records that are able to spill records on disk. I
>> think
>>>>>>>>>> statestore.uncommitted.max.bytes needs to limit the uncommitted
>>>> bytes
>>>>>>>>>> irrespective if they reside in memory or disk. Since Streams will
>>>> use
>>>>>>>>>> this config to decide if it needs to trigger a commit, state store
>>>>>>>>>> implementations that can spill to disk will never be able to spill
>>>> to
>>>>>>>>>> disk. You would only need to change the doc of the config, if you
>>>>>> agree
>>>>>>>>>> with me.
>>>>>>>>>>
>>>>>>>>>> 8.
>>>>>>>>>> Section "Transaction Management" about the wrappers is rather a
>>>>>>>>>> implementation detail that should not be in the KIP.
>>>>>>>>>>
>>>>>>>>>> 9.
>>>>>>>>>> Could you add a section that describes how failover will work with
>>>> the
>>>>>>>>>> transactional state stores? I think section "Error handling" is
>>>>>> already
>>>>>>>>>> a good start.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> Bruno
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 15.05.23 11:04, Nick Telford wrote:
>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>
>>>>>>>>>>> Quick update: I've added a new section to the KIP: "Offsets for
>>>>>>>> Consumer
>>>>>>>>>>> Rebalances", that outlines my solution to the problem that
>>>>>>>>>>> StreamsPartitionAssignor needs to read StateStore offsets even if
>>>>>>>> they're
>>>>>>>>>>> not currently open.
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>> Nick
>>>>>>>>>>>
>>>>>>>>>>> On Wed, 3 May 2023 at 11:34, Nick Telford <
>> nick.telford@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hi Bruno,
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for reviewing my proposal.
>>>>>>>>>>>>
>>>>>>>>>>>> 1.
>>>>>>>>>>>> The main reason I added it was because it was easy to do. If we
>>>> see
>>>>>> no
>>>>>>>>>>>> value in it, I can remove it.
>>>>>>>>>>>>
>>>>>>>>>>>> 2.
>>>>>>>>>>>> Global StateStores can have multiple partitions in their input
>>>>>> topics
>>>>>>>>>>>> (which function as their changelogs), so they would have more
>> than
>>>>>> one
>>>>>>>>>>>> partition.
>>>>>>>>>>>>
>>>>>>>>>>>> 3.
>>>>>>>>>>>> That's a good point. At present, the only method it adds is
>>>>>>>>>>>> isolationLevel(), which is likely not necessary outside of
>>>>>>>> StateStores.
>>>>>>>>>>>> It *does* provide slightly different guarantees in the
>>>> documentation
>>>>>>>> to
>>>>>>>>>>>> several of the methods (hence the overrides). I'm not sure if
>> this
>>>>>> is
>>>>>>>>>>>> enough to warrant a new interface though.
>>>>>>>>>>>> I think the question that remains is whether this interface
>> makes
>>>> it
>>>>>>>>>>>> easier to implement custom transactional StateStores than if we
>>>> were
>>>>>>>> to
>>>>>>>>>>>> remove it? Probably not.
>>>>>>>>>>>>
>>>>>>>>>>>> 4.
>>>>>>>>>>>> The main motivation for the Atomic Checkpointing is actually
>>>>>>>>>> performance.
>>>>>>>>>>>> My team has been testing out an implementation of this KIP
>> without
>>>>>> it,
>>>>>>>>>> and
>>>>>>>>>>>> we had problems with RocksDB doing *much* more compaction, due
>> to
>>>>>> the
>>>>>>>>>>>> significantly increased flush rate. It was enough of a problem
>>>> that
>>>>>>>> (for
>>>>>>>>>>>> the time being), we had to revert back to Kafka Streams proper.
>>>>>>>>>>>> I think the best way to solve this, as you say, is to keep the
>>>>>>>>>> .checkpoint
>>>>>>>>>>>> files *in addition* to the offsets being stored within the store
>>>>>>>> itself.
>>>>>>>>>>>> Essentially, when closing StateStores, we force a memtable
>> flush,
>>>>>> then
>>>>>>>>>>>> call getCommittedOffsets and write those out to the .checkpoint
>>>>>> file.
>>>>>>>>>>>> That would ensure the metadata is available to the
>>>>>>>>>>>> StreamsPartitionAssignor for all closed stores.
>>>>>>>>>>>> If there's a crash (no clean close), then we won't be able to
>>>>>>>> guarantee
>>>>>>>>>>>> which offsets were flushed to disk by RocksDB, so we'd need to
>>>> open
>>>>>> (
>>>>>>>>>>>> init()), read offsets, and then close() those stores. But since
>>>> this
>>>>>>>> is
>>>>>>>>>>>> the exception, and will only occur once (provided it doesn't
>> crash
>>>>>>>> every
>>>>>>>>>>>> time!), I think the performance impact here would be acceptable.
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for the feedback, please let me know if you have any more
>>>>>>>>>> comments
>>>>>>>>>>>> or questions!
>>>>>>>>>>>>
>>>>>>>>>>>> I'm currently working on rebasing against trunk. This involves
>>>>>> adding
>>>>>>>>>>>> support for transactionality to VersionedStateStores. I will
>>>>>> probably
>>>>>>>>>> need
>>>>>>>>>>>> to revise my implementation for transactional "segmented"
>> stores,
>>>>>> both
>>>>>>>>>> to
>>>>>>>>>>>> accommodate VersionedStateStore, and to clean up some other
>> stuff.
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>> Nick
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org>
>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks for the updates!
>>>>>>>>>>>>>
>>>>>>>>>>>>> I have a couple of questions/comments.
>>>>>>>>>>>>>
>>>>>>>>>>>>> 1.
>>>>>>>>>>>>> Why do you propose a configuration that involves max. bytes and
>>>>>> max.
>>>>>>>>>>>>> reords? I think we are mainly concerned about memory
>> consumption
>>>>>>>>>> because
>>>>>>>>>>>>> we want to limit the off-heap memory used. I cannot think of a
>>>> case
>>>>>>>>>>>>> where one would want to set the max. number of records.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2.
>>>>>>>>>>>>> Why does
>>>>>>>>>>>>>
>>>>>>>>>>>>>         default void commit(final Map<TopicPartition, Long>
>>>>>>>>>> changelogOffsets) {
>>>>>>>>>>>>>             flush();
>>>>>>>>>>>>>         }
>>>>>>>>>>>>>
>>>>>>>>>>>>> take a map of partitions to changelog offsets?
>>>>>>>>>>>>> The mapping between state stores to partitions is a 1:1
>>>>>> relationship.
>>>>>>>>>>>>> Passing in a single changelog offset should suffice.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 3.
>>>>>>>>>>>>> Why do we need the Transaction interface? It should be possible
>>>> to
>>>>>>>> hide
>>>>>>>>>>>>> beginning and committing a transactions withing the state store
>>>>>>>>>>>>> implementation, so that from outside the state store, it does
>> not
>>>>>>>>>> matter
>>>>>>>>>>>>> whether the state store is transactional or not. What would be
>>>> the
>>>>>>>>>>>>> advantage of using the Transaction interface?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 4.
>>>>>>>>>>>>> Regarding checkpointing offsets, I think we should keep the
>>>>>>>> checkpoint
>>>>>>>>>>>>> file in any case for the reason you mentioned about
>> rebalancing.
>>>>>> Even
>>>>>>>>>> if
>>>>>>>>>>>>> that would not be an issue, I would propose to move the change
>> to
>>>>>>>>>> offset
>>>>>>>>>>>>> management to a new KIP and to not add more complexity than
>>>> needed
>>>>>> to
>>>>>>>>>>>>> this one. I would not be too concerned about the consistency
>>>>>>>> violation
>>>>>>>>>>>>> you mention. As far as I understand, with transactional state
>>>>>> stores
>>>>>>>>>>>>> Streams would write the checkpoint file during every commit
>> even
>>>>>>>> under
>>>>>>>>>>>>> EOS. In the failure case you describe, Streams would restore
>> the
>>>>>>>> state
>>>>>>>>>>>>> stores from the offsets found in the checkpoint file written
>>>> during
>>>>>>>> the
>>>>>>>>>>>>> penultimate commit instead of during the last commit.
>> Basically,
>>>>>>>>>> Streams
>>>>>>>>>>>>> would overwrite the records written to the state store between
>>>> the
>>>>>>>> last
>>>>>>>>>>>>> two commits with the same records read from the changelogs.
>>>> While I
>>>>>>>>>>>>> understand that this is wasteful, it is -- at the same time --
>>>>>>>>>>>>> acceptable and most importantly it does not break EOS.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> Bruno
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 27.04.23 12:34, Nick Telford wrote:
>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I find myself (again) considering removing the offset
>> management
>>>>>>>> from
>>>>>>>>>>>>>> StateStores, and keeping the old checkpoint file system. The
>>>>>> reason
>>>>>>>> is
>>>>>>>>>>>>> that
>>>>>>>>>>>>>> the StreamPartitionAssignor directly reads checkpoint files in
>>>>>> order
>>>>>>>>>> to
>>>>>>>>>>>>>> determine which instance has the most up-to-date copy of the
>>>> local
>>>>>>>>>>>>> state.
>>>>>>>>>>>>>> If we move offsets into the StateStore itself, then we will
>> need
>>>>>> to
>>>>>>>>>>>>> open,
>>>>>>>>>>>>>> initialize, read offsets and then close each StateStore (that
>> is
>>>>>> not
>>>>>>>>>>>>>> already assigned and open) for which we have *any* local
>> state,
>>>> on
>>>>>>>>>> every
>>>>>>>>>>>>>> rebalance.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Generally, I don't think there are many "orphan" stores like
>>>> this
>>>>>>>>>>>>> sitting
>>>>>>>>>>>>>> around on most instances, but even a few would introduce
>>>>>> additional
>>>>>>>>>>>>> latency
>>>>>>>>>>>>>> to an already somewhat lengthy rebalance procedure.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'm leaning towards Colt's (Slack) suggestion of just keeping
>>>>>> things
>>>>>>>>>> in
>>>>>>>>>>>>> the
>>>>>>>>>>>>>> checkpoint file(s) for now, and not worrying about the race.
>> The
>>>>>>>>>>>>> downside
>>>>>>>>>>>>>> is that we wouldn't be able to remove the explicit RocksDB
>> flush
>>>>>>>>>>>>> on-commit,
>>>>>>>>>>>>>> which likely hurts performance.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If anyone has any thoughts or ideas on this subject, I would
>>>>>>>>>> appreciate
>>>>>>>>>>>>> it!
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, 19 Apr 2023 at 15:05, Nick Telford <
>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The issue is that if there's a crash between 2 and 3, then
>> you
>>>>>>>> still
>>>>>>>>>>>>> end
>>>>>>>>>>>>>>> up with inconsistent data in RocksDB. The only way to
>> guarantee
>>>>>>>> that
>>>>>>>>>>>>> your
>>>>>>>>>>>>>>> checkpoint offsets and locally stored data are consistent
>> with
>>>>>> each
>>>>>>>>>>>>> other
>>>>>>>>>>>>>>> are to atomically commit them, which can be achieved by
>> having
>>>>>> the
>>>>>>>>>>>>> offsets
>>>>>>>>>>>>>>> stored in RocksDB.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The offsets column family is likely to be extremely small
>> (one
>>>>>>>>>>>>>>> per-changelog partition + one per Topology input partition
>> for
>>>>>>>>>> regular
>>>>>>>>>>>>>>> stores, one per input partition for global stores). So the
>>>>>> overhead
>>>>>>>>>>>>> will be
>>>>>>>>>>>>>>> minimal.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> A major benefit of doing this is that we can remove the
>>>> explicit
>>>>>>>>>> calls
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> db.flush(), which forcibly flushes memtables to disk
>> on-commit.
>>>>>> It
>>>>>>>>>>>>> turns
>>>>>>>>>>>>>>> out, RocksDB memtable flushes are largely dictated by Kafka
>>>>>> Streams
>>>>>>>>>>>>>>> commits, *not* RocksDB configuration, which could be a major
>>>>>> source
>>>>>>>>>> of
>>>>>>>>>>>>>>> confusion. Atomic checkpointing makes it safe to remove these
>>>>>>>>>> explicit
>>>>>>>>>>>>>>> flushes, because it no longer matters exactly when RocksDB
>>>>>> flushes
>>>>>>>>>>>>> data to
>>>>>>>>>>>>>>> disk; since the data and corresponding checkpoint offsets
>> will
>>>>>>>> always
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>> flushed together, the local store is always in a consistent
>>>>>> state,
>>>>>>>>>> and
>>>>>>>>>>>>>>> on-restart, it can always safely resume restoration from the
>>>>>>>> on-disk
>>>>>>>>>>>>>>> offsets, restoring the small amount of data that hadn't been
>>>>>>>> flushed
>>>>>>>>>>>>> when
>>>>>>>>>>>>>>> the app exited/crashed.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <
>>>> colt@littlehorse.io>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for your reply. Ack to A) and B).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> For item C), I see what you're referring to. Your proposed
>>>>>>>> solution
>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>> work, so no need to change it. What I was suggesting was
>> that
>>>> it
>>>>>>>>>>>>> might be
>>>>>>>>>>>>>>>> possible to achieve this with only one column family. So
>> long
>>>>>> as:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>           - No uncommitted records (i.e. not committed to the
>>>>>>>> changelog)
>>>>>>>>>> are
>>>>>>>>>>>>>>>>           *committed* to the state store, AND
>>>>>>>>>>>>>>>>           - The Checkpoint offset (which refers to the
>> changelog
>>>>>>>> topic)
>>>>>>>>>> is
>>>>>>>>>>>>> less
>>>>>>>>>>>>>>>>           than or equal to the last written changelog offset
>> in
>>>>>>>> rocksdb
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I don't see the need to do the full restoration from
>> scratch.
>>>> My
>>>>>>>>>>>>>>>> understanding was that prior to 844/892, full restorations
>>>> were
>>>>>>>>>>>>> required
>>>>>>>>>>>>>>>> because there could be uncommitted records written to
>> RocksDB;
>>>>>>>>>>>>> however,
>>>>>>>>>>>>>>>> given your use of RocksDB transactions, that can be avoided
>>>> with
>>>>>>>> the
>>>>>>>>>>>>>>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
>>>>>>>>>>>>> transaction, 3)
>>>>>>>>>>>>>>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <
>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> A. I've done my best to de-couple the StateStore stuff from
>>>> the
>>>>>>>>>> rest
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> Streams engine. The fact that there will be only one
>> ongoing
>>>>>>>>>> (write)
>>>>>>>>>>>>>>>>> transaction at a time is not guaranteed by any API, and is
>>>>>> just a
>>>>>>>>>>>>>>>>> consequence of the way Streams operates. To that end, I
>> tried
>>>>>> to
>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> documentation and guarantees provided by the new APIs are
>>>>>>>>>>>>> independent of
>>>>>>>>>>>>>>>>> this incidental behaviour. In practice, you're right, this
>>>>>>>>>>>>> essentially
>>>>>>>>>>>>>>>>> refers to "interactive queries", which are technically
>> "read
>>>>>>>>>>>>>>>> transactions",
>>>>>>>>>>>>>>>>> even if they don't actually use the transaction API to
>>>> isolate
>>>>>>>>>>>>>>>> themselves.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> B. Yes, although not ideal. This is for backwards
>>>>>> compatibility,
>>>>>>>>>>>>>>>> because:
>>>>>>>>>>>>>>>>>            1) Existing custom StateStore implementations
>> will
>>>>>>>> implement
>>>>>>>>>>>>>>>> flush(),
>>>>>>>>>>>>>>>>> and not commit(), but the Streams engine now calls
>> commit(),
>>>> so
>>>>>>>>>> those
>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>> need to be forwarded to flush() for these legacy stores.
>>>>>>>>>>>>>>>>>            2) Existing StateStore *users*, i.e. outside of
>> the
>>>>>>>> Streams
>>>>>>>>>>>>> engine
>>>>>>>>>>>>>>>>> itself, may depend on explicitly calling flush(), so for
>>>> these
>>>>>>>>>> cases,
>>>>>>>>>>>>>>>>> flush() needs to be redirected to call commit().
>>>>>>>>>>>>>>>>> If anyone has a better way to guarantee compatibility
>> without
>>>>>>>>>>>>>>>> introducing
>>>>>>>>>>>>>>>>> this potential recursion loop, I'm open to changes!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> C. This is described in the "Atomic Checkpointing" section.
>>>>>>>> Offsets
>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>> stored in a separate RocksDB column family, which is
>>>> guaranteed
>>>>>>>> to
>>>>>>>>>> be
>>>>>>>>>>>>>>>>> atomically flushed to disk with all other column families.
>>>> The
>>>>>>>>>> issue
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> checkpoints being written to disk after commit causing
>>>>>>>>>> inconsistency
>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>> crashes in between is the reason why, under EOS, checkpoint
>>>>>> files
>>>>>>>>>> are
>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>> written on clean shutdown. This is one of the major causes
>> of
>>>>>>>> "full
>>>>>>>>>>>>>>>>> restorations", so moving the offsets into a place where
>> they
>>>>>> can
>>>>>>>> be
>>>>>>>>>>>>>>>>> guaranteed to be atomically written with the data they
>>>>>> checkpoint
>>>>>>>>>>>>>>>> allows us
>>>>>>>>>>>>>>>>> to write the checkpoint offsets *on every commit*, not just
>>>> on
>>>>>>>>>> clean
>>>>>>>>>>>>>>>>> shutdown.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <
>>>>>> colt@littlehorse.io>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thank you for continuing this work. I have a few minor
>>>>>>>> clarifying
>>>>>>>>>>>>>>>>>> questions.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> A) "Records written to any transaction are visible to all
>>>>>> other
>>>>>>>>>>>>>>>>>> transactions immediately." I am confused here—I thought
>>>> there
>>>>>>>>>> could
>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> one transaction going on at a time for a given state store
>>>>>> given
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> threading model for processing records on a Task. Do you
>>>> mean
>>>>>>>>>>>>>>>> Interactive
>>>>>>>>>>>>>>>>>> Queries by "other transactions"? (If so, then everything
>>>> makes
>>>>>>>>>>>>> sense—I
>>>>>>>>>>>>>>>>>> thought that since IQ were read-only then they didn't
>> count
>>>> as
>>>>>>>>>>>>>>>>>> transactions).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> B) Is it intentional that the default implementations of
>> the
>>>>>>>>>> flush()
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> commit() methods in the StateStore class refer to each
>> other
>>>>>> in
>>>>>>>>>> some
>>>>>>>>>>>>>>>> sort
>>>>>>>>>>>>>>>>>> of unbounded recursion?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> C) How will the getCommittedOffset() method work? At
>> first I
>>>>>>>>>> thought
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> way to do it would be using a special key in the RocksDB
>>>> store
>>>>>>>> to
>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> offset, and committing that with the transaction. But upon
>>>>>>>> second
>>>>>>>>>>>>>>>>> thought,
>>>>>>>>>>>>>>>>>> since restoration from the changelog is an idempotent
>>>>>>>> procedure, I
>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>> would be fine to 1) commit the RocksDB transaction and
>> then
>>>> 2)
>>>>>>>>>> write
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> offset to disk in a checkpoint file. If there is a crash
>>>>>> between
>>>>>>>>>> 1)
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> 2),
>>>>>>>>>>>>>>>>>> I think the only downside is 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 <
>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I've updated the KIP to reflect the latest version of the
>>>>>>>> design:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> There are several changes in there that reflect feedback
>>>> from
>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> thread,
>>>>>>>>>>>>>>>>>>> and there's a new section and a bunch of interface
>> changes
>>>>>>>>>> relating
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> Atomic Checkpointing, which is the final piece of the
>>>> puzzle
>>>>>> to
>>>>>>>>>>>>>>>> making
>>>>>>>>>>>>>>>>>>> everything robust.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Let me know what you think!
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <
>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi Lucas,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks for looking over my KIP.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> A) The bound is per-instance, not per-Task. This was a
>>>> typo
>>>>>> in
>>>>>>>>>> the
>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>> that I've now corrected. It was originally per-Task,
>> but I
>>>>>>>>>>>>>>>> changed it
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> per-instance for exactly the reason you highlighted.
>>>>>>>>>>>>>>>>>>>> B) It's worth noting that transactionality is only
>> enabled
>>>>>>>> under
>>>>>>>>>>>>>>>> EOS,
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> in the default mode of operation (ALOS), there should be
>>>> no
>>>>>>>>>>>>>>>> change in
>>>>>>>>>>>>>>>>>>>> behavior at all. I think, under EOS, we can mitigate the
>>>>>>>> impact
>>>>>>>>>> on
>>>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>> sufficiently low default values for the memory bound
>>>>>>>>>>>>>>>> configuration. I
>>>>>>>>>>>>>>>>>>>> understand your hesitation to include a significant
>> change
>>>>>> of
>>>>>>>>>>>>>>>>>> behaviour,
>>>>>>>>>>>>>>>>>>>> especially in a minor release, but I suspect that most
>>>> users
>>>>>>>>>> will
>>>>>>>>>>>>>>>>>> prefer
>>>>>>>>>>>>>>>>>>>> the memory impact (under EOS) to the existing behaviour
>> of
>>>>>>>>>>>>>>>> frequent
>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>> restorations! If this is a problem, the changes can wait
>>>>>> until
>>>>>>>>>> the
>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>> major release. I'll be running a patched version of
>>>> streams
>>>>>> in
>>>>>>>>>>>>>>>>>> production
>>>>>>>>>>>>>>>>>>>> with these changes as soon as they're ready, so it won't
>>>>>>>> disrupt
>>>>>>>>>>>>>>>> me
>>>>>>>>>>>>>>>>> :-D
>>>>>>>>>>>>>>>>>>>> C) The main purpose of this sentence was just to note
>> that
>>>>>>>> some
>>>>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>>>> will need to be made to the way Segments are handled in
>>>>>> order
>>>>>>>> to
>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>> they also benefit from transactions. At the time I wrote
>>>>>> it, I
>>>>>>>>>>>>>>>> hadn't
>>>>>>>>>>>>>>>>>>>> figured out the specific changes necessary, so it was
>>>>>>>>>> deliberately
>>>>>>>>>>>>>>>>>> vague.
>>>>>>>>>>>>>>>>>>>> This is the one outstanding problem I'm currently
>> working
>>>>>> on,
>>>>>>>>>> and
>>>>>>>>>>>>>>>>> I'll
>>>>>>>>>>>>>>>>>>>> update this section with more detail once I have figured
>>>> out
>>>>>>>> the
>>>>>>>>>>>>>>>>> exact
>>>>>>>>>>>>>>>>>>>> changes required.
>>>>>>>>>>>>>>>>>>>> D) newTransaction() provides the necessary isolation
>>>>>>>> guarantees.
>>>>>>>>>>>>>>>>> While
>>>>>>>>>>>>>>>>>>>> the RocksDB implementation of transactions doesn't
>>>>>> technically
>>>>>>>>>>>>>>>> *need*
>>>>>>>>>>>>>>>>>>>> read-only users to call newTransaction(), other
>>>>>>>> implementations
>>>>>>>>>>>>>>>>> (e.g. a
>>>>>>>>>>>>>>>>>>>> hypothetical PostgresStore) may require it. Calling
>>>>>>>>>>>>>>>> newTransaction()
>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>> no transaction is necessary is essentially free, as it
>>>> will
>>>>>>>> just
>>>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I
>> think
>>>> it
>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> fairly obvious where the performance problems stem from:
>>>>>>>> writes
>>>>>>>>>>>>>>>> under
>>>>>>>>>>>>>>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it
>> with
>>>>>> the
>>>>>>>>>>>>>>>>>>>> tombstone/record flag, 1 to decode it from the
>>>>>>>> tombstone/record
>>>>>>>>>>>>>>>> flag,
>>>>>>>>>>>>>>>>>>> and 1
>>>>>>>>>>>>>>>>>>>> to copy the record from the "temporary" store to the
>>>> "main"
>>>>>>>>>> store,
>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> transaction commits. The different approach taken by
>>>> KIP-869
>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>> perform
>>>>>>>>>>>>>>>>>>>> much better, as it avoids all these copies, and may
>>>> actually
>>>>>>>>>>>>>>>> perform
>>>>>>>>>>>>>>>>>>>> slightly better than trunk, due to batched writes in
>>>> RocksDB
>>>>>>>>>>>>>>>>> performing
>>>>>>>>>>>>>>>>>>>> better than non-batched writes.[1]
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 1:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
>>>>>>>>>>>>>>>> lbrutschy@confluent.io
>>>>>>>>>>>>>>>>>>> .invalid>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I'm just starting to read up on the whole discussion
>>>> about
>>>>>>>>>>>>>>>> KIP-892
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
>>>>>>>>>>>>>>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do
>>>> have
>>>>>>>> some
>>>>>>>>>>>>>>>>>>>>> questions about the latest draft.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>         A) If I understand correctly, you propose to
>> put a
>>>>>> bound
>>>>>>>> on
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> (native) memory consumed by each task. However, I
>> wonder
>>>> if
>>>>>>>>>> this
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> sufficient if we have temporary imbalances in the
>>>> cluster.
>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>> example, depending on the timing of rebalances during a
>>>>>>>> cluster
>>>>>>>>>>>>>>>>>>>>> restart, it could happen that a single streams node is
>>>>>>>>>> assigned a
>>>>>>>>>>>>>>>>> lot
>>>>>>>>>>>>>>>>>>>>> more tasks than expected. With your proposed change,
>> this
>>>>>>>> would
>>>>>>>>>>>>>>>> mean
>>>>>>>>>>>>>>>>>>>>> that the memory required by this one node could be a
>>>>>> multiple
>>>>>>>>>> of
>>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>>>> is required during normal operation. I wonder if it
>>>>>> wouldn't
>>>>>>>> be
>>>>>>>>>>>>>>>>> safer
>>>>>>>>>>>>>>>>>>>>> to put a global bound on the memory use, across all
>>>> tasks.
>>>>>>>>>>>>>>>>>>>>>         B) Generally, the memory concerns still give me
>> the
>>>>>>>> feeling
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> should not be enabled by default for all users in a
>> minor
>>>>>>>>>>>>>>>> release.
>>>>>>>>>>>>>>>>>>>>>         C) In section "Transaction Management": the
>>>> sentence
>>>>>> "A
>>>>>>>>>> similar
>>>>>>>>>>>>>>>>>>>>> analogue will be created to automatically manage
>>>> `Segment`
>>>>>>>>>>>>>>>>>>>>> transactions.". Maybe this is just me lacking some
>>>>>>>> background,
>>>>>>>>>>>>>>>> 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 you to find out why the KIP-844 PoC is 20x
>>>>>>>> slower
>>>>>>>>>>>>>>>> (e.g.
>>>>>>>>>>>>>>>>>>>>> by providing a flame graph), that would be quite
>>>>>> interesting.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>>>> Lucas
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP with a more detailed design,
>> which
>>>>>>>>>>>>>>>> reflects
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> implementation I've been working on:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> This new design should address the outstanding points
>>>>>>>> already
>>>>>>>>>>>>>>>> made
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Please let me know if there are areas that are unclear
>>>> or
>>>>>>>> need
>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>>> clarification.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I have a (nearly) working implementation. I'm
>> confident
>>>>>> that
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> remaining
>>>>>>>>>>>>>>>>>>>>>> work (making Segments behave) will not impact the
>>>>>> documented
>>>>>>>>>>>>>>>>> design.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <
>>>>>>>>>> colt@littlehorse.io
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thank you for the reply; that makes sense. I was
>> hoping
>>>>>>>> that,
>>>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>>>> reading
>>>>>>>>>>>>>>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
>>>>>>>>>>>>>>>> documented
>>>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>>>> maybe
>>>>>>>>>>>>>>>>>>>>>>> you *wouldn't* have to wait for the next major
>> release
>>>> to
>>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>> change;
>>>>>>>>>>>>>>>>>>>>>>> but given 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 <
>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally
>> intended
>>>>>> to
>>>>>>>>>>>>>>>> try
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> keep the
>>>>>>>>>>>>>>>>>>>>>>>> behaviour unchanged as much as possible, otherwise
>>>> we'd
>>>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> wait for
>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> major version release to land these changes.
>>>>>>>>>>>>>>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
>>>>>>>>>>>>>>>> guarantee,
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> typically longer commit intervals would be
>> problematic
>>>>>>>> when
>>>>>>>>>>>>>>>>>>> reading
>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>> "committed" records.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I've been away for 5 days recovering from minor
>>>> surgery,
>>>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>>>> spent a
>>>>>>>>>>>>>>>>>>>>>>>> considerable amount of that time working through
>> ideas
>>>>>> for
>>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>>>> solutions in my head. I think your suggestion of
>>>> keeping
>>>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>>>>>> as-is, but
>>>>>>>>>>>>>>>>>>>>>>>> buffering writes for EOS is the right path forwards,
>>>>>>>>>>>>>>>> although
>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>>>> solution that both expands on this, and provides for
>>>>>> some
>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>> formal
>>>>>>>>>>>>>>>>>>>>>>>> guarantees.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Essentially, adding support to KeyValueStores for
>>>>>>>>>>>>>>>>>> "Transactions",
>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>> clearly defined IsolationLevels. Using "Read
>>>> Committed"
>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>> under
>>>>>>>>>>>>>>>>>>>>> EOS,
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>> "Read Uncommitted" under ALOS.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> The nice thing about this approach is that it gives
>> us
>>>>>>>> much
>>>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>>> clearly
>>>>>>>>>>>>>>>>>>>>>>>> defined isolation behaviour that can be properly
>>>>>>>>>>>>>>>> documented to
>>>>>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>>>>>>>>> know what to expect.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I'm still working out the kinks in the design, and
>>>> will
>>>>>>>>>>>>>>>> update
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>>>> I have something. The main struggle is trying to
>>>>>> implement
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>>>> making any major changes to the existing interfaces
>> or
>>>>>>>>>>>>>>>>> breaking
>>>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>>>> implementations, because currently everything
>> expects
>>>> to
>>>>>>>>>>>>>>>>> operate
>>>>>>>>>>>>>>>>>>>>> directly
>>>>>>>>>>>>>>>>>>>>>>>> on a StateStore, and not a Transaction of that
>> store.
>>>> I
>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>> close, although sadly I won't be able to progress
>> much
>>>>>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>>>> week
>>>>>>>>>>>>>>>>>>>>>>> due
>>>>>>>>>>>>>>>>>>>>>>>> to some work commitments.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
>>>>>>>>>>>>>>>>> colt@littlehorse.io>
>>>>>>>
> 

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

Posted by Nick Telford <ni...@gmail.com>.
Here's what I'm thinking: based on Bruno's earlier feedback, I'm going to
try to simplify my original design down such that it needs no/minimal
changes to the public interface.

If that succeeds, then it should also be possible to transparently
implement the "no memtables" solution as a performance optimization when
the record cache is enabled. I consider this approach only an optimisation,
because of the need to still support stores with the cache disabled.

For that reason, I think the "no memtables" approach would probably best be
suited as a follow-up KIP, but that we keep it in mind during the design of
this one.

What do you think?

Regards,
Nick


On Tue, 20 Jun 2023, 22:26 John Roesler, <vv...@apache.org> wrote:

> Oh, that's a good point.
>
> On the topic of a behavioral switch for disabled caches, the typical use
> case for disabling the cache is to cause each individual update to
> propagate down the topology, so another thought might be to just go
> ahead and add the memory we would have used for the memtables to the
> cache size, but if people did disable the cache entirely, then we could
> still go ahead and forward the records on each write?
>
> I know that Guozhang was also proposing for a while to actually decouple
> caching and forwarding, which might provide a way to side-step this
> dilemma (i.e., we just always forward and only apply the cache to state
> and changelog writes).
>
> By the way, I'm basing my statement on why you'd disable caches on
> memory, but also on the guidance here:
>
> https://docs.confluent.io/platform/current/streams/developer-guide/memory-mgmt.html
> . That doc also contains a section on how to bound the total memory
> usage across RocksDB memtables, which points to another benefit of
> disabling memtables and managing the write buffer ourselves (simplified
> memory configuration).
>
> Thanks,
> -John
>
> On 6/20/23 16:05, Nick Telford wrote:
> > Potentially we could just go the memorable with Rocks WriteBatches route
> if
> > the cache is disabled?
> >
> > On Tue, 20 Jun 2023, 22:00 John Roesler, <jo...@vvcephei.org> wrote:
> >
> >> Touché!
> >>
> >> Ok, I agree that figuring out the case of a disabled cache would be
> >> non-trivial. Ingesting single-record SST files will probably not be
> >> performant, but benchmarking may prove different. Or maybe we can have
> >> some reserved cache space on top of the user-configured cache, which we
> >> would have reclaimed from the memtable space. Or some other, more
> >> creative solution.
> >>
> >> Thanks,
> >> -John
> >>
> >> On 6/20/23 15:30, Nick Telford wrote:
> >>>> Note that users can disable the cache, which would still be
> >>> ok, I think. We wouldn't ingest the SST files on every record, but just
> >>> append to them and only ingest them on commit, when we're already
> >>> waiting for acks and a RocksDB commit.
> >>>
> >>> In this case, how would uncommitted records be read by joins?
> >>>
> >>> On Tue, 20 Jun 2023, 20:51 John Roesler, <vv...@apache.org> wrote:
> >>>
> >>>> Ah, sorry Nick,
> >>>>
> >>>> I just meant the regular heap based cache that we maintain in
> Streams. I
> >>>> see that it's not called "RecordCache" (my mistake).
> >>>>
> >>>> The actual cache is ThreadCache:
> >>>>
> >>>>
> >>
> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
> >>>>
> >>>> Here's the example of how we use the cache in KeyValueStore:
> >>>>
> >>>>
> >>
> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
> >>>>
> >>>> It's basically just an on-heap Map of records that have not yet been
> >>>> written to the changelog or flushed into the underlying store. It gets
> >>>> flushed when the total cache size exceeds `cache.max.bytes.buffering`
> or
> >>>> the `commit.interval.ms` elapses.
> >>>>
> >>>> Speaking of those configs, another benefit to this idea is that we
> would
> >>>> no longer need to trigger extra commits based on the size of the
> ongoing
> >>>> transaction. Instead, we'd just preserve the existing cache-flush
> >>>> behavior. Note that users can disable the cache, which would still be
> >>>> ok, I think. We wouldn't ingest the SST files on every record, but
> just
> >>>> append to them and only ingest them on commit, when we're already
> >>>> waiting for acks and a RocksDB commit.
> >>>>
> >>>> Thanks,
> >>>> -John
> >>>>
> >>>> On 6/20/23 14:09, Nick Telford wrote:
> >>>>> Hi John,
> >>>>>
> >>>>> By "RecordCache", do you mean the RocksDB "WriteBatch"? I can't find
> >> any
> >>>>> class called "RecordCache"...
> >>>>>
> >>>>> Cheers,
> >>>>>
> >>>>> Nick
> >>>>>
> >>>>> On Tue, 20 Jun 2023 at 19:42, John Roesler <vv...@apache.org>
> >> wrote:
> >>>>>
> >>>>>> Hi Nick,
> >>>>>>
> >>>>>> Thanks for picking this up again!
> >>>>>>
> >>>>>> I did have one new thought over the intervening months, which I'd
> like
> >>>>>> your take on.
> >>>>>>
> >>>>>> What if, instead of using the RocksDB atomic write primitive at all,
> >> we
> >>>>>> instead just:
> >>>>>> 1. disable memtables entirely
> >>>>>> 2. directly write the RecordCache into SST files when we flush
> >>>>>> 3. atomically ingest the SST file(s) into RocksDB when we get the
> ACK
> >>>>>> from the changelog (see
> >>>>>>
> >>>>>>
> >>>>
> >>
> https://github.com/EighteenZi/rocksdb_wiki/blob/master/Creating-and-Ingesting-SST-files.md
> >>>>>> and
> >>>>>>
> >>>>>>
> >>>>
> >>
> https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/IngestExternalFileOptions.java
> >>>>>> and
> >>>>>>
> >>>>>>
> >>>>
> >>
> https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L1413-L1429
> >>>>>> )
> >>>>>> 4. track the changelog offsets either in another CF or the same CF
> >> with
> >>>>>> a reserved key, either of which will make the changelog offset
> update
> >>>>>> atomic with the file ingestions
> >>>>>>
> >>>>>> I suspect this'll have a number of benefits:
> >>>>>> * writes to RocksDB will always be atomic
> >>>>>> * we don't fragment memory between the RecordCache and the memtables
> >>>>>> * RecordCache gives far higher performance than memtable for reads
> and
> >>>>>> writes
> >>>>>> * we don't need any new "transaction" concepts or memory bound
> configs
> >>>>>>
> >>>>>> What do you think?
> >>>>>>
> >>>>>> Thanks,
> >>>>>> -John
> >>>>>>
> >>>>>> On 6/20/23 10:51, Nick Telford wrote:
> >>>>>>> Hi Bruno,
> >>>>>>>
> >>>>>>> Thanks for reviewing the KIP. It's been a long road, I started
> >> working
> >>>> on
> >>>>>>> this more than a year ago, and most of the time in the last 6
> months
> >>>> has
> >>>>>>> been spent on the "Atomic Checkpointing" stuff that's been benched,
> >> so
> >>>>>> some
> >>>>>>> of the reasoning behind some of my decisions have been lost, but
> I'll
> >>>> do
> >>>>>> my
> >>>>>>> best to reconstruct them.
> >>>>>>>
> >>>>>>> 1.
> >>>>>>> IIRC, this was the initial approach I tried. I don't remember the
> >> exact
> >>>>>>> reasons I changed it to use a separate "view" of the StateStore
> that
> >>>>>>> encapsulates the transaction, but I believe it had something to do
> >> with
> >>>>>>> concurrent access to the StateStore from Interactive Query threads.
> >>>> Reads
> >>>>>>> from interactive queries need to be isolated from the currently
> >> ongoing
> >>>>>>> transaction, both for consistency (so interactive queries don't
> >> observe
> >>>>>>> changes that are subsequently rolled-back), but also to prevent
> >>>> Iterators
> >>>>>>> opened by an interactive query from being closed and invalidated by
> >> the
> >>>>>>> StreamThread when it commits the transaction, which causes your
> >>>>>> interactive
> >>>>>>> queries to crash.
> >>>>>>>
> >>>>>>> Another reason I believe I implemented it this way was a separation
> >> of
> >>>>>>> concerns. Recall that newTransaction() originally created an object
> >> of
> >>>>>> type
> >>>>>>> Transaction, not StateStore. My intent was to improve the
> type-safety
> >>>> of
> >>>>>>> the API, in an effort to ensure Transactions weren't used
> >> incorrectly.
> >>>>>>> Unfortunately, this didn't pan out, but newTransaction() remained.
> >>>>>>>
> >>>>>>> Finally, this had the added benefit that implementations could
> easily
> >>>> add
> >>>>>>> support for transactions *without* re-writing their existing,
> >>>>>>> non-transactional implementation. I think this can be a benefit
> both
> >>>> for
> >>>>>>> implementers of custom StateStores, but also for anyone extending
> >>>>>>> RocksDbStore, as they can rely on the existing access methods
> working
> >>>> how
> >>>>>>> they expect them to.
> >>>>>>>
> >>>>>>> I'm not too happy with the way the current design has panned out,
> so
> >>>> I'm
> >>>>>>> open to ideas on how to improve it. Key to this is finding some way
> >> to
> >>>>>>> ensure that reads from Interactive Query threads are properly
> >> isolated
> >>>>>> from
> >>>>>>> the transaction, *without* the performance overhead of checking
> which
> >>>>>>> thread the method is being called from on every access.
> >>>>>>>
> >>>>>>> As for replacing flush() with commit() - I saw no reason to add
> this
> >>>>>>> complexity to the KIP, unless there was a need to add arguments to
> >> the
> >>>>>>> flush/commit method. This need arises with Atomic Checkpointing,
> but
> >>>> that
> >>>>>>> will be implemented separately, in a future KIP. Do you see a need
> >> for
> >>>>>> some
> >>>>>>> arguments to the flush/commit method that I've missed? Or were you
> >>>> simply
> >>>>>>> suggesting a rename?
> >>>>>>>
> >>>>>>> 2.
> >>>>>>> This is simply due to the practical reason that isolationLevel() is
> >>>>>> really
> >>>>>>> a proxy for checking if the app is under EOS. The application
> >>>>>> configuration
> >>>>>>> is not provided to the constructor of StateStores, but it *is*
> >> provided
> >>>>>> to
> >>>>>>> init(), via StateStoreContext. For this reason, it seemed somewhat
> >>>>>> natural
> >>>>>>> to add it to StateStoreContext. I think this makes sense, since the
> >>>>>>> IsolationLevel of all StateStores in an application *must* be the
> >> same,
> >>>>>> and
> >>>>>>> since those stores are all initialized with the same
> >> StateStoreContext,
> >>>>>> it
> >>>>>>> seems natural for that context to carry the desired IsolationLevel
> to
> >>>>>> use.
> >>>>>>>
> >>>>>>> 3.
> >>>>>>> Using IsolationLevel instead of just passing `boolean eosEnabled`,
> >> like
> >>>>>>> much of the internals was an attempt to logically de-couple the
> >>>>>> StateStore
> >>>>>>> API from the internals of Kafka Streams. Technically, StateStores
> >> don't
> >>>>>>> need to know/care what processing mode the KS app is using, all
> they
> >>>> need
> >>>>>>> to know is the isolation level expected of them.
> >>>>>>>
> >>>>>>> Having formal definitions for the expectations of the two required
> >>>>>>> IsolationLevels allow implementers to implement transactional
> stores
> >>>>>>> without having to dig through the internals of Kafka Streams and
> >>>>>> understand
> >>>>>>> exactly how they are used. The tight coupling between state stores
> >> and
> >>>>>>> internal behaviour has actually significantly hindered my progress
> on
> >>>>>> this
> >>>>>>> KIP, and encouraged me to avoid increasing this logical coupling as
> >>>> much
> >>>>>> as
> >>>>>>> possible.
> >>>>>>>
> >>>>>>> This also frees implementations to satisfy those requirements in
> any
> >>>> way
> >>>>>>> they choose. Transactions might not be the only/available approach
> to
> >>>> an
> >>>>>>> implementation, but they might have an alternative way to satisfy
> the
> >>>>>>> isolation requirements. I admit that this point is more about
> >>>> semantics,
> >>>>>>> but "transactional" would need to be formally defined in order for
> >>>>>>> implementers to provide a valid implementation, and these
> >>>> IsolationLevels
> >>>>>>> provide that formal definition.
> >>>>>>>
> >>>>>>> 4.
> >>>>>>> I can remove them. I added them only as I planned to include them
> in
> >>>> the
> >>>>>>> org.apache.kafka.streams.state package, as a recommended base
> >>>>>>> implementation for all StateStores, including those implemented by
> >>>>>> users. I
> >>>>>>> had assumed that anything in "public" packages, such as
> >>>>>>> org.apache.kafka.streams.state, should be included in a KIP. Is
> that
> >>>>>> wrong?
> >>>>>>>
> >>>>>>> 5.
> >>>>>>> RocksDB provides no way to measure the actual size of a
> >>>>>>> WriteBatch(WithIndex), so we're limited to tracking the sum total
> of
> >>>> the
> >>>>>>> size of keys + values that are written to the transaction. This
> >>>> obviously
> >>>>>>> under-estimates the actual memory usage, because WriteBatch
> no-doubt
> >>>>>>> includes some record overheads, and WriteBatchWithIndex has to
> >> maintain
> >>>>>> an
> >>>>>>> index.
> >>>>>>>
> >>>>>>> Ideally, we could trivially add a method upstream to
> >>>> WriteBatchInterface
> >>>>>>> that provides the exact size of the batch, but that would require
> an
> >>>>>>> upgrade of RocksDB, which won't happen soon. So for the time being,
> >>>> we're
> >>>>>>> stuck with an approximation, so I felt that the new method should
> >>>> reflect
> >>>>>>> that.
> >>>>>>>
> >>>>>>> Would you prefer the new method name ignores this constraint and
> that
> >>>> we
> >>>>>>> simply make the rocks measurement more accurate in the future?
> >>>>>>>
> >>>>>>> 6.
> >>>>>>> Done
> >>>>>>>
> >>>>>>> 7.
> >>>>>>> Very good point. The KIP already specifically calls out memory in
> the
> >>>>>>> documentation of the config: "Maximum number of memory bytes to be
> >> used
> >>>>>> to
> >>>>>>> buffer uncommitted state-store records." - did you have something
> >> else
> >>>> in
> >>>>>>> mind?
> >>>>>>>
> >>>>>>> Should we also make this clearer by renaming the config property
> >>>> itself?
> >>>>>>> Perhaps to something like statestore.transaction.buffer.max.bytes?
> >>>>>>>
> >>>>>>> 8.
> >>>>>>> OK, I can remove this. The intent here was to describe how Streams
> >>>> itself
> >>>>>>> will manage transaction roll-over etc. Presumably that means we
> also
> >>>>>> don't
> >>>>>>> need a description of how Streams will manage the commit of
> changelog
> >>>>>>> transactions, state store transactions and checkpointing?
> >>>>>>>
> >>>>>>> 9.
> >>>>>>> What do you mean by fail-over? Do you mean failing over an Active
> >> Task
> >>>> to
> >>>>>>> an instance already hosting a Standby Task?
> >>>>>>>
> >>>>>>> Thanks again and sorry for the essay of a response!
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>> Nick
> >>>>>>>
> >>>>>>> On Tue, 20 Jun 2023 at 10:49, Bruno Cadonna <ca...@apache.org>
> >>>> wrote:
> >>>>>>>
> >>>>>>>> Hi Nick,
> >>>>>>>>
> >>>>>>>> Thanks for the updates!
> >>>>>>>>
> >>>>>>>> I really appreciate that you simplified the KIP by removing some
> >>>>>>>> aspects. As I have already told you, I think the removed aspects
> are
> >>>>>>>> also good ideas and we can discuss them on follow-up KIPs.
> >>>>>>>>
> >>>>>>>> Regarding the current KIP, I have the following feedback.
> >>>>>>>>
> >>>>>>>> 1.
> >>>>>>>> Is there a good reason to add method newTransaction() to the
> >>>> StateStore
> >>>>>>>> interface? As far as I understand, the idea is that users of a
> state
> >>>>>>>> store (transactional or not) call this method at start-up and
> after
> >>>> each
> >>>>>>>> commit. Since the call to newTransaction() is done in any case
> and I
> >>>>>>>> think it would simplify the caller code if we just start a new
> >>>>>>>> transaction after a commit in the implementation?
> >>>>>>>> As far as I understand, you plan to commit the transaction in the
> >>>>>>>> flush() method. I find the idea to replace flush() with commit()
> >>>>>>>> presented in KIP-844 an elegant solution.
> >>>>>>>>
> >>>>>>>> 2.
> >>>>>>>> Why is the method to query the isolation level added to the state
> >>>> store
> >>>>>>>> context?
> >>>>>>>>
> >>>>>>>> 3.
> >>>>>>>> Do we need all the isolation level definitions? I think it is good
> >> to
> >>>>>>>> know the guarantees of the transactionality of the state store.
> >>>>>>>> However, currently, Streams guarantees that there will only be one
> >>>>>>>> transaction that writes to the state store. Only the stream thread
> >>>> that
> >>>>>>>> executes the active task that owns the state store will write to
> the
> >>>>>>>> state store. I think it should be enough to know if the state
> store
> >> is
> >>>>>>>> transactional or not. So my proposal would be to just add a method
> >> on
> >>>>>>>> the state store interface the returns if a state store is
> >>>> transactional
> >>>>>>>> or not by returning a boolean or an enum.
> >>>>>>>>
> >>>>>>>> 4.
> >>>>>>>> I am wondering why AbstractTransaction and
> >> AbstractTransactionalStore
> >>>>>>>> are part of the KIP. They look like implementation details that
> >> should
> >>>>>>>> not be exposed in the public API.
> >>>>>>>>
> >>>>>>>> 5.
> >>>>>>>> Why does StateStore#approximateNumUncommittedBytes() return an
> >>>>>>>> approximate number of bytes?
> >>>>>>>>
> >>>>>>>> 6.
> >>>>>>>> RocksDB is just one implementation of the state stores in Streams.
> >>>>>>>> However, the issues regarding OOM errors might also apply to other
> >>>>>>>> custom implementations. So in the KIP I would extract that part
> from
> >>>>>>>> section "RocksDB Transaction". I would also move section "RocksDB
> >>>>>>>> Transaction" to the end of section "Proposed Changes" and handle
> it
> >> as
> >>>>>>>> an example implementation for a state store.
> >>>>>>>>
> >>>>>>>> 7.
> >>>>>>>> Should statestore.uncommitted.max.bytes only limit the uncommitted
> >>>> bytes
> >>>>>>>> or the uncommitted bytes that reside in memory? In future, other
> >>>>>>>> transactional state store implementations might implement a buffer
> >> for
> >>>>>>>> uncommitted records that are able to spill records on disk. I
> think
> >>>>>>>> statestore.uncommitted.max.bytes needs to limit the uncommitted
> >> bytes
> >>>>>>>> irrespective if they reside in memory or disk. Since Streams will
> >> use
> >>>>>>>> this config to decide if it needs to trigger a commit, state store
> >>>>>>>> implementations that can spill to disk will never be able to spill
> >> to
> >>>>>>>> disk. You would only need to change the doc of the config, if you
> >>>> agree
> >>>>>>>> with me.
> >>>>>>>>
> >>>>>>>> 8.
> >>>>>>>> Section "Transaction Management" about the wrappers is rather a
> >>>>>>>> implementation detail that should not be in the KIP.
> >>>>>>>>
> >>>>>>>> 9.
> >>>>>>>> Could you add a section that describes how failover will work with
> >> the
> >>>>>>>> transactional state stores? I think section "Error handling" is
> >>>> already
> >>>>>>>> a good start.
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> Best,
> >>>>>>>> Bruno
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On 15.05.23 11:04, Nick Telford wrote:
> >>>>>>>>> Hi everyone,
> >>>>>>>>>
> >>>>>>>>> Quick update: I've added a new section to the KIP: "Offsets for
> >>>>>> Consumer
> >>>>>>>>> Rebalances", that outlines my solution to the problem that
> >>>>>>>>> StreamsPartitionAssignor needs to read StateStore offsets even if
> >>>>>> they're
> >>>>>>>>> not currently open.
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>> Nick
> >>>>>>>>>
> >>>>>>>>> On Wed, 3 May 2023 at 11:34, Nick Telford <
> nick.telford@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi Bruno,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for reviewing my proposal.
> >>>>>>>>>>
> >>>>>>>>>> 1.
> >>>>>>>>>> The main reason I added it was because it was easy to do. If we
> >> see
> >>>> no
> >>>>>>>>>> value in it, I can remove it.
> >>>>>>>>>>
> >>>>>>>>>> 2.
> >>>>>>>>>> Global StateStores can have multiple partitions in their input
> >>>> topics
> >>>>>>>>>> (which function as their changelogs), so they would have more
> than
> >>>> one
> >>>>>>>>>> partition.
> >>>>>>>>>>
> >>>>>>>>>> 3.
> >>>>>>>>>> That's a good point. At present, the only method it adds is
> >>>>>>>>>> isolationLevel(), which is likely not necessary outside of
> >>>>>> StateStores.
> >>>>>>>>>> It *does* provide slightly different guarantees in the
> >> documentation
> >>>>>> to
> >>>>>>>>>> several of the methods (hence the overrides). I'm not sure if
> this
> >>>> is
> >>>>>>>>>> enough to warrant a new interface though.
> >>>>>>>>>> I think the question that remains is whether this interface
> makes
> >> it
> >>>>>>>>>> easier to implement custom transactional StateStores than if we
> >> were
> >>>>>> to
> >>>>>>>>>> remove it? Probably not.
> >>>>>>>>>>
> >>>>>>>>>> 4.
> >>>>>>>>>> The main motivation for the Atomic Checkpointing is actually
> >>>>>>>> performance.
> >>>>>>>>>> My team has been testing out an implementation of this KIP
> without
> >>>> it,
> >>>>>>>> and
> >>>>>>>>>> we had problems with RocksDB doing *much* more compaction, due
> to
> >>>> the
> >>>>>>>>>> significantly increased flush rate. It was enough of a problem
> >> that
> >>>>>> (for
> >>>>>>>>>> the time being), we had to revert back to Kafka Streams proper.
> >>>>>>>>>> I think the best way to solve this, as you say, is to keep the
> >>>>>>>> .checkpoint
> >>>>>>>>>> files *in addition* to the offsets being stored within the store
> >>>>>> itself.
> >>>>>>>>>> Essentially, when closing StateStores, we force a memtable
> flush,
> >>>> then
> >>>>>>>>>> call getCommittedOffsets and write those out to the .checkpoint
> >>>> file.
> >>>>>>>>>> That would ensure the metadata is available to the
> >>>>>>>>>> StreamsPartitionAssignor for all closed stores.
> >>>>>>>>>> If there's a crash (no clean close), then we won't be able to
> >>>>>> guarantee
> >>>>>>>>>> which offsets were flushed to disk by RocksDB, so we'd need to
> >> open
> >>>> (
> >>>>>>>>>> init()), read offsets, and then close() those stores. But since
> >> this
> >>>>>> is
> >>>>>>>>>> the exception, and will only occur once (provided it doesn't
> crash
> >>>>>> every
> >>>>>>>>>> time!), I think the performance impact here would be acceptable.
> >>>>>>>>>>
> >>>>>>>>>> Thanks for the feedback, please let me know if you have any more
> >>>>>>>> comments
> >>>>>>>>>> or questions!
> >>>>>>>>>>
> >>>>>>>>>> I'm currently working on rebasing against trunk. This involves
> >>>> adding
> >>>>>>>>>> support for transactionality to VersionedStateStores. I will
> >>>> probably
> >>>>>>>> need
> >>>>>>>>>> to revise my implementation for transactional "segmented"
> stores,
> >>>> both
> >>>>>>>> to
> >>>>>>>>>> accommodate VersionedStateStore, and to clean up some other
> stuff.
> >>>>>>>>>>
> >>>>>>>>>> Regards,
> >>>>>>>>>> Nick
> >>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org>
> >>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks for the updates!
> >>>>>>>>>>>
> >>>>>>>>>>> I have a couple of questions/comments.
> >>>>>>>>>>>
> >>>>>>>>>>> 1.
> >>>>>>>>>>> Why do you propose a configuration that involves max. bytes and
> >>>> max.
> >>>>>>>>>>> reords? I think we are mainly concerned about memory
> consumption
> >>>>>>>> because
> >>>>>>>>>>> we want to limit the off-heap memory used. I cannot think of a
> >> case
> >>>>>>>>>>> where one would want to set the max. number of records.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 2.
> >>>>>>>>>>> Why does
> >>>>>>>>>>>
> >>>>>>>>>>>        default void commit(final Map<TopicPartition, Long>
> >>>>>>>> changelogOffsets) {
> >>>>>>>>>>>            flush();
> >>>>>>>>>>>        }
> >>>>>>>>>>>
> >>>>>>>>>>> take a map of partitions to changelog offsets?
> >>>>>>>>>>> The mapping between state stores to partitions is a 1:1
> >>>> relationship.
> >>>>>>>>>>> Passing in a single changelog offset should suffice.
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 3.
> >>>>>>>>>>> Why do we need the Transaction interface? It should be possible
> >> to
> >>>>>> hide
> >>>>>>>>>>> beginning and committing a transactions withing the state store
> >>>>>>>>>>> implementation, so that from outside the state store, it does
> not
> >>>>>>>> matter
> >>>>>>>>>>> whether the state store is transactional or not. What would be
> >> the
> >>>>>>>>>>> advantage of using the Transaction interface?
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> 4.
> >>>>>>>>>>> Regarding checkpointing offsets, I think we should keep the
> >>>>>> checkpoint
> >>>>>>>>>>> file in any case for the reason you mentioned about
> rebalancing.
> >>>> Even
> >>>>>>>> if
> >>>>>>>>>>> that would not be an issue, I would propose to move the change
> to
> >>>>>>>> offset
> >>>>>>>>>>> management to a new KIP and to not add more complexity than
> >> needed
> >>>> to
> >>>>>>>>>>> this one. I would not be too concerned about the consistency
> >>>>>> violation
> >>>>>>>>>>> you mention. As far as I understand, with transactional state
> >>>> stores
> >>>>>>>>>>> Streams would write the checkpoint file during every commit
> even
> >>>>>> under
> >>>>>>>>>>> EOS. In the failure case you describe, Streams would restore
> the
> >>>>>> state
> >>>>>>>>>>> stores from the offsets found in the checkpoint file written
> >> during
> >>>>>> the
> >>>>>>>>>>> penultimate commit instead of during the last commit.
> Basically,
> >>>>>>>> Streams
> >>>>>>>>>>> would overwrite the records written to the state store between
> >> the
> >>>>>> last
> >>>>>>>>>>> two commits with the same records read from the changelogs.
> >> While I
> >>>>>>>>>>> understand that this is wasteful, it is -- at the same time --
> >>>>>>>>>>> acceptable and most importantly it does not break EOS.
> >>>>>>>>>>>
> >>>>>>>>>>> Best,
> >>>>>>>>>>> Bruno
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> On 27.04.23 12:34, Nick Telford wrote:
> >>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>
> >>>>>>>>>>>> I find myself (again) considering removing the offset
> management
> >>>>>> from
> >>>>>>>>>>>> StateStores, and keeping the old checkpoint file system. The
> >>>> reason
> >>>>>> is
> >>>>>>>>>>> that
> >>>>>>>>>>>> the StreamPartitionAssignor directly reads checkpoint files in
> >>>> order
> >>>>>>>> to
> >>>>>>>>>>>> determine which instance has the most up-to-date copy of the
> >> local
> >>>>>>>>>>> state.
> >>>>>>>>>>>> If we move offsets into the StateStore itself, then we will
> need
> >>>> to
> >>>>>>>>>>> open,
> >>>>>>>>>>>> initialize, read offsets and then close each StateStore (that
> is
> >>>> not
> >>>>>>>>>>>> already assigned and open) for which we have *any* local
> state,
> >> on
> >>>>>>>> every
> >>>>>>>>>>>> rebalance.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Generally, I don't think there are many "orphan" stores like
> >> this
> >>>>>>>>>>> sitting
> >>>>>>>>>>>> around on most instances, but even a few would introduce
> >>>> additional
> >>>>>>>>>>> latency
> >>>>>>>>>>>> to an already somewhat lengthy rebalance procedure.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I'm leaning towards Colt's (Slack) suggestion of just keeping
> >>>> things
> >>>>>>>> in
> >>>>>>>>>>> the
> >>>>>>>>>>>> checkpoint file(s) for now, and not worrying about the race.
> The
> >>>>>>>>>>> downside
> >>>>>>>>>>>> is that we wouldn't be able to remove the explicit RocksDB
> flush
> >>>>>>>>>>> on-commit,
> >>>>>>>>>>>> which likely hurts performance.
> >>>>>>>>>>>>
> >>>>>>>>>>>> If anyone has any thoughts or ideas on this subject, I would
> >>>>>>>> appreciate
> >>>>>>>>>>> it!
> >>>>>>>>>>>>
> >>>>>>>>>>>> Regards,
> >>>>>>>>>>>> Nick
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Wed, 19 Apr 2023 at 15:05, Nick Telford <
> >>>> nick.telford@gmail.com>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> The issue is that if there's a crash between 2 and 3, then
> you
> >>>>>> still
> >>>>>>>>>>> end
> >>>>>>>>>>>>> up with inconsistent data in RocksDB. The only way to
> guarantee
> >>>>>> that
> >>>>>>>>>>> your
> >>>>>>>>>>>>> checkpoint offsets and locally stored data are consistent
> with
> >>>> each
> >>>>>>>>>>> other
> >>>>>>>>>>>>> are to atomically commit them, which can be achieved by
> having
> >>>> the
> >>>>>>>>>>> offsets
> >>>>>>>>>>>>> stored in RocksDB.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> The offsets column family is likely to be extremely small
> (one
> >>>>>>>>>>>>> per-changelog partition + one per Topology input partition
> for
> >>>>>>>> regular
> >>>>>>>>>>>>> stores, one per input partition for global stores). So the
> >>>> overhead
> >>>>>>>>>>> will be
> >>>>>>>>>>>>> minimal.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> A major benefit of doing this is that we can remove the
> >> explicit
> >>>>>>>> calls
> >>>>>>>>>>> to
> >>>>>>>>>>>>> db.flush(), which forcibly flushes memtables to disk
> on-commit.
> >>>> It
> >>>>>>>>>>> turns
> >>>>>>>>>>>>> out, RocksDB memtable flushes are largely dictated by Kafka
> >>>> Streams
> >>>>>>>>>>>>> commits, *not* RocksDB configuration, which could be a major
> >>>> source
> >>>>>>>> of
> >>>>>>>>>>>>> confusion. Atomic checkpointing makes it safe to remove these
> >>>>>>>> explicit
> >>>>>>>>>>>>> flushes, because it no longer matters exactly when RocksDB
> >>>> flushes
> >>>>>>>>>>> data to
> >>>>>>>>>>>>> disk; since the data and corresponding checkpoint offsets
> will
> >>>>>> always
> >>>>>>>>>>> be
> >>>>>>>>>>>>> flushed together, the local store is always in a consistent
> >>>> state,
> >>>>>>>> and
> >>>>>>>>>>>>> on-restart, it can always safely resume restoration from the
> >>>>>> on-disk
> >>>>>>>>>>>>> offsets, restoring the small amount of data that hadn't been
> >>>>>> flushed
> >>>>>>>>>>> when
> >>>>>>>>>>>>> the app exited/crashed.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <
> >> colt@littlehorse.io>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks for your reply. Ack to A) and B).
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> For item C), I see what you're referring to. Your proposed
> >>>>>> solution
> >>>>>>>>>>> will
> >>>>>>>>>>>>>> work, so no need to change it. What I was suggesting was
> that
> >> it
> >>>>>>>>>>> might be
> >>>>>>>>>>>>>> possible to achieve this with only one column family. So
> long
> >>>> as:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>          - No uncommitted records (i.e. not committed to the
> >>>>>> changelog)
> >>>>>>>> are
> >>>>>>>>>>>>>>          *committed* to the state store, AND
> >>>>>>>>>>>>>>          - The Checkpoint offset (which refers to the
> changelog
> >>>>>> topic)
> >>>>>>>> is
> >>>>>>>>>>> less
> >>>>>>>>>>>>>>          than or equal to the last written changelog offset
> in
> >>>>>> rocksdb
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I don't see the need to do the full restoration from
> scratch.
> >> My
> >>>>>>>>>>>>>> understanding was that prior to 844/892, full restorations
> >> were
> >>>>>>>>>>> required
> >>>>>>>>>>>>>> because there could be uncommitted records written to
> RocksDB;
> >>>>>>>>>>> however,
> >>>>>>>>>>>>>> given your use of RocksDB transactions, that can be avoided
> >> with
> >>>>>> the
> >>>>>>>>>>>>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
> >>>>>>>>>>> transaction, 3)
> >>>>>>>>>>>>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <
> >>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> A. I've done my best to de-couple the StateStore stuff from
> >> the
> >>>>>>>> rest
> >>>>>>>>>>> of
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> Streams engine. The fact that there will be only one
> ongoing
> >>>>>>>> (write)
> >>>>>>>>>>>>>>> transaction at a time is not guaranteed by any API, and is
> >>>> just a
> >>>>>>>>>>>>>>> consequence of the way Streams operates. To that end, I
> tried
> >>>> to
> >>>>>>>>>>> ensure
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> documentation and guarantees provided by the new APIs are
> >>>>>>>>>>> independent of
> >>>>>>>>>>>>>>> this incidental behaviour. In practice, you're right, this
> >>>>>>>>>>> essentially
> >>>>>>>>>>>>>>> refers to "interactive queries", which are technically
> "read
> >>>>>>>>>>>>>> transactions",
> >>>>>>>>>>>>>>> even if they don't actually use the transaction API to
> >> isolate
> >>>>>>>>>>>>>> themselves.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> B. Yes, although not ideal. This is for backwards
> >>>> compatibility,
> >>>>>>>>>>>>>> because:
> >>>>>>>>>>>>>>>           1) Existing custom StateStore implementations
> will
> >>>>>> implement
> >>>>>>>>>>>>>> flush(),
> >>>>>>>>>>>>>>> and not commit(), but the Streams engine now calls
> commit(),
> >> so
> >>>>>>>> those
> >>>>>>>>>>>>>> calls
> >>>>>>>>>>>>>>> need to be forwarded to flush() for these legacy stores.
> >>>>>>>>>>>>>>>           2) Existing StateStore *users*, i.e. outside of
> the
> >>>>>> Streams
> >>>>>>>>>>> engine
> >>>>>>>>>>>>>>> itself, may depend on explicitly calling flush(), so for
> >> these
> >>>>>>>> cases,
> >>>>>>>>>>>>>>> flush() needs to be redirected to call commit().
> >>>>>>>>>>>>>>> If anyone has a better way to guarantee compatibility
> without
> >>>>>>>>>>>>>> introducing
> >>>>>>>>>>>>>>> this potential recursion loop, I'm open to changes!
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> C. This is described in the "Atomic Checkpointing" section.
> >>>>>> Offsets
> >>>>>>>>>>> are
> >>>>>>>>>>>>>>> stored in a separate RocksDB column family, which is
> >> guaranteed
> >>>>>> to
> >>>>>>>> be
> >>>>>>>>>>>>>>> atomically flushed to disk with all other column families.
> >> The
> >>>>>>>> issue
> >>>>>>>>>>> of
> >>>>>>>>>>>>>>> checkpoints being written to disk after commit causing
> >>>>>>>> inconsistency
> >>>>>>>>>>> if
> >>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>> crashes in between is the reason why, under EOS, checkpoint
> >>>> files
> >>>>>>>> are
> >>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>> written on clean shutdown. This is one of the major causes
> of
> >>>>>> "full
> >>>>>>>>>>>>>>> restorations", so moving the offsets into a place where
> they
> >>>> can
> >>>>>> be
> >>>>>>>>>>>>>>> guaranteed to be atomically written with the data they
> >>>> checkpoint
> >>>>>>>>>>>>>> allows us
> >>>>>>>>>>>>>>> to write the checkpoint offsets *on every commit*, not just
> >> on
> >>>>>>>> clean
> >>>>>>>>>>>>>>> shutdown.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <
> >>>> colt@littlehorse.io>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thank you for continuing this work. I have a few minor
> >>>>>> clarifying
> >>>>>>>>>>>>>>>> questions.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> A) "Records written to any transaction are visible to all
> >>>> other
> >>>>>>>>>>>>>>>> transactions immediately." I am confused here—I thought
> >> there
> >>>>>>>> could
> >>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>> one transaction going on at a time for a given state store
> >>>> given
> >>>>>>>> the
> >>>>>>>>>>>>>>>> threading model for processing records on a Task. Do you
> >> mean
> >>>>>>>>>>>>>> Interactive
> >>>>>>>>>>>>>>>> Queries by "other transactions"? (If so, then everything
> >> makes
> >>>>>>>>>>> sense—I
> >>>>>>>>>>>>>>>> thought that since IQ were read-only then they didn't
> count
> >> as
> >>>>>>>>>>>>>>>> transactions).
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> B) Is it intentional that the default implementations of
> the
> >>>>>>>> flush()
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>> commit() methods in the StateStore class refer to each
> other
> >>>> in
> >>>>>>>> some
> >>>>>>>>>>>>>> sort
> >>>>>>>>>>>>>>>> of unbounded recursion?
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> C) How will the getCommittedOffset() method work? At
> first I
> >>>>>>>> thought
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> way to do it would be using a special key in the RocksDB
> >> store
> >>>>>> to
> >>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> offset, and committing that with the transaction. But upon
> >>>>>> second
> >>>>>>>>>>>>>>> thought,
> >>>>>>>>>>>>>>>> since restoration from the changelog is an idempotent
> >>>>>> procedure, I
> >>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>> would be fine to 1) commit the RocksDB transaction and
> then
> >> 2)
> >>>>>>>> write
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> offset to disk in a checkpoint file. If there is a crash
> >>>> between
> >>>>>>>> 1)
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>> 2),
> >>>>>>>>>>>>>>>> I think the only downside is 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 <
> >>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I've updated the KIP to reflect the latest version of the
> >>>>>> design:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> There are several changes in there that reflect feedback
> >> from
> >>>>>>>> this
> >>>>>>>>>>>>>>>> thread,
> >>>>>>>>>>>>>>>>> and there's a new section and a bunch of interface
> changes
> >>>>>>>> relating
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> Atomic Checkpointing, which is the final piece of the
> >> puzzle
> >>>> to
> >>>>>>>>>>>>>> making
> >>>>>>>>>>>>>>>>> everything robust.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Let me know what you think!
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <
> >>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi Lucas,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thanks for looking over my KIP.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> A) The bound is per-instance, not per-Task. This was a
> >> typo
> >>>> in
> >>>>>>>> the
> >>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>> that I've now corrected. It was originally per-Task,
> but I
> >>>>>>>>>>>>>> changed it
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> per-instance for exactly the reason you highlighted.
> >>>>>>>>>>>>>>>>>> B) It's worth noting that transactionality is only
> enabled
> >>>>>> under
> >>>>>>>>>>>>>> EOS,
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> in the default mode of operation (ALOS), there should be
> >> no
> >>>>>>>>>>>>>> change in
> >>>>>>>>>>>>>>>>>> behavior at all. I think, under EOS, we can mitigate the
> >>>>>> impact
> >>>>>>>> on
> >>>>>>>>>>>>>>>> users
> >>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>> sufficiently low default values for the memory bound
> >>>>>>>>>>>>>> configuration. I
> >>>>>>>>>>>>>>>>>> understand your hesitation to include a significant
> change
> >>>> of
> >>>>>>>>>>>>>>>> behaviour,
> >>>>>>>>>>>>>>>>>> especially in a minor release, but I suspect that most
> >> users
> >>>>>>>> will
> >>>>>>>>>>>>>>>> prefer
> >>>>>>>>>>>>>>>>>> the memory impact (under EOS) to the existing behaviour
> of
> >>>>>>>>>>>>>> frequent
> >>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>> restorations! If this is a problem, the changes can wait
> >>>> until
> >>>>>>>> the
> >>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>> major release. I'll be running a patched version of
> >> streams
> >>>> in
> >>>>>>>>>>>>>>>> production
> >>>>>>>>>>>>>>>>>> with these changes as soon as they're ready, so it won't
> >>>>>> disrupt
> >>>>>>>>>>>>>> me
> >>>>>>>>>>>>>>> :-D
> >>>>>>>>>>>>>>>>>> C) The main purpose of this sentence was just to note
> that
> >>>>>> some
> >>>>>>>>>>>>>>> changes
> >>>>>>>>>>>>>>>>>> will need to be made to the way Segments are handled in
> >>>> order
> >>>>>> to
> >>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>> they also benefit from transactions. At the time I wrote
> >>>> it, I
> >>>>>>>>>>>>>> hadn't
> >>>>>>>>>>>>>>>>>> figured out the specific changes necessary, so it was
> >>>>>>>> deliberately
> >>>>>>>>>>>>>>>> vague.
> >>>>>>>>>>>>>>>>>> This is the one outstanding problem I'm currently
> working
> >>>> on,
> >>>>>>>> and
> >>>>>>>>>>>>>>> I'll
> >>>>>>>>>>>>>>>>>> update this section with more detail once I have figured
> >> out
> >>>>>> the
> >>>>>>>>>>>>>>> exact
> >>>>>>>>>>>>>>>>>> changes required.
> >>>>>>>>>>>>>>>>>> D) newTransaction() provides the necessary isolation
> >>>>>> guarantees.
> >>>>>>>>>>>>>>> While
> >>>>>>>>>>>>>>>>>> the RocksDB implementation of transactions doesn't
> >>>> technically
> >>>>>>>>>>>>>> *need*
> >>>>>>>>>>>>>>>>>> read-only users to call newTransaction(), other
> >>>>>> implementations
> >>>>>>>>>>>>>>> (e.g. a
> >>>>>>>>>>>>>>>>>> hypothetical PostgresStore) may require it. Calling
> >>>>>>>>>>>>>> newTransaction()
> >>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>> no transaction is necessary is essentially free, as it
> >> will
> >>>>>> just
> >>>>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I
> think
> >> it
> >>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>> fairly obvious where the performance problems stem from:
> >>>>>> writes
> >>>>>>>>>>>>>> under
> >>>>>>>>>>>>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it
> with
> >>>> the
> >>>>>>>>>>>>>>>>>> tombstone/record flag, 1 to decode it from the
> >>>>>> tombstone/record
> >>>>>>>>>>>>>> flag,
> >>>>>>>>>>>>>>>>> and 1
> >>>>>>>>>>>>>>>>>> to copy the record from the "temporary" store to the
> >> "main"
> >>>>>>>> store,
> >>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> transaction commits. The different approach taken by
> >> KIP-869
> >>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>> perform
> >>>>>>>>>>>>>>>>>> much better, as it avoids all these copies, and may
> >> actually
> >>>>>>>>>>>>>> perform
> >>>>>>>>>>>>>>>>>> slightly better than trunk, due to batched writes in
> >> RocksDB
> >>>>>>>>>>>>>>> performing
> >>>>>>>>>>>>>>>>>> better than non-batched writes.[1]
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 1:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
> >>>>>>>>>>>>>> lbrutschy@confluent.io
> >>>>>>>>>>>>>>>>> .invalid>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I'm just starting to read up on the whole discussion
> >> about
> >>>>>>>>>>>>>> KIP-892
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
> >>>>>>>>>>>>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do
> >> have
> >>>>>> some
> >>>>>>>>>>>>>>>>>>> questions about the latest draft.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>        A) If I understand correctly, you propose to
> put a
> >>>> bound
> >>>>>> on
> >>>>>>>> the
> >>>>>>>>>>>>>>>>>>> (native) memory consumed by each task. However, I
> wonder
> >> if
> >>>>>>>> this
> >>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> sufficient if we have temporary imbalances in the
> >> cluster.
> >>>>>> For
> >>>>>>>>>>>>>>>>>>> example, depending on the timing of rebalances during a
> >>>>>> cluster
> >>>>>>>>>>>>>>>>>>> restart, it could happen that a single streams node is
> >>>>>>>> assigned a
> >>>>>>>>>>>>>>> lot
> >>>>>>>>>>>>>>>>>>> more tasks than expected. With your proposed change,
> this
> >>>>>> would
> >>>>>>>>>>>>>> mean
> >>>>>>>>>>>>>>>>>>> that the memory required by this one node could be a
> >>>> multiple
> >>>>>>>> of
> >>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>>>> is required during normal operation. I wonder if it
> >>>> wouldn't
> >>>>>> be
> >>>>>>>>>>>>>>> safer
> >>>>>>>>>>>>>>>>>>> to put a global bound on the memory use, across all
> >> tasks.
> >>>>>>>>>>>>>>>>>>>        B) Generally, the memory concerns still give me
> the
> >>>>>> feeling
> >>>>>>>>>>> that
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> should not be enabled by default for all users in a
> minor
> >>>>>>>>>>>>>> release.
> >>>>>>>>>>>>>>>>>>>        C) In section "Transaction Management": the
> >> sentence
> >>>> "A
> >>>>>>>> similar
> >>>>>>>>>>>>>>>>>>> analogue will be created to automatically manage
> >> `Segment`
> >>>>>>>>>>>>>>>>>>> transactions.". Maybe this is just me lacking some
> >>>>>> background,
> >>>>>>>>>>>>>> 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 you to find out why the KIP-844 PoC is 20x
> >>>>>> slower
> >>>>>>>>>>>>>> (e.g.
> >>>>>>>>>>>>>>>>>>> by providing a flame graph), that would be quite
> >>>> interesting.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>>>> Lucas
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
> >>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I've updated the KIP with a more detailed design,
> which
> >>>>>>>>>>>>>> reflects
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> implementation I've been working on:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> This new design should address the outstanding points
> >>>>>> already
> >>>>>>>>>>>>>> made
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Please let me know if there are areas that are unclear
> >> or
> >>>>>> need
> >>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>>> clarification.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I have a (nearly) working implementation. I'm
> confident
> >>>> that
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> remaining
> >>>>>>>>>>>>>>>>>>>> work (making Segments behave) will not impact the
> >>>> documented
> >>>>>>>>>>>>>>> design.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <
> >>>>>>>> colt@littlehorse.io
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thank you for the reply; that makes sense. I was
> hoping
> >>>>>> that,
> >>>>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>>>> reading
> >>>>>>>>>>>>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
> >>>>>>>>>>>>>> documented
> >>>>>>>>>>>>>>>> API,
> >>>>>>>>>>>>>>>>>>> maybe
> >>>>>>>>>>>>>>>>>>>>> you *wouldn't* have to wait for the next major
> release
> >> to
> >>>>>>>>>>>>>> make
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> change;
> >>>>>>>>>>>>>>>>>>>>> but given 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 <
> >>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally
> intended
> >>>> to
> >>>>>>>>>>>>>> try
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> keep the
> >>>>>>>>>>>>>>>>>>>>>> behaviour unchanged as much as possible, otherwise
> >> we'd
> >>>>>>>>>>>>>> have
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> wait for
> >>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>> major version release to land these changes.
> >>>>>>>>>>>>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
> >>>>>>>>>>>>>> guarantee,
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> typically longer commit intervals would be
> problematic
> >>>>>> when
> >>>>>>>>>>>>>>>>> reading
> >>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>> "committed" records.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I've been away for 5 days recovering from minor
> >> surgery,
> >>>>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>>>> spent a
> >>>>>>>>>>>>>>>>>>>>>> considerable amount of that time working through
> ideas
> >>>> for
> >>>>>>>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>>>>>>>> solutions in my head. I think your suggestion of
> >> keeping
> >>>>>>>>>>>>>> ALOS
> >>>>>>>>>>>>>>>>>>> as-is, but
> >>>>>>>>>>>>>>>>>>>>>> buffering writes for EOS is the right path forwards,
> >>>>>>>>>>>>>> although
> >>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>>>> solution that both expands on this, and provides for
> >>>> some
> >>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>> formal
> >>>>>>>>>>>>>>>>>>>>>> guarantees.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Essentially, adding support to KeyValueStores for
> >>>>>>>>>>>>>>>> "Transactions",
> >>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>> clearly defined IsolationLevels. Using "Read
> >> Committed"
> >>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>> under
> >>>>>>>>>>>>>>>>>>> EOS,
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>> "Read Uncommitted" under ALOS.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> The nice thing about this approach is that it gives
> us
> >>>>>> much
> >>>>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>>> clearly
> >>>>>>>>>>>>>>>>>>>>>> defined isolation behaviour that can be properly
> >>>>>>>>>>>>>> documented to
> >>>>>>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>>>>> users
> >>>>>>>>>>>>>>>>>>>>>> know what to expect.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I'm still working out the kinks in the design, and
> >> will
> >>>>>>>>>>>>>> update
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>>>> I have something. The main struggle is trying to
> >>>> implement
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>>>>>> making any major changes to the existing interfaces
> or
> >>>>>>>>>>>>>>> breaking
> >>>>>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>>>>> implementations, because currently everything
> expects
> >> to
> >>>>>>>>>>>>>>> operate
> >>>>>>>>>>>>>>>>>>> directly
> >>>>>>>>>>>>>>>>>>>>>> on a StateStore, and not a Transaction of that
> store.
> >> I
> >>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>>>>>>> close, although sadly I won't be able to progress
> much
> >>>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>>>> week
> >>>>>>>>>>>>>>>>>>>>> due
> >>>>>>>>>>>>>>>>>>>>>> to some work commitments.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
> >>>>>>>>>>>>>>> colt@littlehorse.io>
> >>>>>

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

Posted by John Roesler <vv...@apache.org>.
Oh, that's a good point.

On the topic of a behavioral switch for disabled caches, the typical use 
case for disabling the cache is to cause each individual update to 
propagate down the topology, so another thought might be to just go 
ahead and add the memory we would have used for the memtables to the 
cache size, but if people did disable the cache entirely, then we could 
still go ahead and forward the records on each write?

I know that Guozhang was also proposing for a while to actually decouple 
caching and forwarding, which might provide a way to side-step this 
dilemma (i.e., we just always forward and only apply the cache to state 
and changelog writes).

By the way, I'm basing my statement on why you'd disable caches on 
memory, but also on the guidance here: 
https://docs.confluent.io/platform/current/streams/developer-guide/memory-mgmt.html 
. That doc also contains a section on how to bound the total memory 
usage across RocksDB memtables, which points to another benefit of 
disabling memtables and managing the write buffer ourselves (simplified 
memory configuration).

Thanks,
-John

On 6/20/23 16:05, Nick Telford wrote:
> Potentially we could just go the memorable with Rocks WriteBatches route if
> the cache is disabled?
> 
> On Tue, 20 Jun 2023, 22:00 John Roesler, <jo...@vvcephei.org> wrote:
> 
>> Touché!
>>
>> Ok, I agree that figuring out the case of a disabled cache would be
>> non-trivial. Ingesting single-record SST files will probably not be
>> performant, but benchmarking may prove different. Or maybe we can have
>> some reserved cache space on top of the user-configured cache, which we
>> would have reclaimed from the memtable space. Or some other, more
>> creative solution.
>>
>> Thanks,
>> -John
>>
>> On 6/20/23 15:30, Nick Telford wrote:
>>>> Note that users can disable the cache, which would still be
>>> ok, I think. We wouldn't ingest the SST files on every record, but just
>>> append to them and only ingest them on commit, when we're already
>>> waiting for acks and a RocksDB commit.
>>>
>>> In this case, how would uncommitted records be read by joins?
>>>
>>> On Tue, 20 Jun 2023, 20:51 John Roesler, <vv...@apache.org> wrote:
>>>
>>>> Ah, sorry Nick,
>>>>
>>>> I just meant the regular heap based cache that we maintain in Streams. I
>>>> see that it's not called "RecordCache" (my mistake).
>>>>
>>>> The actual cache is ThreadCache:
>>>>
>>>>
>> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
>>>>
>>>> Here's the example of how we use the cache in KeyValueStore:
>>>>
>>>>
>> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
>>>>
>>>> It's basically just an on-heap Map of records that have not yet been
>>>> written to the changelog or flushed into the underlying store. It gets
>>>> flushed when the total cache size exceeds `cache.max.bytes.buffering` or
>>>> the `commit.interval.ms` elapses.
>>>>
>>>> Speaking of those configs, another benefit to this idea is that we would
>>>> no longer need to trigger extra commits based on the size of the ongoing
>>>> transaction. Instead, we'd just preserve the existing cache-flush
>>>> behavior. Note that users can disable the cache, which would still be
>>>> ok, I think. We wouldn't ingest the SST files on every record, but just
>>>> append to them and only ingest them on commit, when we're already
>>>> waiting for acks and a RocksDB commit.
>>>>
>>>> Thanks,
>>>> -John
>>>>
>>>> On 6/20/23 14:09, Nick Telford wrote:
>>>>> Hi John,
>>>>>
>>>>> By "RecordCache", do you mean the RocksDB "WriteBatch"? I can't find
>> any
>>>>> class called "RecordCache"...
>>>>>
>>>>> Cheers,
>>>>>
>>>>> Nick
>>>>>
>>>>> On Tue, 20 Jun 2023 at 19:42, John Roesler <vv...@apache.org>
>> wrote:
>>>>>
>>>>>> Hi Nick,
>>>>>>
>>>>>> Thanks for picking this up again!
>>>>>>
>>>>>> I did have one new thought over the intervening months, which I'd like
>>>>>> your take on.
>>>>>>
>>>>>> What if, instead of using the RocksDB atomic write primitive at all,
>> we
>>>>>> instead just:
>>>>>> 1. disable memtables entirely
>>>>>> 2. directly write the RecordCache into SST files when we flush
>>>>>> 3. atomically ingest the SST file(s) into RocksDB when we get the ACK
>>>>>> from the changelog (see
>>>>>>
>>>>>>
>>>>
>> https://github.com/EighteenZi/rocksdb_wiki/blob/master/Creating-and-Ingesting-SST-files.md
>>>>>> and
>>>>>>
>>>>>>
>>>>
>> https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/IngestExternalFileOptions.java
>>>>>> and
>>>>>>
>>>>>>
>>>>
>> https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L1413-L1429
>>>>>> )
>>>>>> 4. track the changelog offsets either in another CF or the same CF
>> with
>>>>>> a reserved key, either of which will make the changelog offset update
>>>>>> atomic with the file ingestions
>>>>>>
>>>>>> I suspect this'll have a number of benefits:
>>>>>> * writes to RocksDB will always be atomic
>>>>>> * we don't fragment memory between the RecordCache and the memtables
>>>>>> * RecordCache gives far higher performance than memtable for reads and
>>>>>> writes
>>>>>> * we don't need any new "transaction" concepts or memory bound configs
>>>>>>
>>>>>> What do you think?
>>>>>>
>>>>>> Thanks,
>>>>>> -John
>>>>>>
>>>>>> On 6/20/23 10:51, Nick Telford wrote:
>>>>>>> Hi Bruno,
>>>>>>>
>>>>>>> Thanks for reviewing the KIP. It's been a long road, I started
>> working
>>>> on
>>>>>>> this more than a year ago, and most of the time in the last 6 months
>>>> has
>>>>>>> been spent on the "Atomic Checkpointing" stuff that's been benched,
>> so
>>>>>> some
>>>>>>> of the reasoning behind some of my decisions have been lost, but I'll
>>>> do
>>>>>> my
>>>>>>> best to reconstruct them.
>>>>>>>
>>>>>>> 1.
>>>>>>> IIRC, this was the initial approach I tried. I don't remember the
>> exact
>>>>>>> reasons I changed it to use a separate "view" of the StateStore that
>>>>>>> encapsulates the transaction, but I believe it had something to do
>> with
>>>>>>> concurrent access to the StateStore from Interactive Query threads.
>>>> Reads
>>>>>>> from interactive queries need to be isolated from the currently
>> ongoing
>>>>>>> transaction, both for consistency (so interactive queries don't
>> observe
>>>>>>> changes that are subsequently rolled-back), but also to prevent
>>>> Iterators
>>>>>>> opened by an interactive query from being closed and invalidated by
>> the
>>>>>>> StreamThread when it commits the transaction, which causes your
>>>>>> interactive
>>>>>>> queries to crash.
>>>>>>>
>>>>>>> Another reason I believe I implemented it this way was a separation
>> of
>>>>>>> concerns. Recall that newTransaction() originally created an object
>> of
>>>>>> type
>>>>>>> Transaction, not StateStore. My intent was to improve the type-safety
>>>> of
>>>>>>> the API, in an effort to ensure Transactions weren't used
>> incorrectly.
>>>>>>> Unfortunately, this didn't pan out, but newTransaction() remained.
>>>>>>>
>>>>>>> Finally, this had the added benefit that implementations could easily
>>>> add
>>>>>>> support for transactions *without* re-writing their existing,
>>>>>>> non-transactional implementation. I think this can be a benefit both
>>>> for
>>>>>>> implementers of custom StateStores, but also for anyone extending
>>>>>>> RocksDbStore, as they can rely on the existing access methods working
>>>> how
>>>>>>> they expect them to.
>>>>>>>
>>>>>>> I'm not too happy with the way the current design has panned out, so
>>>> I'm
>>>>>>> open to ideas on how to improve it. Key to this is finding some way
>> to
>>>>>>> ensure that reads from Interactive Query threads are properly
>> isolated
>>>>>> from
>>>>>>> the transaction, *without* the performance overhead of checking which
>>>>>>> thread the method is being called from on every access.
>>>>>>>
>>>>>>> As for replacing flush() with commit() - I saw no reason to add this
>>>>>>> complexity to the KIP, unless there was a need to add arguments to
>> the
>>>>>>> flush/commit method. This need arises with Atomic Checkpointing, but
>>>> that
>>>>>>> will be implemented separately, in a future KIP. Do you see a need
>> for
>>>>>> some
>>>>>>> arguments to the flush/commit method that I've missed? Or were you
>>>> simply
>>>>>>> suggesting a rename?
>>>>>>>
>>>>>>> 2.
>>>>>>> This is simply due to the practical reason that isolationLevel() is
>>>>>> really
>>>>>>> a proxy for checking if the app is under EOS. The application
>>>>>> configuration
>>>>>>> is not provided to the constructor of StateStores, but it *is*
>> provided
>>>>>> to
>>>>>>> init(), via StateStoreContext. For this reason, it seemed somewhat
>>>>>> natural
>>>>>>> to add it to StateStoreContext. I think this makes sense, since the
>>>>>>> IsolationLevel of all StateStores in an application *must* be the
>> same,
>>>>>> and
>>>>>>> since those stores are all initialized with the same
>> StateStoreContext,
>>>>>> it
>>>>>>> seems natural for that context to carry the desired IsolationLevel to
>>>>>> use.
>>>>>>>
>>>>>>> 3.
>>>>>>> Using IsolationLevel instead of just passing `boolean eosEnabled`,
>> like
>>>>>>> much of the internals was an attempt to logically de-couple the
>>>>>> StateStore
>>>>>>> API from the internals of Kafka Streams. Technically, StateStores
>> don't
>>>>>>> need to know/care what processing mode the KS app is using, all they
>>>> need
>>>>>>> to know is the isolation level expected of them.
>>>>>>>
>>>>>>> Having formal definitions for the expectations of the two required
>>>>>>> IsolationLevels allow implementers to implement transactional stores
>>>>>>> without having to dig through the internals of Kafka Streams and
>>>>>> understand
>>>>>>> exactly how they are used. The tight coupling between state stores
>> and
>>>>>>> internal behaviour has actually significantly hindered my progress on
>>>>>> this
>>>>>>> KIP, and encouraged me to avoid increasing this logical coupling as
>>>> much
>>>>>> as
>>>>>>> possible.
>>>>>>>
>>>>>>> This also frees implementations to satisfy those requirements in any
>>>> way
>>>>>>> they choose. Transactions might not be the only/available approach to
>>>> an
>>>>>>> implementation, but they might have an alternative way to satisfy the
>>>>>>> isolation requirements. I admit that this point is more about
>>>> semantics,
>>>>>>> but "transactional" would need to be formally defined in order for
>>>>>>> implementers to provide a valid implementation, and these
>>>> IsolationLevels
>>>>>>> provide that formal definition.
>>>>>>>
>>>>>>> 4.
>>>>>>> I can remove them. I added them only as I planned to include them in
>>>> the
>>>>>>> org.apache.kafka.streams.state package, as a recommended base
>>>>>>> implementation for all StateStores, including those implemented by
>>>>>> users. I
>>>>>>> had assumed that anything in "public" packages, such as
>>>>>>> org.apache.kafka.streams.state, should be included in a KIP. Is that
>>>>>> wrong?
>>>>>>>
>>>>>>> 5.
>>>>>>> RocksDB provides no way to measure the actual size of a
>>>>>>> WriteBatch(WithIndex), so we're limited to tracking the sum total of
>>>> the
>>>>>>> size of keys + values that are written to the transaction. This
>>>> obviously
>>>>>>> under-estimates the actual memory usage, because WriteBatch no-doubt
>>>>>>> includes some record overheads, and WriteBatchWithIndex has to
>> maintain
>>>>>> an
>>>>>>> index.
>>>>>>>
>>>>>>> Ideally, we could trivially add a method upstream to
>>>> WriteBatchInterface
>>>>>>> that provides the exact size of the batch, but that would require an
>>>>>>> upgrade of RocksDB, which won't happen soon. So for the time being,
>>>> we're
>>>>>>> stuck with an approximation, so I felt that the new method should
>>>> reflect
>>>>>>> that.
>>>>>>>
>>>>>>> Would you prefer the new method name ignores this constraint and that
>>>> we
>>>>>>> simply make the rocks measurement more accurate in the future?
>>>>>>>
>>>>>>> 6.
>>>>>>> Done
>>>>>>>
>>>>>>> 7.
>>>>>>> Very good point. The KIP already specifically calls out memory in the
>>>>>>> documentation of the config: "Maximum number of memory bytes to be
>> used
>>>>>> to
>>>>>>> buffer uncommitted state-store records." - did you have something
>> else
>>>> in
>>>>>>> mind?
>>>>>>>
>>>>>>> Should we also make this clearer by renaming the config property
>>>> itself?
>>>>>>> Perhaps to something like statestore.transaction.buffer.max.bytes?
>>>>>>>
>>>>>>> 8.
>>>>>>> OK, I can remove this. The intent here was to describe how Streams
>>>> itself
>>>>>>> will manage transaction roll-over etc. Presumably that means we also
>>>>>> don't
>>>>>>> need a description of how Streams will manage the commit of changelog
>>>>>>> transactions, state store transactions and checkpointing?
>>>>>>>
>>>>>>> 9.
>>>>>>> What do you mean by fail-over? Do you mean failing over an Active
>> Task
>>>> to
>>>>>>> an instance already hosting a Standby Task?
>>>>>>>
>>>>>>> Thanks again and sorry for the essay of a response!
>>>>>>>
>>>>>>> Regards,
>>>>>>> Nick
>>>>>>>
>>>>>>> On Tue, 20 Jun 2023 at 10:49, Bruno Cadonna <ca...@apache.org>
>>>> wrote:
>>>>>>>
>>>>>>>> Hi Nick,
>>>>>>>>
>>>>>>>> Thanks for the updates!
>>>>>>>>
>>>>>>>> I really appreciate that you simplified the KIP by removing some
>>>>>>>> aspects. As I have already told you, I think the removed aspects are
>>>>>>>> also good ideas and we can discuss them on follow-up KIPs.
>>>>>>>>
>>>>>>>> Regarding the current KIP, I have the following feedback.
>>>>>>>>
>>>>>>>> 1.
>>>>>>>> Is there a good reason to add method newTransaction() to the
>>>> StateStore
>>>>>>>> interface? As far as I understand, the idea is that users of a state
>>>>>>>> store (transactional or not) call this method at start-up and after
>>>> each
>>>>>>>> commit. Since the call to newTransaction() is done in any case and I
>>>>>>>> think it would simplify the caller code if we just start a new
>>>>>>>> transaction after a commit in the implementation?
>>>>>>>> As far as I understand, you plan to commit the transaction in the
>>>>>>>> flush() method. I find the idea to replace flush() with commit()
>>>>>>>> presented in KIP-844 an elegant solution.
>>>>>>>>
>>>>>>>> 2.
>>>>>>>> Why is the method to query the isolation level added to the state
>>>> store
>>>>>>>> context?
>>>>>>>>
>>>>>>>> 3.
>>>>>>>> Do we need all the isolation level definitions? I think it is good
>> to
>>>>>>>> know the guarantees of the transactionality of the state store.
>>>>>>>> However, currently, Streams guarantees that there will only be one
>>>>>>>> transaction that writes to the state store. Only the stream thread
>>>> that
>>>>>>>> executes the active task that owns the state store will write to the
>>>>>>>> state store. I think it should be enough to know if the state store
>> is
>>>>>>>> transactional or not. So my proposal would be to just add a method
>> on
>>>>>>>> the state store interface the returns if a state store is
>>>> transactional
>>>>>>>> or not by returning a boolean or an enum.
>>>>>>>>
>>>>>>>> 4.
>>>>>>>> I am wondering why AbstractTransaction and
>> AbstractTransactionalStore
>>>>>>>> are part of the KIP. They look like implementation details that
>> should
>>>>>>>> not be exposed in the public API.
>>>>>>>>
>>>>>>>> 5.
>>>>>>>> Why does StateStore#approximateNumUncommittedBytes() return an
>>>>>>>> approximate number of bytes?
>>>>>>>>
>>>>>>>> 6.
>>>>>>>> RocksDB is just one implementation of the state stores in Streams.
>>>>>>>> However, the issues regarding OOM errors might also apply to other
>>>>>>>> custom implementations. So in the KIP I would extract that part from
>>>>>>>> section "RocksDB Transaction". I would also move section "RocksDB
>>>>>>>> Transaction" to the end of section "Proposed Changes" and handle it
>> as
>>>>>>>> an example implementation for a state store.
>>>>>>>>
>>>>>>>> 7.
>>>>>>>> Should statestore.uncommitted.max.bytes only limit the uncommitted
>>>> bytes
>>>>>>>> or the uncommitted bytes that reside in memory? In future, other
>>>>>>>> transactional state store implementations might implement a buffer
>> for
>>>>>>>> uncommitted records that are able to spill records on disk. I think
>>>>>>>> statestore.uncommitted.max.bytes needs to limit the uncommitted
>> bytes
>>>>>>>> irrespective if they reside in memory or disk. Since Streams will
>> use
>>>>>>>> this config to decide if it needs to trigger a commit, state store
>>>>>>>> implementations that can spill to disk will never be able to spill
>> to
>>>>>>>> disk. You would only need to change the doc of the config, if you
>>>> agree
>>>>>>>> with me.
>>>>>>>>
>>>>>>>> 8.
>>>>>>>> Section "Transaction Management" about the wrappers is rather a
>>>>>>>> implementation detail that should not be in the KIP.
>>>>>>>>
>>>>>>>> 9.
>>>>>>>> Could you add a section that describes how failover will work with
>> the
>>>>>>>> transactional state stores? I think section "Error handling" is
>>>> already
>>>>>>>> a good start.
>>>>>>>>
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Bruno
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 15.05.23 11:04, Nick Telford wrote:
>>>>>>>>> Hi everyone,
>>>>>>>>>
>>>>>>>>> Quick update: I've added a new section to the KIP: "Offsets for
>>>>>> Consumer
>>>>>>>>> Rebalances", that outlines my solution to the problem that
>>>>>>>>> StreamsPartitionAssignor needs to read StateStore offsets even if
>>>>>> they're
>>>>>>>>> not currently open.
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>> Nick
>>>>>>>>>
>>>>>>>>> On Wed, 3 May 2023 at 11:34, Nick Telford <ni...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Bruno,
>>>>>>>>>>
>>>>>>>>>> Thanks for reviewing my proposal.
>>>>>>>>>>
>>>>>>>>>> 1.
>>>>>>>>>> The main reason I added it was because it was easy to do. If we
>> see
>>>> no
>>>>>>>>>> value in it, I can remove it.
>>>>>>>>>>
>>>>>>>>>> 2.
>>>>>>>>>> Global StateStores can have multiple partitions in their input
>>>> topics
>>>>>>>>>> (which function as their changelogs), so they would have more than
>>>> one
>>>>>>>>>> partition.
>>>>>>>>>>
>>>>>>>>>> 3.
>>>>>>>>>> That's a good point. At present, the only method it adds is
>>>>>>>>>> isolationLevel(), which is likely not necessary outside of
>>>>>> StateStores.
>>>>>>>>>> It *does* provide slightly different guarantees in the
>> documentation
>>>>>> to
>>>>>>>>>> several of the methods (hence the overrides). I'm not sure if this
>>>> is
>>>>>>>>>> enough to warrant a new interface though.
>>>>>>>>>> I think the question that remains is whether this interface makes
>> it
>>>>>>>>>> easier to implement custom transactional StateStores than if we
>> were
>>>>>> to
>>>>>>>>>> remove it? Probably not.
>>>>>>>>>>
>>>>>>>>>> 4.
>>>>>>>>>> The main motivation for the Atomic Checkpointing is actually
>>>>>>>> performance.
>>>>>>>>>> My team has been testing out an implementation of this KIP without
>>>> it,
>>>>>>>> and
>>>>>>>>>> we had problems with RocksDB doing *much* more compaction, due to
>>>> the
>>>>>>>>>> significantly increased flush rate. It was enough of a problem
>> that
>>>>>> (for
>>>>>>>>>> the time being), we had to revert back to Kafka Streams proper.
>>>>>>>>>> I think the best way to solve this, as you say, is to keep the
>>>>>>>> .checkpoint
>>>>>>>>>> files *in addition* to the offsets being stored within the store
>>>>>> itself.
>>>>>>>>>> Essentially, when closing StateStores, we force a memtable flush,
>>>> then
>>>>>>>>>> call getCommittedOffsets and write those out to the .checkpoint
>>>> file.
>>>>>>>>>> That would ensure the metadata is available to the
>>>>>>>>>> StreamsPartitionAssignor for all closed stores.
>>>>>>>>>> If there's a crash (no clean close), then we won't be able to
>>>>>> guarantee
>>>>>>>>>> which offsets were flushed to disk by RocksDB, so we'd need to
>> open
>>>> (
>>>>>>>>>> init()), read offsets, and then close() those stores. But since
>> this
>>>>>> is
>>>>>>>>>> the exception, and will only occur once (provided it doesn't crash
>>>>>> every
>>>>>>>>>> time!), I think the performance impact here would be acceptable.
>>>>>>>>>>
>>>>>>>>>> Thanks for the feedback, please let me know if you have any more
>>>>>>>> comments
>>>>>>>>>> or questions!
>>>>>>>>>>
>>>>>>>>>> I'm currently working on rebasing against trunk. This involves
>>>> adding
>>>>>>>>>> support for transactionality to VersionedStateStores. I will
>>>> probably
>>>>>>>> need
>>>>>>>>>> to revise my implementation for transactional "segmented" stores,
>>>> both
>>>>>>>> to
>>>>>>>>>> accommodate VersionedStateStore, and to clean up some other stuff.
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>> Nick
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org>
>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>
>>>>>>>>>>> Thanks for the updates!
>>>>>>>>>>>
>>>>>>>>>>> I have a couple of questions/comments.
>>>>>>>>>>>
>>>>>>>>>>> 1.
>>>>>>>>>>> Why do you propose a configuration that involves max. bytes and
>>>> max.
>>>>>>>>>>> reords? I think we are mainly concerned about memory consumption
>>>>>>>> because
>>>>>>>>>>> we want to limit the off-heap memory used. I cannot think of a
>> case
>>>>>>>>>>> where one would want to set the max. number of records.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 2.
>>>>>>>>>>> Why does
>>>>>>>>>>>
>>>>>>>>>>>        default void commit(final Map<TopicPartition, Long>
>>>>>>>> changelogOffsets) {
>>>>>>>>>>>            flush();
>>>>>>>>>>>        }
>>>>>>>>>>>
>>>>>>>>>>> take a map of partitions to changelog offsets?
>>>>>>>>>>> The mapping between state stores to partitions is a 1:1
>>>> relationship.
>>>>>>>>>>> Passing in a single changelog offset should suffice.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 3.
>>>>>>>>>>> Why do we need the Transaction interface? It should be possible
>> to
>>>>>> hide
>>>>>>>>>>> beginning and committing a transactions withing the state store
>>>>>>>>>>> implementation, so that from outside the state store, it does not
>>>>>>>> matter
>>>>>>>>>>> whether the state store is transactional or not. What would be
>> the
>>>>>>>>>>> advantage of using the Transaction interface?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 4.
>>>>>>>>>>> Regarding checkpointing offsets, I think we should keep the
>>>>>> checkpoint
>>>>>>>>>>> file in any case for the reason you mentioned about rebalancing.
>>>> Even
>>>>>>>> if
>>>>>>>>>>> that would not be an issue, I would propose to move the change to
>>>>>>>> offset
>>>>>>>>>>> management to a new KIP and to not add more complexity than
>> needed
>>>> to
>>>>>>>>>>> this one. I would not be too concerned about the consistency
>>>>>> violation
>>>>>>>>>>> you mention. As far as I understand, with transactional state
>>>> stores
>>>>>>>>>>> Streams would write the checkpoint file during every commit even
>>>>>> under
>>>>>>>>>>> EOS. In the failure case you describe, Streams would restore the
>>>>>> state
>>>>>>>>>>> stores from the offsets found in the checkpoint file written
>> during
>>>>>> the
>>>>>>>>>>> penultimate commit instead of during the last commit. Basically,
>>>>>>>> Streams
>>>>>>>>>>> would overwrite the records written to the state store between
>> the
>>>>>> last
>>>>>>>>>>> two commits with the same records read from the changelogs.
>> While I
>>>>>>>>>>> understand that this is wasteful, it is -- at the same time --
>>>>>>>>>>> acceptable and most importantly it does not break EOS.
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> Bruno
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On 27.04.23 12:34, Nick Telford wrote:
>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>
>>>>>>>>>>>> I find myself (again) considering removing the offset management
>>>>>> from
>>>>>>>>>>>> StateStores, and keeping the old checkpoint file system. The
>>>> reason
>>>>>> is
>>>>>>>>>>> that
>>>>>>>>>>>> the StreamPartitionAssignor directly reads checkpoint files in
>>>> order
>>>>>>>> to
>>>>>>>>>>>> determine which instance has the most up-to-date copy of the
>> local
>>>>>>>>>>> state.
>>>>>>>>>>>> If we move offsets into the StateStore itself, then we will need
>>>> to
>>>>>>>>>>> open,
>>>>>>>>>>>> initialize, read offsets and then close each StateStore (that is
>>>> not
>>>>>>>>>>>> already assigned and open) for which we have *any* local state,
>> on
>>>>>>>> every
>>>>>>>>>>>> rebalance.
>>>>>>>>>>>>
>>>>>>>>>>>> Generally, I don't think there are many "orphan" stores like
>> this
>>>>>>>>>>> sitting
>>>>>>>>>>>> around on most instances, but even a few would introduce
>>>> additional
>>>>>>>>>>> latency
>>>>>>>>>>>> to an already somewhat lengthy rebalance procedure.
>>>>>>>>>>>>
>>>>>>>>>>>> I'm leaning towards Colt's (Slack) suggestion of just keeping
>>>> things
>>>>>>>> in
>>>>>>>>>>> the
>>>>>>>>>>>> checkpoint file(s) for now, and not worrying about the race. The
>>>>>>>>>>> downside
>>>>>>>>>>>> is that we wouldn't be able to remove the explicit RocksDB flush
>>>>>>>>>>> on-commit,
>>>>>>>>>>>> which likely hurts performance.
>>>>>>>>>>>>
>>>>>>>>>>>> If anyone has any thoughts or ideas on this subject, I would
>>>>>>>> appreciate
>>>>>>>>>>> it!
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>> Nick
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, 19 Apr 2023 at 15:05, Nick Telford <
>>>> nick.telford@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>
>>>>>>>>>>>>> The issue is that if there's a crash between 2 and 3, then you
>>>>>> still
>>>>>>>>>>> end
>>>>>>>>>>>>> up with inconsistent data in RocksDB. The only way to guarantee
>>>>>> that
>>>>>>>>>>> your
>>>>>>>>>>>>> checkpoint offsets and locally stored data are consistent with
>>>> each
>>>>>>>>>>> other
>>>>>>>>>>>>> are to atomically commit them, which can be achieved by having
>>>> the
>>>>>>>>>>> offsets
>>>>>>>>>>>>> stored in RocksDB.
>>>>>>>>>>>>>
>>>>>>>>>>>>> The offsets column family is likely to be extremely small (one
>>>>>>>>>>>>> per-changelog partition + one per Topology input partition for
>>>>>>>> regular
>>>>>>>>>>>>> stores, one per input partition for global stores). So the
>>>> overhead
>>>>>>>>>>> will be
>>>>>>>>>>>>> minimal.
>>>>>>>>>>>>>
>>>>>>>>>>>>> A major benefit of doing this is that we can remove the
>> explicit
>>>>>>>> calls
>>>>>>>>>>> to
>>>>>>>>>>>>> db.flush(), which forcibly flushes memtables to disk on-commit.
>>>> It
>>>>>>>>>>> turns
>>>>>>>>>>>>> out, RocksDB memtable flushes are largely dictated by Kafka
>>>> Streams
>>>>>>>>>>>>> commits, *not* RocksDB configuration, which could be a major
>>>> source
>>>>>>>> of
>>>>>>>>>>>>> confusion. Atomic checkpointing makes it safe to remove these
>>>>>>>> explicit
>>>>>>>>>>>>> flushes, because it no longer matters exactly when RocksDB
>>>> flushes
>>>>>>>>>>> data to
>>>>>>>>>>>>> disk; since the data and corresponding checkpoint offsets will
>>>>>> always
>>>>>>>>>>> be
>>>>>>>>>>>>> flushed together, the local store is always in a consistent
>>>> state,
>>>>>>>> and
>>>>>>>>>>>>> on-restart, it can always safely resume restoration from the
>>>>>> on-disk
>>>>>>>>>>>>> offsets, restoring the small amount of data that hadn't been
>>>>>> flushed
>>>>>>>>>>> when
>>>>>>>>>>>>> the app exited/crashed.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <
>> colt@littlehorse.io>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for your reply. Ack to A) and B).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> For item C), I see what you're referring to. Your proposed
>>>>>> solution
>>>>>>>>>>> will
>>>>>>>>>>>>>> work, so no need to change it. What I was suggesting was that
>> it
>>>>>>>>>>> might be
>>>>>>>>>>>>>> possible to achieve this with only one column family. So long
>>>> as:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>          - No uncommitted records (i.e. not committed to the
>>>>>> changelog)
>>>>>>>> are
>>>>>>>>>>>>>>          *committed* to the state store, AND
>>>>>>>>>>>>>>          - The Checkpoint offset (which refers to the changelog
>>>>>> topic)
>>>>>>>> is
>>>>>>>>>>> less
>>>>>>>>>>>>>>          than or equal to the last written changelog offset in
>>>>>> rocksdb
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I don't see the need to do the full restoration from scratch.
>> My
>>>>>>>>>>>>>> understanding was that prior to 844/892, full restorations
>> were
>>>>>>>>>>> required
>>>>>>>>>>>>>> because there could be uncommitted records written to RocksDB;
>>>>>>>>>>> however,
>>>>>>>>>>>>>> given your use of RocksDB transactions, that can be avoided
>> with
>>>>>> the
>>>>>>>>>>>>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
>>>>>>>>>>> transaction, 3)
>>>>>>>>>>>>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <
>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> A. I've done my best to de-couple the StateStore stuff from
>> the
>>>>>>>> rest
>>>>>>>>>>> of
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> Streams engine. The fact that there will be only one ongoing
>>>>>>>> (write)
>>>>>>>>>>>>>>> transaction at a time is not guaranteed by any API, and is
>>>> just a
>>>>>>>>>>>>>>> consequence of the way Streams operates. To that end, I tried
>>>> to
>>>>>>>>>>> ensure
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> documentation and guarantees provided by the new APIs are
>>>>>>>>>>> independent of
>>>>>>>>>>>>>>> this incidental behaviour. In practice, you're right, this
>>>>>>>>>>> essentially
>>>>>>>>>>>>>>> refers to "interactive queries", which are technically "read
>>>>>>>>>>>>>> transactions",
>>>>>>>>>>>>>>> even if they don't actually use the transaction API to
>> isolate
>>>>>>>>>>>>>> themselves.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> B. Yes, although not ideal. This is for backwards
>>>> compatibility,
>>>>>>>>>>>>>> because:
>>>>>>>>>>>>>>>           1) Existing custom StateStore implementations will
>>>>>> implement
>>>>>>>>>>>>>> flush(),
>>>>>>>>>>>>>>> and not commit(), but the Streams engine now calls commit(),
>> so
>>>>>>>> those
>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>> need to be forwarded to flush() for these legacy stores.
>>>>>>>>>>>>>>>           2) Existing StateStore *users*, i.e. outside of the
>>>>>> Streams
>>>>>>>>>>> engine
>>>>>>>>>>>>>>> itself, may depend on explicitly calling flush(), so for
>> these
>>>>>>>> cases,
>>>>>>>>>>>>>>> flush() needs to be redirected to call commit().
>>>>>>>>>>>>>>> If anyone has a better way to guarantee compatibility without
>>>>>>>>>>>>>> introducing
>>>>>>>>>>>>>>> this potential recursion loop, I'm open to changes!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> C. This is described in the "Atomic Checkpointing" section.
>>>>>> Offsets
>>>>>>>>>>> are
>>>>>>>>>>>>>>> stored in a separate RocksDB column family, which is
>> guaranteed
>>>>>> to
>>>>>>>> be
>>>>>>>>>>>>>>> atomically flushed to disk with all other column families.
>> The
>>>>>>>> issue
>>>>>>>>>>> of
>>>>>>>>>>>>>>> checkpoints being written to disk after commit causing
>>>>>>>> inconsistency
>>>>>>>>>>> if
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>> crashes in between is the reason why, under EOS, checkpoint
>>>> files
>>>>>>>> are
>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>> written on clean shutdown. This is one of the major causes of
>>>>>> "full
>>>>>>>>>>>>>>> restorations", so moving the offsets into a place where they
>>>> can
>>>>>> be
>>>>>>>>>>>>>>> guaranteed to be atomically written with the data they
>>>> checkpoint
>>>>>>>>>>>>>> allows us
>>>>>>>>>>>>>>> to write the checkpoint offsets *on every commit*, not just
>> on
>>>>>>>> clean
>>>>>>>>>>>>>>> shutdown.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <
>>>> colt@littlehorse.io>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thank you for continuing this work. I have a few minor
>>>>>> clarifying
>>>>>>>>>>>>>>>> questions.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> A) "Records written to any transaction are visible to all
>>>> other
>>>>>>>>>>>>>>>> transactions immediately." I am confused here—I thought
>> there
>>>>>>>> could
>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>> one transaction going on at a time for a given state store
>>>> given
>>>>>>>> the
>>>>>>>>>>>>>>>> threading model for processing records on a Task. Do you
>> mean
>>>>>>>>>>>>>> Interactive
>>>>>>>>>>>>>>>> Queries by "other transactions"? (If so, then everything
>> makes
>>>>>>>>>>> sense—I
>>>>>>>>>>>>>>>> thought that since IQ were read-only then they didn't count
>> as
>>>>>>>>>>>>>>>> transactions).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> B) Is it intentional that the default implementations of the
>>>>>>>> flush()
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> commit() methods in the StateStore class refer to each other
>>>> in
>>>>>>>> some
>>>>>>>>>>>>>> sort
>>>>>>>>>>>>>>>> of unbounded recursion?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> C) How will the getCommittedOffset() method work? At first I
>>>>>>>> thought
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> way to do it would be using a special key in the RocksDB
>> store
>>>>>> to
>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> offset, and committing that with the transaction. But upon
>>>>>> second
>>>>>>>>>>>>>>> thought,
>>>>>>>>>>>>>>>> since restoration from the changelog is an idempotent
>>>>>> procedure, I
>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> would be fine to 1) commit the RocksDB transaction and then
>> 2)
>>>>>>>> write
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> offset to disk in a checkpoint file. If there is a crash
>>>> between
>>>>>>>> 1)
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> 2),
>>>>>>>>>>>>>>>> I think the only downside is 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 <
>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I've updated the KIP to reflect the latest version of the
>>>>>> design:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> There are several changes in there that reflect feedback
>> from
>>>>>>>> this
>>>>>>>>>>>>>>>> thread,
>>>>>>>>>>>>>>>>> and there's a new section and a bunch of interface changes
>>>>>>>> relating
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> Atomic Checkpointing, which is the final piece of the
>> puzzle
>>>> to
>>>>>>>>>>>>>> making
>>>>>>>>>>>>>>>>> everything robust.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Let me know what you think!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <
>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi Lucas,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks for looking over my KIP.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> A) The bound is per-instance, not per-Task. This was a
>> typo
>>>> in
>>>>>>>> the
>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>> that I've now corrected. It was originally per-Task, but I
>>>>>>>>>>>>>> changed it
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> per-instance for exactly the reason you highlighted.
>>>>>>>>>>>>>>>>>> B) It's worth noting that transactionality is only enabled
>>>>>> under
>>>>>>>>>>>>>> EOS,
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> in the default mode of operation (ALOS), there should be
>> no
>>>>>>>>>>>>>> change in
>>>>>>>>>>>>>>>>>> behavior at all. I think, under EOS, we can mitigate the
>>>>>> impact
>>>>>>>> on
>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>> sufficiently low default values for the memory bound
>>>>>>>>>>>>>> configuration. I
>>>>>>>>>>>>>>>>>> understand your hesitation to include a significant change
>>>> of
>>>>>>>>>>>>>>>> behaviour,
>>>>>>>>>>>>>>>>>> especially in a minor release, but I suspect that most
>> users
>>>>>>>> will
>>>>>>>>>>>>>>>> prefer
>>>>>>>>>>>>>>>>>> the memory impact (under EOS) to the existing behaviour of
>>>>>>>>>>>>>> frequent
>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>> restorations! If this is a problem, the changes can wait
>>>> until
>>>>>>>> the
>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>> major release. I'll be running a patched version of
>> streams
>>>> in
>>>>>>>>>>>>>>>> production
>>>>>>>>>>>>>>>>>> with these changes as soon as they're ready, so it won't
>>>>>> disrupt
>>>>>>>>>>>>>> me
>>>>>>>>>>>>>>> :-D
>>>>>>>>>>>>>>>>>> C) The main purpose of this sentence was just to note that
>>>>>> some
>>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>> will need to be made to the way Segments are handled in
>>>> order
>>>>>> to
>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>> they also benefit from transactions. At the time I wrote
>>>> it, I
>>>>>>>>>>>>>> hadn't
>>>>>>>>>>>>>>>>>> figured out the specific changes necessary, so it was
>>>>>>>> deliberately
>>>>>>>>>>>>>>>> vague.
>>>>>>>>>>>>>>>>>> This is the one outstanding problem I'm currently working
>>>> on,
>>>>>>>> and
>>>>>>>>>>>>>>> I'll
>>>>>>>>>>>>>>>>>> update this section with more detail once I have figured
>> out
>>>>>> the
>>>>>>>>>>>>>>> exact
>>>>>>>>>>>>>>>>>> changes required.
>>>>>>>>>>>>>>>>>> D) newTransaction() provides the necessary isolation
>>>>>> guarantees.
>>>>>>>>>>>>>>> While
>>>>>>>>>>>>>>>>>> the RocksDB implementation of transactions doesn't
>>>> technically
>>>>>>>>>>>>>> *need*
>>>>>>>>>>>>>>>>>> read-only users to call newTransaction(), other
>>>>>> implementations
>>>>>>>>>>>>>>> (e.g. a
>>>>>>>>>>>>>>>>>> hypothetical PostgresStore) may require it. Calling
>>>>>>>>>>>>>> newTransaction()
>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>> no transaction is necessary is essentially free, as it
>> will
>>>>>> just
>>>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think
>> it
>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> fairly obvious where the performance problems stem from:
>>>>>> writes
>>>>>>>>>>>>>> under
>>>>>>>>>>>>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with
>>>> the
>>>>>>>>>>>>>>>>>> tombstone/record flag, 1 to decode it from the
>>>>>> tombstone/record
>>>>>>>>>>>>>> flag,
>>>>>>>>>>>>>>>>> and 1
>>>>>>>>>>>>>>>>>> to copy the record from the "temporary" store to the
>> "main"
>>>>>>>> store,
>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> transaction commits. The different approach taken by
>> KIP-869
>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>> perform
>>>>>>>>>>>>>>>>>> much better, as it avoids all these copies, and may
>> actually
>>>>>>>>>>>>>> perform
>>>>>>>>>>>>>>>>>> slightly better than trunk, due to batched writes in
>> RocksDB
>>>>>>>>>>>>>>> performing
>>>>>>>>>>>>>>>>>> better than non-batched writes.[1]
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 1:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>
>>>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
>>>>>>>>>>>>>> lbrutschy@confluent.io
>>>>>>>>>>>>>>>>> .invalid>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I'm just starting to read up on the whole discussion
>> about
>>>>>>>>>>>>>> KIP-892
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
>>>>>>>>>>>>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do
>> have
>>>>>> some
>>>>>>>>>>>>>>>>>>> questions about the latest draft.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>        A) If I understand correctly, you propose to put a
>>>> bound
>>>>>> on
>>>>>>>> the
>>>>>>>>>>>>>>>>>>> (native) memory consumed by each task. However, I wonder
>> if
>>>>>>>> this
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> sufficient if we have temporary imbalances in the
>> cluster.
>>>>>> For
>>>>>>>>>>>>>>>>>>> example, depending on the timing of rebalances during a
>>>>>> cluster
>>>>>>>>>>>>>>>>>>> restart, it could happen that a single streams node is
>>>>>>>> assigned a
>>>>>>>>>>>>>>> lot
>>>>>>>>>>>>>>>>>>> more tasks than expected. With your proposed change, this
>>>>>> would
>>>>>>>>>>>>>> mean
>>>>>>>>>>>>>>>>>>> that the memory required by this one node could be a
>>>> multiple
>>>>>>>> of
>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>> is required during normal operation. I wonder if it
>>>> wouldn't
>>>>>> be
>>>>>>>>>>>>>>> safer
>>>>>>>>>>>>>>>>>>> to put a global bound on the memory use, across all
>> tasks.
>>>>>>>>>>>>>>>>>>>        B) Generally, the memory concerns still give me the
>>>>>> feeling
>>>>>>>>>>> that
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> should not be enabled by default for all users in a minor
>>>>>>>>>>>>>> release.
>>>>>>>>>>>>>>>>>>>        C) In section "Transaction Management": the
>> sentence
>>>> "A
>>>>>>>> similar
>>>>>>>>>>>>>>>>>>> analogue will be created to automatically manage
>> `Segment`
>>>>>>>>>>>>>>>>>>> transactions.". Maybe this is just me lacking some
>>>>>> background,
>>>>>>>>>>>>>> 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 you to find out why the KIP-844 PoC is 20x
>>>>>> slower
>>>>>>>>>>>>>> (e.g.
>>>>>>>>>>>>>>>>>>> by providing a flame graph), that would be quite
>>>> interesting.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>>>> Lucas
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I've updated the KIP with a more detailed design, which
>>>>>>>>>>>>>> reflects
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> implementation I've been working on:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> This new design should address the outstanding points
>>>>>> already
>>>>>>>>>>>>>> made
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Please let me know if there are areas that are unclear
>> or
>>>>>> need
>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>>> clarification.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I have a (nearly) working implementation. I'm confident
>>>> that
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> remaining
>>>>>>>>>>>>>>>>>>>> work (making Segments behave) will not impact the
>>>> documented
>>>>>>>>>>>>>>> design.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <
>>>>>>>> colt@littlehorse.io
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thank you for the reply; that makes sense. I was hoping
>>>>>> that,
>>>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>>>> reading
>>>>>>>>>>>>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
>>>>>>>>>>>>>> documented
>>>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>>>> maybe
>>>>>>>>>>>>>>>>>>>>> you *wouldn't* have to wait for the next major release
>> to
>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> change;
>>>>>>>>>>>>>>>>>>>>> but given 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 <
>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended
>>>> to
>>>>>>>>>>>>>> try
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> keep the
>>>>>>>>>>>>>>>>>>>>>> behaviour unchanged as much as possible, otherwise
>> we'd
>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> wait for
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>> major version release to land these changes.
>>>>>>>>>>>>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
>>>>>>>>>>>>>> guarantee,
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> typically longer commit intervals would be problematic
>>>>>> when
>>>>>>>>>>>>>>>>> reading
>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>> "committed" records.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I've been away for 5 days recovering from minor
>> surgery,
>>>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>>>> spent a
>>>>>>>>>>>>>>>>>>>>>> considerable amount of that time working through ideas
>>>> for
>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>>>> solutions in my head. I think your suggestion of
>> keeping
>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>>>> as-is, but
>>>>>>>>>>>>>>>>>>>>>> buffering writes for EOS is the right path forwards,
>>>>>>>>>>>>>> although
>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>>>> solution that both expands on this, and provides for
>>>> some
>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>> formal
>>>>>>>>>>>>>>>>>>>>>> guarantees.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Essentially, adding support to KeyValueStores for
>>>>>>>>>>>>>>>> "Transactions",
>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>> clearly defined IsolationLevels. Using "Read
>> Committed"
>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>> under
>>>>>>>>>>>>>>>>>>> EOS,
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> "Read Uncommitted" under ALOS.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> The nice thing about this approach is that it gives us
>>>>>> much
>>>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>>> clearly
>>>>>>>>>>>>>>>>>>>>>> defined isolation behaviour that can be properly
>>>>>>>>>>>>>> documented to
>>>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>>>>>>> know what to expect.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I'm still working out the kinks in the design, and
>> will
>>>>>>>>>>>>>> update
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>>>> I have something. The main struggle is trying to
>>>> implement
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>>>> making any major changes to the existing interfaces or
>>>>>>>>>>>>>>> breaking
>>>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>> implementations, because currently everything expects
>> to
>>>>>>>>>>>>>>> operate
>>>>>>>>>>>>>>>>>>> directly
>>>>>>>>>>>>>>>>>>>>>> on a StateStore, and not a Transaction of that store.
>> I
>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>> close, although sadly I won't be able to progress much
>>>>>>>>>>>>>> until
>>>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>>>> week
>>>>>>>>>>>>>>>>>>>>> due
>>>>>>>>>>>>>>>>>>>>>> to some work commitments.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
>>>>>>>>>>>>>>> colt@littlehorse.io>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thank you for the explanation, and also for the
>> updated
>>>>>>>>>>>>>>> KIP. I
>>>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>>>>>>>> eager for this improvement to be released as it would
>>>>>>>>>>>>>>> greatly
>>>>>>>>>>>>>>>>>>> reduce
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> operational difficulties of EOS streams apps.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Two questions:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 10)
>>>>>>>>>>>>>>>>>>>>>>>> When reading records, we will use the
>>>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
>>>>>>>>>>>>>>>>>>>>>>>        and WriteBatchWithIndex#newIteratorWithBase
>>>>>> utilities in
>>>>>>>>>>>>>>>> order
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>>>>> that uncommitted writes are available to query.
>>>>>>>>>>>>>>>>>>>>>>> Why do extra work to enable the reading of
>> uncommitted
>>>>>>>>>>>>>>> writes
>>>>>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>>>>>> IQ?
>>>>>>>>>>>>>>>>>>>>>>> Code complexity aside, reading uncommitted writes is,
>>>> in
>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>> opinion, a
>>>>>>>>>>>>>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have
>> the
>>>>>>>>>>>>>>>>> guarantee
>>>>>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>>>>>>>> with EOS, IQ only reads committed records. In order
>> to
>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>> dirty
>>>>>>>>>>>>>>>>>>>>> reads,
>>>>>>>>>>>>>>>>>>>>>>> one currently must query a standby replica (but this
>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>>>> fully
>>>>>>>>>>>>>>>>>>>>>>> guarantee monotonic reads).
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 20) Is it also necessary to enable this optimization
>> on
>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>>>> stores?
>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need
>> to
>>>>>>>>>>>>>>> restore
>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
>>>>>>>>>>>>>>> acceptable
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> accept
>>>>>>>>>>>>>>>>>>>>>>> that there may have been uncommitted writes on disk.
>>>> On a
>>>>>>>>>>>>>>> side
>>>>>>>>>>>>>>>>>>> note, if
>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>> enable this type of store on ALOS processors, the
>>>>>>>>>>>>>> community
>>>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
>>>>>>>>>>>>>>> longer
>>>>>>>>>>>>>>>>>>> include
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> StateStore management of checkpointing. This can be
>>>>>>>>>>>>>> added
>>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>>>> to further optimize the consistency and performance
>> of
>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>> stores.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I've also added a section discussing some of the
>>>>>>>>>>>>>> concerns
>>>>>>>>>>>>>>>>> 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 2022 at 16:36, Nick Telford <
>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I didn't do any profiling, but the 844
>>>>>>>>>>>>>> implementation:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>          - Writes uncommitted records to a temporary
>>>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>>>>>> instance
>>>>>>>>>>>>>>>>>>>>>>>>>             - Since tombstones need to be flagged,
>> all
>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>> values are
>>>>>>>>>>>>>>>>>>>>>>>>>             prefixed with a value/tombstone marker.
>>>> This
>>>>>>>>>>>>>>>>>>> necessitates a
>>>>>>>>>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>>>>>>>> copy.
>>>>>>>>>>>>>>>>>>>>>>>>>          - On-commit, iterates all records in this
>>>>>>>>>>>>>> temporary
>>>>>>>>>>>>>>>>>>> instance and
>>>>>>>>>>>>>>>>>>>>>>>>>          writes them to the main RocksDB store.
>>>>>>>>>>>>>>>>>>>>>>>>>          - While iterating, the value/tombston
> 

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

Posted by Nick Telford <ni...@gmail.com>.
Potentially we could just go the memorable with Rocks WriteBatches route if
the cache is disabled?

On Tue, 20 Jun 2023, 22:00 John Roesler, <jo...@vvcephei.org> wrote:

> Touché!
>
> Ok, I agree that figuring out the case of a disabled cache would be
> non-trivial. Ingesting single-record SST files will probably not be
> performant, but benchmarking may prove different. Or maybe we can have
> some reserved cache space on top of the user-configured cache, which we
> would have reclaimed from the memtable space. Or some other, more
> creative solution.
>
> Thanks,
> -John
>
> On 6/20/23 15:30, Nick Telford wrote:
> >> Note that users can disable the cache, which would still be
> > ok, I think. We wouldn't ingest the SST files on every record, but just
> > append to them and only ingest them on commit, when we're already
> > waiting for acks and a RocksDB commit.
> >
> > In this case, how would uncommitted records be read by joins?
> >
> > On Tue, 20 Jun 2023, 20:51 John Roesler, <vv...@apache.org> wrote:
> >
> >> Ah, sorry Nick,
> >>
> >> I just meant the regular heap based cache that we maintain in Streams. I
> >> see that it's not called "RecordCache" (my mistake).
> >>
> >> The actual cache is ThreadCache:
> >>
> >>
> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
> >>
> >> Here's the example of how we use the cache in KeyValueStore:
> >>
> >>
> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
> >>
> >> It's basically just an on-heap Map of records that have not yet been
> >> written to the changelog or flushed into the underlying store. It gets
> >> flushed when the total cache size exceeds `cache.max.bytes.buffering` or
> >> the `commit.interval.ms` elapses.
> >>
> >> Speaking of those configs, another benefit to this idea is that we would
> >> no longer need to trigger extra commits based on the size of the ongoing
> >> transaction. Instead, we'd just preserve the existing cache-flush
> >> behavior. Note that users can disable the cache, which would still be
> >> ok, I think. We wouldn't ingest the SST files on every record, but just
> >> append to them and only ingest them on commit, when we're already
> >> waiting for acks and a RocksDB commit.
> >>
> >> Thanks,
> >> -John
> >>
> >> On 6/20/23 14:09, Nick Telford wrote:
> >>> Hi John,
> >>>
> >>> By "RecordCache", do you mean the RocksDB "WriteBatch"? I can't find
> any
> >>> class called "RecordCache"...
> >>>
> >>> Cheers,
> >>>
> >>> Nick
> >>>
> >>> On Tue, 20 Jun 2023 at 19:42, John Roesler <vv...@apache.org>
> wrote:
> >>>
> >>>> Hi Nick,
> >>>>
> >>>> Thanks for picking this up again!
> >>>>
> >>>> I did have one new thought over the intervening months, which I'd like
> >>>> your take on.
> >>>>
> >>>> What if, instead of using the RocksDB atomic write primitive at all,
> we
> >>>> instead just:
> >>>> 1. disable memtables entirely
> >>>> 2. directly write the RecordCache into SST files when we flush
> >>>> 3. atomically ingest the SST file(s) into RocksDB when we get the ACK
> >>>> from the changelog (see
> >>>>
> >>>>
> >>
> https://github.com/EighteenZi/rocksdb_wiki/blob/master/Creating-and-Ingesting-SST-files.md
> >>>> and
> >>>>
> >>>>
> >>
> https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/IngestExternalFileOptions.java
> >>>> and
> >>>>
> >>>>
> >>
> https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L1413-L1429
> >>>> )
> >>>> 4. track the changelog offsets either in another CF or the same CF
> with
> >>>> a reserved key, either of which will make the changelog offset update
> >>>> atomic with the file ingestions
> >>>>
> >>>> I suspect this'll have a number of benefits:
> >>>> * writes to RocksDB will always be atomic
> >>>> * we don't fragment memory between the RecordCache and the memtables
> >>>> * RecordCache gives far higher performance than memtable for reads and
> >>>> writes
> >>>> * we don't need any new "transaction" concepts or memory bound configs
> >>>>
> >>>> What do you think?
> >>>>
> >>>> Thanks,
> >>>> -John
> >>>>
> >>>> On 6/20/23 10:51, Nick Telford wrote:
> >>>>> Hi Bruno,
> >>>>>
> >>>>> Thanks for reviewing the KIP. It's been a long road, I started
> working
> >> on
> >>>>> this more than a year ago, and most of the time in the last 6 months
> >> has
> >>>>> been spent on the "Atomic Checkpointing" stuff that's been benched,
> so
> >>>> some
> >>>>> of the reasoning behind some of my decisions have been lost, but I'll
> >> do
> >>>> my
> >>>>> best to reconstruct them.
> >>>>>
> >>>>> 1.
> >>>>> IIRC, this was the initial approach I tried. I don't remember the
> exact
> >>>>> reasons I changed it to use a separate "view" of the StateStore that
> >>>>> encapsulates the transaction, but I believe it had something to do
> with
> >>>>> concurrent access to the StateStore from Interactive Query threads.
> >> Reads
> >>>>> from interactive queries need to be isolated from the currently
> ongoing
> >>>>> transaction, both for consistency (so interactive queries don't
> observe
> >>>>> changes that are subsequently rolled-back), but also to prevent
> >> Iterators
> >>>>> opened by an interactive query from being closed and invalidated by
> the
> >>>>> StreamThread when it commits the transaction, which causes your
> >>>> interactive
> >>>>> queries to crash.
> >>>>>
> >>>>> Another reason I believe I implemented it this way was a separation
> of
> >>>>> concerns. Recall that newTransaction() originally created an object
> of
> >>>> type
> >>>>> Transaction, not StateStore. My intent was to improve the type-safety
> >> of
> >>>>> the API, in an effort to ensure Transactions weren't used
> incorrectly.
> >>>>> Unfortunately, this didn't pan out, but newTransaction() remained.
> >>>>>
> >>>>> Finally, this had the added benefit that implementations could easily
> >> add
> >>>>> support for transactions *without* re-writing their existing,
> >>>>> non-transactional implementation. I think this can be a benefit both
> >> for
> >>>>> implementers of custom StateStores, but also for anyone extending
> >>>>> RocksDbStore, as they can rely on the existing access methods working
> >> how
> >>>>> they expect them to.
> >>>>>
> >>>>> I'm not too happy with the way the current design has panned out, so
> >> I'm
> >>>>> open to ideas on how to improve it. Key to this is finding some way
> to
> >>>>> ensure that reads from Interactive Query threads are properly
> isolated
> >>>> from
> >>>>> the transaction, *without* the performance overhead of checking which
> >>>>> thread the method is being called from on every access.
> >>>>>
> >>>>> As for replacing flush() with commit() - I saw no reason to add this
> >>>>> complexity to the KIP, unless there was a need to add arguments to
> the
> >>>>> flush/commit method. This need arises with Atomic Checkpointing, but
> >> that
> >>>>> will be implemented separately, in a future KIP. Do you see a need
> for
> >>>> some
> >>>>> arguments to the flush/commit method that I've missed? Or were you
> >> simply
> >>>>> suggesting a rename?
> >>>>>
> >>>>> 2.
> >>>>> This is simply due to the practical reason that isolationLevel() is
> >>>> really
> >>>>> a proxy for checking if the app is under EOS. The application
> >>>> configuration
> >>>>> is not provided to the constructor of StateStores, but it *is*
> provided
> >>>> to
> >>>>> init(), via StateStoreContext. For this reason, it seemed somewhat
> >>>> natural
> >>>>> to add it to StateStoreContext. I think this makes sense, since the
> >>>>> IsolationLevel of all StateStores in an application *must* be the
> same,
> >>>> and
> >>>>> since those stores are all initialized with the same
> StateStoreContext,
> >>>> it
> >>>>> seems natural for that context to carry the desired IsolationLevel to
> >>>> use.
> >>>>>
> >>>>> 3.
> >>>>> Using IsolationLevel instead of just passing `boolean eosEnabled`,
> like
> >>>>> much of the internals was an attempt to logically de-couple the
> >>>> StateStore
> >>>>> API from the internals of Kafka Streams. Technically, StateStores
> don't
> >>>>> need to know/care what processing mode the KS app is using, all they
> >> need
> >>>>> to know is the isolation level expected of them.
> >>>>>
> >>>>> Having formal definitions for the expectations of the two required
> >>>>> IsolationLevels allow implementers to implement transactional stores
> >>>>> without having to dig through the internals of Kafka Streams and
> >>>> understand
> >>>>> exactly how they are used. The tight coupling between state stores
> and
> >>>>> internal behaviour has actually significantly hindered my progress on
> >>>> this
> >>>>> KIP, and encouraged me to avoid increasing this logical coupling as
> >> much
> >>>> as
> >>>>> possible.
> >>>>>
> >>>>> This also frees implementations to satisfy those requirements in any
> >> way
> >>>>> they choose. Transactions might not be the only/available approach to
> >> an
> >>>>> implementation, but they might have an alternative way to satisfy the
> >>>>> isolation requirements. I admit that this point is more about
> >> semantics,
> >>>>> but "transactional" would need to be formally defined in order for
> >>>>> implementers to provide a valid implementation, and these
> >> IsolationLevels
> >>>>> provide that formal definition.
> >>>>>
> >>>>> 4.
> >>>>> I can remove them. I added them only as I planned to include them in
> >> the
> >>>>> org.apache.kafka.streams.state package, as a recommended base
> >>>>> implementation for all StateStores, including those implemented by
> >>>> users. I
> >>>>> had assumed that anything in "public" packages, such as
> >>>>> org.apache.kafka.streams.state, should be included in a KIP. Is that
> >>>> wrong?
> >>>>>
> >>>>> 5.
> >>>>> RocksDB provides no way to measure the actual size of a
> >>>>> WriteBatch(WithIndex), so we're limited to tracking the sum total of
> >> the
> >>>>> size of keys + values that are written to the transaction. This
> >> obviously
> >>>>> under-estimates the actual memory usage, because WriteBatch no-doubt
> >>>>> includes some record overheads, and WriteBatchWithIndex has to
> maintain
> >>>> an
> >>>>> index.
> >>>>>
> >>>>> Ideally, we could trivially add a method upstream to
> >> WriteBatchInterface
> >>>>> that provides the exact size of the batch, but that would require an
> >>>>> upgrade of RocksDB, which won't happen soon. So for the time being,
> >> we're
> >>>>> stuck with an approximation, so I felt that the new method should
> >> reflect
> >>>>> that.
> >>>>>
> >>>>> Would you prefer the new method name ignores this constraint and that
> >> we
> >>>>> simply make the rocks measurement more accurate in the future?
> >>>>>
> >>>>> 6.
> >>>>> Done
> >>>>>
> >>>>> 7.
> >>>>> Very good point. The KIP already specifically calls out memory in the
> >>>>> documentation of the config: "Maximum number of memory bytes to be
> used
> >>>> to
> >>>>> buffer uncommitted state-store records." - did you have something
> else
> >> in
> >>>>> mind?
> >>>>>
> >>>>> Should we also make this clearer by renaming the config property
> >> itself?
> >>>>> Perhaps to something like statestore.transaction.buffer.max.bytes?
> >>>>>
> >>>>> 8.
> >>>>> OK, I can remove this. The intent here was to describe how Streams
> >> itself
> >>>>> will manage transaction roll-over etc. Presumably that means we also
> >>>> don't
> >>>>> need a description of how Streams will manage the commit of changelog
> >>>>> transactions, state store transactions and checkpointing?
> >>>>>
> >>>>> 9.
> >>>>> What do you mean by fail-over? Do you mean failing over an Active
> Task
> >> to
> >>>>> an instance already hosting a Standby Task?
> >>>>>
> >>>>> Thanks again and sorry for the essay of a response!
> >>>>>
> >>>>> Regards,
> >>>>> Nick
> >>>>>
> >>>>> On Tue, 20 Jun 2023 at 10:49, Bruno Cadonna <ca...@apache.org>
> >> wrote:
> >>>>>
> >>>>>> Hi Nick,
> >>>>>>
> >>>>>> Thanks for the updates!
> >>>>>>
> >>>>>> I really appreciate that you simplified the KIP by removing some
> >>>>>> aspects. As I have already told you, I think the removed aspects are
> >>>>>> also good ideas and we can discuss them on follow-up KIPs.
> >>>>>>
> >>>>>> Regarding the current KIP, I have the following feedback.
> >>>>>>
> >>>>>> 1.
> >>>>>> Is there a good reason to add method newTransaction() to the
> >> StateStore
> >>>>>> interface? As far as I understand, the idea is that users of a state
> >>>>>> store (transactional or not) call this method at start-up and after
> >> each
> >>>>>> commit. Since the call to newTransaction() is done in any case and I
> >>>>>> think it would simplify the caller code if we just start a new
> >>>>>> transaction after a commit in the implementation?
> >>>>>> As far as I understand, you plan to commit the transaction in the
> >>>>>> flush() method. I find the idea to replace flush() with commit()
> >>>>>> presented in KIP-844 an elegant solution.
> >>>>>>
> >>>>>> 2.
> >>>>>> Why is the method to query the isolation level added to the state
> >> store
> >>>>>> context?
> >>>>>>
> >>>>>> 3.
> >>>>>> Do we need all the isolation level definitions? I think it is good
> to
> >>>>>> know the guarantees of the transactionality of the state store.
> >>>>>> However, currently, Streams guarantees that there will only be one
> >>>>>> transaction that writes to the state store. Only the stream thread
> >> that
> >>>>>> executes the active task that owns the state store will write to the
> >>>>>> state store. I think it should be enough to know if the state store
> is
> >>>>>> transactional or not. So my proposal would be to just add a method
> on
> >>>>>> the state store interface the returns if a state store is
> >> transactional
> >>>>>> or not by returning a boolean or an enum.
> >>>>>>
> >>>>>> 4.
> >>>>>> I am wondering why AbstractTransaction and
> AbstractTransactionalStore
> >>>>>> are part of the KIP. They look like implementation details that
> should
> >>>>>> not be exposed in the public API.
> >>>>>>
> >>>>>> 5.
> >>>>>> Why does StateStore#approximateNumUncommittedBytes() return an
> >>>>>> approximate number of bytes?
> >>>>>>
> >>>>>> 6.
> >>>>>> RocksDB is just one implementation of the state stores in Streams.
> >>>>>> However, the issues regarding OOM errors might also apply to other
> >>>>>> custom implementations. So in the KIP I would extract that part from
> >>>>>> section "RocksDB Transaction". I would also move section "RocksDB
> >>>>>> Transaction" to the end of section "Proposed Changes" and handle it
> as
> >>>>>> an example implementation for a state store.
> >>>>>>
> >>>>>> 7.
> >>>>>> Should statestore.uncommitted.max.bytes only limit the uncommitted
> >> bytes
> >>>>>> or the uncommitted bytes that reside in memory? In future, other
> >>>>>> transactional state store implementations might implement a buffer
> for
> >>>>>> uncommitted records that are able to spill records on disk. I think
> >>>>>> statestore.uncommitted.max.bytes needs to limit the uncommitted
> bytes
> >>>>>> irrespective if they reside in memory or disk. Since Streams will
> use
> >>>>>> this config to decide if it needs to trigger a commit, state store
> >>>>>> implementations that can spill to disk will never be able to spill
> to
> >>>>>> disk. You would only need to change the doc of the config, if you
> >> agree
> >>>>>> with me.
> >>>>>>
> >>>>>> 8.
> >>>>>> Section "Transaction Management" about the wrappers is rather a
> >>>>>> implementation detail that should not be in the KIP.
> >>>>>>
> >>>>>> 9.
> >>>>>> Could you add a section that describes how failover will work with
> the
> >>>>>> transactional state stores? I think section "Error handling" is
> >> already
> >>>>>> a good start.
> >>>>>>
> >>>>>>
> >>>>>> Best,
> >>>>>> Bruno
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On 15.05.23 11:04, Nick Telford wrote:
> >>>>>>> Hi everyone,
> >>>>>>>
> >>>>>>> Quick update: I've added a new section to the KIP: "Offsets for
> >>>> Consumer
> >>>>>>> Rebalances", that outlines my solution to the problem that
> >>>>>>> StreamsPartitionAssignor needs to read StateStore offsets even if
> >>>> they're
> >>>>>>> not currently open.
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>> Nick
> >>>>>>>
> >>>>>>> On Wed, 3 May 2023 at 11:34, Nick Telford <ni...@gmail.com>
> >>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi Bruno,
> >>>>>>>>
> >>>>>>>> Thanks for reviewing my proposal.
> >>>>>>>>
> >>>>>>>> 1.
> >>>>>>>> The main reason I added it was because it was easy to do. If we
> see
> >> no
> >>>>>>>> value in it, I can remove it.
> >>>>>>>>
> >>>>>>>> 2.
> >>>>>>>> Global StateStores can have multiple partitions in their input
> >> topics
> >>>>>>>> (which function as their changelogs), so they would have more than
> >> one
> >>>>>>>> partition.
> >>>>>>>>
> >>>>>>>> 3.
> >>>>>>>> That's a good point. At present, the only method it adds is
> >>>>>>>> isolationLevel(), which is likely not necessary outside of
> >>>> StateStores.
> >>>>>>>> It *does* provide slightly different guarantees in the
> documentation
> >>>> to
> >>>>>>>> several of the methods (hence the overrides). I'm not sure if this
> >> is
> >>>>>>>> enough to warrant a new interface though.
> >>>>>>>> I think the question that remains is whether this interface makes
> it
> >>>>>>>> easier to implement custom transactional StateStores than if we
> were
> >>>> to
> >>>>>>>> remove it? Probably not.
> >>>>>>>>
> >>>>>>>> 4.
> >>>>>>>> The main motivation for the Atomic Checkpointing is actually
> >>>>>> performance.
> >>>>>>>> My team has been testing out an implementation of this KIP without
> >> it,
> >>>>>> and
> >>>>>>>> we had problems with RocksDB doing *much* more compaction, due to
> >> the
> >>>>>>>> significantly increased flush rate. It was enough of a problem
> that
> >>>> (for
> >>>>>>>> the time being), we had to revert back to Kafka Streams proper.
> >>>>>>>> I think the best way to solve this, as you say, is to keep the
> >>>>>> .checkpoint
> >>>>>>>> files *in addition* to the offsets being stored within the store
> >>>> itself.
> >>>>>>>> Essentially, when closing StateStores, we force a memtable flush,
> >> then
> >>>>>>>> call getCommittedOffsets and write those out to the .checkpoint
> >> file.
> >>>>>>>> That would ensure the metadata is available to the
> >>>>>>>> StreamsPartitionAssignor for all closed stores.
> >>>>>>>> If there's a crash (no clean close), then we won't be able to
> >>>> guarantee
> >>>>>>>> which offsets were flushed to disk by RocksDB, so we'd need to
> open
> >> (
> >>>>>>>> init()), read offsets, and then close() those stores. But since
> this
> >>>> is
> >>>>>>>> the exception, and will only occur once (provided it doesn't crash
> >>>> every
> >>>>>>>> time!), I think the performance impact here would be acceptable.
> >>>>>>>>
> >>>>>>>> Thanks for the feedback, please let me know if you have any more
> >>>>>> comments
> >>>>>>>> or questions!
> >>>>>>>>
> >>>>>>>> I'm currently working on rebasing against trunk. This involves
> >> adding
> >>>>>>>> support for transactionality to VersionedStateStores. I will
> >> probably
> >>>>>> need
> >>>>>>>> to revise my implementation for transactional "segmented" stores,
> >> both
> >>>>>> to
> >>>>>>>> accommodate VersionedStateStore, and to clean up some other stuff.
> >>>>>>>>
> >>>>>>>> Regards,
> >>>>>>>> Nick
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org>
> >>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi Nick,
> >>>>>>>>>
> >>>>>>>>> Thanks for the updates!
> >>>>>>>>>
> >>>>>>>>> I have a couple of questions/comments.
> >>>>>>>>>
> >>>>>>>>> 1.
> >>>>>>>>> Why do you propose a configuration that involves max. bytes and
> >> max.
> >>>>>>>>> reords? I think we are mainly concerned about memory consumption
> >>>>>> because
> >>>>>>>>> we want to limit the off-heap memory used. I cannot think of a
> case
> >>>>>>>>> where one would want to set the max. number of records.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 2.
> >>>>>>>>> Why does
> >>>>>>>>>
> >>>>>>>>>       default void commit(final Map<TopicPartition, Long>
> >>>>>> changelogOffsets) {
> >>>>>>>>>           flush();
> >>>>>>>>>       }
> >>>>>>>>>
> >>>>>>>>> take a map of partitions to changelog offsets?
> >>>>>>>>> The mapping between state stores to partitions is a 1:1
> >> relationship.
> >>>>>>>>> Passing in a single changelog offset should suffice.
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 3.
> >>>>>>>>> Why do we need the Transaction interface? It should be possible
> to
> >>>> hide
> >>>>>>>>> beginning and committing a transactions withing the state store
> >>>>>>>>> implementation, so that from outside the state store, it does not
> >>>>>> matter
> >>>>>>>>> whether the state store is transactional or not. What would be
> the
> >>>>>>>>> advantage of using the Transaction interface?
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> 4.
> >>>>>>>>> Regarding checkpointing offsets, I think we should keep the
> >>>> checkpoint
> >>>>>>>>> file in any case for the reason you mentioned about rebalancing.
> >> Even
> >>>>>> if
> >>>>>>>>> that would not be an issue, I would propose to move the change to
> >>>>>> offset
> >>>>>>>>> management to a new KIP and to not add more complexity than
> needed
> >> to
> >>>>>>>>> this one. I would not be too concerned about the consistency
> >>>> violation
> >>>>>>>>> you mention. As far as I understand, with transactional state
> >> stores
> >>>>>>>>> Streams would write the checkpoint file during every commit even
> >>>> under
> >>>>>>>>> EOS. In the failure case you describe, Streams would restore the
> >>>> state
> >>>>>>>>> stores from the offsets found in the checkpoint file written
> during
> >>>> the
> >>>>>>>>> penultimate commit instead of during the last commit. Basically,
> >>>>>> Streams
> >>>>>>>>> would overwrite the records written to the state store between
> the
> >>>> last
> >>>>>>>>> two commits with the same records read from the changelogs.
> While I
> >>>>>>>>> understand that this is wasteful, it is -- at the same time --
> >>>>>>>>> acceptable and most importantly it does not break EOS.
> >>>>>>>>>
> >>>>>>>>> Best,
> >>>>>>>>> Bruno
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On 27.04.23 12:34, Nick Telford wrote:
> >>>>>>>>>> Hi everyone,
> >>>>>>>>>>
> >>>>>>>>>> I find myself (again) considering removing the offset management
> >>>> from
> >>>>>>>>>> StateStores, and keeping the old checkpoint file system. The
> >> reason
> >>>> is
> >>>>>>>>> that
> >>>>>>>>>> the StreamPartitionAssignor directly reads checkpoint files in
> >> order
> >>>>>> to
> >>>>>>>>>> determine which instance has the most up-to-date copy of the
> local
> >>>>>>>>> state.
> >>>>>>>>>> If we move offsets into the StateStore itself, then we will need
> >> to
> >>>>>>>>> open,
> >>>>>>>>>> initialize, read offsets and then close each StateStore (that is
> >> not
> >>>>>>>>>> already assigned and open) for which we have *any* local state,
> on
> >>>>>> every
> >>>>>>>>>> rebalance.
> >>>>>>>>>>
> >>>>>>>>>> Generally, I don't think there are many "orphan" stores like
> this
> >>>>>>>>> sitting
> >>>>>>>>>> around on most instances, but even a few would introduce
> >> additional
> >>>>>>>>> latency
> >>>>>>>>>> to an already somewhat lengthy rebalance procedure.
> >>>>>>>>>>
> >>>>>>>>>> I'm leaning towards Colt's (Slack) suggestion of just keeping
> >> things
> >>>>>> in
> >>>>>>>>> the
> >>>>>>>>>> checkpoint file(s) for now, and not worrying about the race. The
> >>>>>>>>> downside
> >>>>>>>>>> is that we wouldn't be able to remove the explicit RocksDB flush
> >>>>>>>>> on-commit,
> >>>>>>>>>> which likely hurts performance.
> >>>>>>>>>>
> >>>>>>>>>> If anyone has any thoughts or ideas on this subject, I would
> >>>>>> appreciate
> >>>>>>>>> it!
> >>>>>>>>>>
> >>>>>>>>>> Regards,
> >>>>>>>>>> Nick
> >>>>>>>>>>
> >>>>>>>>>> On Wed, 19 Apr 2023 at 15:05, Nick Telford <
> >> nick.telford@gmail.com>
> >>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>
> >>>>>>>>>>> The issue is that if there's a crash between 2 and 3, then you
> >>>> still
> >>>>>>>>> end
> >>>>>>>>>>> up with inconsistent data in RocksDB. The only way to guarantee
> >>>> that
> >>>>>>>>> your
> >>>>>>>>>>> checkpoint offsets and locally stored data are consistent with
> >> each
> >>>>>>>>> other
> >>>>>>>>>>> are to atomically commit them, which can be achieved by having
> >> the
> >>>>>>>>> offsets
> >>>>>>>>>>> stored in RocksDB.
> >>>>>>>>>>>
> >>>>>>>>>>> The offsets column family is likely to be extremely small (one
> >>>>>>>>>>> per-changelog partition + one per Topology input partition for
> >>>>>> regular
> >>>>>>>>>>> stores, one per input partition for global stores). So the
> >> overhead
> >>>>>>>>> will be
> >>>>>>>>>>> minimal.
> >>>>>>>>>>>
> >>>>>>>>>>> A major benefit of doing this is that we can remove the
> explicit
> >>>>>> calls
> >>>>>>>>> to
> >>>>>>>>>>> db.flush(), which forcibly flushes memtables to disk on-commit.
> >> It
> >>>>>>>>> turns
> >>>>>>>>>>> out, RocksDB memtable flushes are largely dictated by Kafka
> >> Streams
> >>>>>>>>>>> commits, *not* RocksDB configuration, which could be a major
> >> source
> >>>>>> of
> >>>>>>>>>>> confusion. Atomic checkpointing makes it safe to remove these
> >>>>>> explicit
> >>>>>>>>>>> flushes, because it no longer matters exactly when RocksDB
> >> flushes
> >>>>>>>>> data to
> >>>>>>>>>>> disk; since the data and corresponding checkpoint offsets will
> >>>> always
> >>>>>>>>> be
> >>>>>>>>>>> flushed together, the local store is always in a consistent
> >> state,
> >>>>>> and
> >>>>>>>>>>> on-restart, it can always safely resume restoration from the
> >>>> on-disk
> >>>>>>>>>>> offsets, restoring the small amount of data that hadn't been
> >>>> flushed
> >>>>>>>>> when
> >>>>>>>>>>> the app exited/crashed.
> >>>>>>>>>>>
> >>>>>>>>>>> Regards,
> >>>>>>>>>>> Nick
> >>>>>>>>>>>
> >>>>>>>>>>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <
> colt@littlehorse.io>
> >>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Nick,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks for your reply. Ack to A) and B).
> >>>>>>>>>>>>
> >>>>>>>>>>>> For item C), I see what you're referring to. Your proposed
> >>>> solution
> >>>>>>>>> will
> >>>>>>>>>>>> work, so no need to change it. What I was suggesting was that
> it
> >>>>>>>>> might be
> >>>>>>>>>>>> possible to achieve this with only one column family. So long
> >> as:
> >>>>>>>>>>>>
> >>>>>>>>>>>>         - No uncommitted records (i.e. not committed to the
> >>>> changelog)
> >>>>>> are
> >>>>>>>>>>>>         *committed* to the state store, AND
> >>>>>>>>>>>>         - The Checkpoint offset (which refers to the changelog
> >>>> topic)
> >>>>>> is
> >>>>>>>>> less
> >>>>>>>>>>>>         than or equal to the last written changelog offset in
> >>>> rocksdb
> >>>>>>>>>>>>
> >>>>>>>>>>>> I don't see the need to do the full restoration from scratch.
> My
> >>>>>>>>>>>> understanding was that prior to 844/892, full restorations
> were
> >>>>>>>>> required
> >>>>>>>>>>>> because there could be uncommitted records written to RocksDB;
> >>>>>>>>> however,
> >>>>>>>>>>>> given your use of RocksDB transactions, that can be avoided
> with
> >>>> the
> >>>>>>>>>>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
> >>>>>>>>> transaction, 3)
> >>>>>>>>>>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <
> >>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> A. I've done my best to de-couple the StateStore stuff from
> the
> >>>>>> rest
> >>>>>>>>> of
> >>>>>>>>>>>> the
> >>>>>>>>>>>>> Streams engine. The fact that there will be only one ongoing
> >>>>>> (write)
> >>>>>>>>>>>>> transaction at a time is not guaranteed by any API, and is
> >> just a
> >>>>>>>>>>>>> consequence of the way Streams operates. To that end, I tried
> >> to
> >>>>>>>>> ensure
> >>>>>>>>>>>> the
> >>>>>>>>>>>>> documentation and guarantees provided by the new APIs are
> >>>>>>>>> independent of
> >>>>>>>>>>>>> this incidental behaviour. In practice, you're right, this
> >>>>>>>>> essentially
> >>>>>>>>>>>>> refers to "interactive queries", which are technically "read
> >>>>>>>>>>>> transactions",
> >>>>>>>>>>>>> even if they don't actually use the transaction API to
> isolate
> >>>>>>>>>>>> themselves.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> B. Yes, although not ideal. This is for backwards
> >> compatibility,
> >>>>>>>>>>>> because:
> >>>>>>>>>>>>>          1) Existing custom StateStore implementations will
> >>>> implement
> >>>>>>>>>>>> flush(),
> >>>>>>>>>>>>> and not commit(), but the Streams engine now calls commit(),
> so
> >>>>>> those
> >>>>>>>>>>>> calls
> >>>>>>>>>>>>> need to be forwarded to flush() for these legacy stores.
> >>>>>>>>>>>>>          2) Existing StateStore *users*, i.e. outside of the
> >>>> Streams
> >>>>>>>>> engine
> >>>>>>>>>>>>> itself, may depend on explicitly calling flush(), so for
> these
> >>>>>> cases,
> >>>>>>>>>>>>> flush() needs to be redirected to call commit().
> >>>>>>>>>>>>> If anyone has a better way to guarantee compatibility without
> >>>>>>>>>>>> introducing
> >>>>>>>>>>>>> this potential recursion loop, I'm open to changes!
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> C. This is described in the "Atomic Checkpointing" section.
> >>>> Offsets
> >>>>>>>>> are
> >>>>>>>>>>>>> stored in a separate RocksDB column family, which is
> guaranteed
> >>>> to
> >>>>>> be
> >>>>>>>>>>>>> atomically flushed to disk with all other column families.
> The
> >>>>>> issue
> >>>>>>>>> of
> >>>>>>>>>>>>> checkpoints being written to disk after commit causing
> >>>>>> inconsistency
> >>>>>>>>> if
> >>>>>>>>>>>> it
> >>>>>>>>>>>>> crashes in between is the reason why, under EOS, checkpoint
> >> files
> >>>>>> are
> >>>>>>>>>>>> only
> >>>>>>>>>>>>> written on clean shutdown. This is one of the major causes of
> >>>> "full
> >>>>>>>>>>>>> restorations", so moving the offsets into a place where they
> >> can
> >>>> be
> >>>>>>>>>>>>> guaranteed to be atomically written with the data they
> >> checkpoint
> >>>>>>>>>>>> allows us
> >>>>>>>>>>>>> to write the checkpoint offsets *on every commit*, not just
> on
> >>>>>> clean
> >>>>>>>>>>>>> shutdown.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <
> >> colt@littlehorse.io>
> >>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thank you for continuing this work. I have a few minor
> >>>> clarifying
> >>>>>>>>>>>>>> questions.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> A) "Records written to any transaction are visible to all
> >> other
> >>>>>>>>>>>>>> transactions immediately." I am confused here—I thought
> there
> >>>>>> could
> >>>>>>>>>>>> only
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>> one transaction going on at a time for a given state store
> >> given
> >>>>>> the
> >>>>>>>>>>>>>> threading model for processing records on a Task. Do you
> mean
> >>>>>>>>>>>> Interactive
> >>>>>>>>>>>>>> Queries by "other transactions"? (If so, then everything
> makes
> >>>>>>>>> sense—I
> >>>>>>>>>>>>>> thought that since IQ were read-only then they didn't count
> as
> >>>>>>>>>>>>>> transactions).
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> B) Is it intentional that the default implementations of the
> >>>>>> flush()
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>> commit() methods in the StateStore class refer to each other
> >> in
> >>>>>> some
> >>>>>>>>>>>> sort
> >>>>>>>>>>>>>> of unbounded recursion?
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> C) How will the getCommittedOffset() method work? At first I
> >>>>>> thought
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>> way to do it would be using a special key in the RocksDB
> store
> >>>> to
> >>>>>>>>>>>> store
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>> offset, and committing that with the transaction. But upon
> >>>> second
> >>>>>>>>>>>>> thought,
> >>>>>>>>>>>>>> since restoration from the changelog is an idempotent
> >>>> procedure, I
> >>>>>>>>>>>> think
> >>>>>>>>>>>>> it
> >>>>>>>>>>>>>> would be fine to 1) commit the RocksDB transaction and then
> 2)
> >>>>>> write
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>> offset to disk in a checkpoint file. If there is a crash
> >> between
> >>>>>> 1)
> >>>>>>>>>>>> and
> >>>>>>>>>>>>> 2),
> >>>>>>>>>>>>>> I think the only downside is 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 <
> >>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I've updated the KIP to reflect the latest version of the
> >>>> design:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> There are several changes in there that reflect feedback
> from
> >>>>>> this
> >>>>>>>>>>>>>> thread,
> >>>>>>>>>>>>>>> and there's a new section and a bunch of interface changes
> >>>>>> relating
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>> Atomic Checkpointing, which is the final piece of the
> puzzle
> >> to
> >>>>>>>>>>>> making
> >>>>>>>>>>>>>>> everything robust.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Let me know what you think!
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <
> >>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi Lucas,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Thanks for looking over my KIP.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> A) The bound is per-instance, not per-Task. This was a
> typo
> >> in
> >>>>>> the
> >>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>> that I've now corrected. It was originally per-Task, but I
> >>>>>>>>>>>> changed it
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> per-instance for exactly the reason you highlighted.
> >>>>>>>>>>>>>>>> B) It's worth noting that transactionality is only enabled
> >>>> under
> >>>>>>>>>>>> EOS,
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>> in the default mode of operation (ALOS), there should be
> no
> >>>>>>>>>>>> change in
> >>>>>>>>>>>>>>>> behavior at all. I think, under EOS, we can mitigate the
> >>>> impact
> >>>>>> on
> >>>>>>>>>>>>>> users
> >>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>> sufficiently low default values for the memory bound
> >>>>>>>>>>>> configuration. I
> >>>>>>>>>>>>>>>> understand your hesitation to include a significant change
> >> of
> >>>>>>>>>>>>>> behaviour,
> >>>>>>>>>>>>>>>> especially in a minor release, but I suspect that most
> users
> >>>>>> will
> >>>>>>>>>>>>>> prefer
> >>>>>>>>>>>>>>>> the memory impact (under EOS) to the existing behaviour of
> >>>>>>>>>>>> frequent
> >>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>> restorations! If this is a problem, the changes can wait
> >> until
> >>>>>> the
> >>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>> major release. I'll be running a patched version of
> streams
> >> in
> >>>>>>>>>>>>>> production
> >>>>>>>>>>>>>>>> with these changes as soon as they're ready, so it won't
> >>>> disrupt
> >>>>>>>>>>>> me
> >>>>>>>>>>>>> :-D
> >>>>>>>>>>>>>>>> C) The main purpose of this sentence was just to note that
> >>>> some
> >>>>>>>>>>>>> changes
> >>>>>>>>>>>>>>>> will need to be made to the way Segments are handled in
> >> order
> >>>> to
> >>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>> they also benefit from transactions. At the time I wrote
> >> it, I
> >>>>>>>>>>>> hadn't
> >>>>>>>>>>>>>>>> figured out the specific changes necessary, so it was
> >>>>>> deliberately
> >>>>>>>>>>>>>> vague.
> >>>>>>>>>>>>>>>> This is the one outstanding problem I'm currently working
> >> on,
> >>>>>> and
> >>>>>>>>>>>>> I'll
> >>>>>>>>>>>>>>>> update this section with more detail once I have figured
> out
> >>>> the
> >>>>>>>>>>>>> exact
> >>>>>>>>>>>>>>>> changes required.
> >>>>>>>>>>>>>>>> D) newTransaction() provides the necessary isolation
> >>>> guarantees.
> >>>>>>>>>>>>> While
> >>>>>>>>>>>>>>>> the RocksDB implementation of transactions doesn't
> >> technically
> >>>>>>>>>>>> *need*
> >>>>>>>>>>>>>>>> read-only users to call newTransaction(), other
> >>>> implementations
> >>>>>>>>>>>>> (e.g. a
> >>>>>>>>>>>>>>>> hypothetical PostgresStore) may require it. Calling
> >>>>>>>>>>>> newTransaction()
> >>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>> no transaction is necessary is essentially free, as it
> will
> >>>> just
> >>>>>>>>>>>>> return
> >>>>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think
> it
> >>>>>>>>>>>> should
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>> fairly obvious where the performance problems stem from:
> >>>> writes
> >>>>>>>>>>>> under
> >>>>>>>>>>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with
> >> the
> >>>>>>>>>>>>>>>> tombstone/record flag, 1 to decode it from the
> >>>> tombstone/record
> >>>>>>>>>>>> flag,
> >>>>>>>>>>>>>>> and 1
> >>>>>>>>>>>>>>>> to copy the record from the "temporary" store to the
> "main"
> >>>>>> store,
> >>>>>>>>>>>>> when
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> transaction commits. The different approach taken by
> KIP-869
> >>>>>>>>>>>> should
> >>>>>>>>>>>>>>> perform
> >>>>>>>>>>>>>>>> much better, as it avoids all these copies, and may
> actually
> >>>>>>>>>>>> perform
> >>>>>>>>>>>>>>>> slightly better than trunk, due to batched writes in
> RocksDB
> >>>>>>>>>>>>> performing
> >>>>>>>>>>>>>>>> better than non-batched writes.[1]
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 1:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>
> >>>>>>
> >> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
> >>>>>>>>>>>> lbrutschy@confluent.io
> >>>>>>>>>>>>>>> .invalid>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I'm just starting to read up on the whole discussion
> about
> >>>>>>>>>>>> KIP-892
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
> >>>>>>>>>>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do
> have
> >>>> some
> >>>>>>>>>>>>>>>>> questions about the latest draft.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>       A) If I understand correctly, you propose to put a
> >> bound
> >>>> on
> >>>>>> the
> >>>>>>>>>>>>>>>>> (native) memory consumed by each task. However, I wonder
> if
> >>>>>> this
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> sufficient if we have temporary imbalances in the
> cluster.
> >>>> For
> >>>>>>>>>>>>>>>>> example, depending on the timing of rebalances during a
> >>>> cluster
> >>>>>>>>>>>>>>>>> restart, it could happen that a single streams node is
> >>>>>> assigned a
> >>>>>>>>>>>>> lot
> >>>>>>>>>>>>>>>>> more tasks than expected. With your proposed change, this
> >>>> would
> >>>>>>>>>>>> mean
> >>>>>>>>>>>>>>>>> that the memory required by this one node could be a
> >> multiple
> >>>>>> of
> >>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>> is required during normal operation. I wonder if it
> >> wouldn't
> >>>> be
> >>>>>>>>>>>>> safer
> >>>>>>>>>>>>>>>>> to put a global bound on the memory use, across all
> tasks.
> >>>>>>>>>>>>>>>>>       B) Generally, the memory concerns still give me the
> >>>> feeling
> >>>>>>>>> that
> >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>> should not be enabled by default for all users in a minor
> >>>>>>>>>>>> release.
> >>>>>>>>>>>>>>>>>       C) In section "Transaction Management": the
> sentence
> >> "A
> >>>>>> similar
> >>>>>>>>>>>>>>>>> analogue will be created to automatically manage
> `Segment`
> >>>>>>>>>>>>>>>>> transactions.". Maybe this is just me lacking some
> >>>> background,
> >>>>>>>>>>>> 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 you to find out why the KIP-844 PoC is 20x
> >>>> slower
> >>>>>>>>>>>> (e.g.
> >>>>>>>>>>>>>>>>> by providing a flame graph), that would be quite
> >> interesting.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>>>> Lucas
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
> >>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I've updated the KIP with a more detailed design, which
> >>>>>>>>>>>> reflects
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> implementation I've been working on:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> This new design should address the outstanding points
> >>>> already
> >>>>>>>>>>>> made
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Please let me know if there are areas that are unclear
> or
> >>>> need
> >>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>>> clarification.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I have a (nearly) working implementation. I'm confident
> >> that
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> remaining
> >>>>>>>>>>>>>>>>>> work (making Segments behave) will not impact the
> >> documented
> >>>>>>>>>>>>> design.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <
> >>>>>> colt@littlehorse.io
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thank you for the reply; that makes sense. I was hoping
> >>>> that,
> >>>>>>>>>>>>>> since
> >>>>>>>>>>>>>>>>> reading
> >>>>>>>>>>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
> >>>>>>>>>>>> documented
> >>>>>>>>>>>>>> API,
> >>>>>>>>>>>>>>>>> maybe
> >>>>>>>>>>>>>>>>>>> you *wouldn't* have to wait for the next major release
> to
> >>>>>>>>>>>> make
> >>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>> change;
> >>>>>>>>>>>>>>>>>>> but given 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 <
> >>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended
> >> to
> >>>>>>>>>>>> try
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> keep the
> >>>>>>>>>>>>>>>>>>>> behaviour unchanged as much as possible, otherwise
> we'd
> >>>>>>>>>>>> have
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> wait for
> >>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> major version release to land these changes.
> >>>>>>>>>>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
> >>>>>>>>>>>> guarantee,
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> typically longer commit intervals would be problematic
> >>>> when
> >>>>>>>>>>>>>>> reading
> >>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>> "committed" records.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I've been away for 5 days recovering from minor
> surgery,
> >>>>>>>>>>>> but I
> >>>>>>>>>>>>>>>>> spent a
> >>>>>>>>>>>>>>>>>>>> considerable amount of that time working through ideas
> >> for
> >>>>>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>>>>>> solutions in my head. I think your suggestion of
> keeping
> >>>>>>>>>>>> ALOS
> >>>>>>>>>>>>>>>>> as-is, but
> >>>>>>>>>>>>>>>>>>>> buffering writes for EOS is the right path forwards,
> >>>>>>>>>>>> although
> >>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>>>> solution that both expands on this, and provides for
> >> some
> >>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>> formal
> >>>>>>>>>>>>>>>>>>>> guarantees.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Essentially, adding support to KeyValueStores for
> >>>>>>>>>>>>>> "Transactions",
> >>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>> clearly defined IsolationLevels. Using "Read
> Committed"
> >>>>>>>>>>>> when
> >>>>>>>>>>>>>> under
> >>>>>>>>>>>>>>>>> EOS,
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>> "Read Uncommitted" under ALOS.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The nice thing about this approach is that it gives us
> >>>> much
> >>>>>>>>>>>>> more
> >>>>>>>>>>>>>>>>> clearly
> >>>>>>>>>>>>>>>>>>>> defined isolation behaviour that can be properly
> >>>>>>>>>>>> documented to
> >>>>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>>> users
> >>>>>>>>>>>>>>>>>>>> know what to expect.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I'm still working out the kinks in the design, and
> will
> >>>>>>>>>>>> update
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>>>> I have something. The main struggle is trying to
> >> implement
> >>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>>>> making any major changes to the existing interfaces or
> >>>>>>>>>>>>> breaking
> >>>>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>>> implementations, because currently everything expects
> to
> >>>>>>>>>>>>> operate
> >>>>>>>>>>>>>>>>> directly
> >>>>>>>>>>>>>>>>>>>> on a StateStore, and not a Transaction of that store.
> I
> >>>>>>>>>>>> think
> >>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>>>>> close, although sadly I won't be able to progress much
> >>>>>>>>>>>> until
> >>>>>>>>>>>>>> next
> >>>>>>>>>>>>>>>>> week
> >>>>>>>>>>>>>>>>>>> due
> >>>>>>>>>>>>>>>>>>>> to some work commitments.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
> >>>>>>>>>>>>> colt@littlehorse.io>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thank you for the explanation, and also for the
> updated
> >>>>>>>>>>>>> KIP. I
> >>>>>>>>>>>>>>> am
> >>>>>>>>>>>>>>>>> quite
> >>>>>>>>>>>>>>>>>>>>> eager for this improvement to be released as it would
> >>>>>>>>>>>>> greatly
> >>>>>>>>>>>>>>>>> reduce
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> operational difficulties of EOS streams apps.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Two questions:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 10)
> >>>>>>>>>>>>>>>>>>>>>> When reading records, we will use the
> >>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
> >>>>>>>>>>>>>>>>>>>>>       and WriteBatchWithIndex#newIteratorWithBase
> >>>> utilities in
> >>>>>>>>>>>>>> order
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>>>>> that uncommitted writes are available to query.
> >>>>>>>>>>>>>>>>>>>>> Why do extra work to enable the reading of
> uncommitted
> >>>>>>>>>>>>> writes
> >>>>>>>>>>>>>>>>> during
> >>>>>>>>>>>>>>>>>>> IQ?
> >>>>>>>>>>>>>>>>>>>>> Code complexity aside, reading uncommitted writes is,
> >> in
> >>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>> opinion, a
> >>>>>>>>>>>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have
> the
> >>>>>>>>>>>>>>> guarantee
> >>>>>>>>>>>>>>>>> that,
> >>>>>>>>>>>>>>>>>>>>> with EOS, IQ only reads committed records. In order
> to
> >>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>> dirty
> >>>>>>>>>>>>>>>>>>> reads,
> >>>>>>>>>>>>>>>>>>>>> one currently must query a standby replica (but this
> >>>>>>>>>>>> still
> >>>>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>>>>> fully
> >>>>>>>>>>>>>>>>>>>>> guarantee monotonic reads).
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 20) Is it also necessary to enable this optimization
> on
> >>>>>>>>>>>> ALOS
> >>>>>>>>>>>>>>>>> stores?
> >>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need
> to
> >>>>>>>>>>>>> restore
> >>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
> >>>>>>>>>>>>> acceptable
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> accept
> >>>>>>>>>>>>>>>>>>>>> that there may have been uncommitted writes on disk.
> >> On a
> >>>>>>>>>>>>> side
> >>>>>>>>>>>>>>>>> note, if
> >>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>> enable this type of store on ALOS processors, the
> >>>>>>>>>>>> community
> >>>>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
> >>>>>>>>>>>>> longer
> >>>>>>>>>>>>>>>>> include
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> StateStore management of checkpointing. This can be
> >>>>>>>>>>>> added
> >>>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>>>> to further optimize the consistency and performance
> of
> >>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>> stores.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I've also added a section discussing some of the
> >>>>>>>>>>>> concerns
> >>>>>>>>>>>>>>> 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 2022 at 16:36, Nick Telford <
> >>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I didn't do any profiling, but the 844
> >>>>>>>>>>>> implementation:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>         - Writes uncommitted records to a temporary
> >>>>>>>>>>>> RocksDB
> >>>>>>>>>>>>>>>>> instance
> >>>>>>>>>>>>>>>>>>>>>>>            - Since tombstones need to be flagged,
> all
> >>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>> values are
> >>>>>>>>>>>>>>>>>>>>>>>            prefixed with a value/tombstone marker.
> >> This
> >>>>>>>>>>>>>>>>> necessitates a
> >>>>>>>>>>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>>>>>> copy.
> >>>>>>>>>>>>>>>>>>>>>>>         - On-commit, iterates all records in this
> >>>>>>>>>>>> temporary
> >>>>>>>>>>>>>>>>> instance and
> >>>>>>>>>>>>>>>>>>>>>>>         writes them to the main RocksDB store.
> >>>>>>>>>>>>>>>>>>>>>>>         - While iterating, the value/tombston

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

Posted by John Roesler <jo...@vvcephei.org>.
Touché!

Ok, I agree that figuring out the case of a disabled cache would be 
non-trivial. Ingesting single-record SST files will probably not be 
performant, but benchmarking may prove different. Or maybe we can have 
some reserved cache space on top of the user-configured cache, which we 
would have reclaimed from the memtable space. Or some other, more 
creative solution.

Thanks,
-John

On 6/20/23 15:30, Nick Telford wrote:
>> Note that users can disable the cache, which would still be
> ok, I think. We wouldn't ingest the SST files on every record, but just
> append to them and only ingest them on commit, when we're already
> waiting for acks and a RocksDB commit.
> 
> In this case, how would uncommitted records be read by joins?
> 
> On Tue, 20 Jun 2023, 20:51 John Roesler, <vv...@apache.org> wrote:
> 
>> Ah, sorry Nick,
>>
>> I just meant the regular heap based cache that we maintain in Streams. I
>> see that it's not called "RecordCache" (my mistake).
>>
>> The actual cache is ThreadCache:
>>
>> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
>>
>> Here's the example of how we use the cache in KeyValueStore:
>>
>> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
>>
>> It's basically just an on-heap Map of records that have not yet been
>> written to the changelog or flushed into the underlying store. It gets
>> flushed when the total cache size exceeds `cache.max.bytes.buffering` or
>> the `commit.interval.ms` elapses.
>>
>> Speaking of those configs, another benefit to this idea is that we would
>> no longer need to trigger extra commits based on the size of the ongoing
>> transaction. Instead, we'd just preserve the existing cache-flush
>> behavior. Note that users can disable the cache, which would still be
>> ok, I think. We wouldn't ingest the SST files on every record, but just
>> append to them and only ingest them on commit, when we're already
>> waiting for acks and a RocksDB commit.
>>
>> Thanks,
>> -John
>>
>> On 6/20/23 14:09, Nick Telford wrote:
>>> Hi John,
>>>
>>> By "RecordCache", do you mean the RocksDB "WriteBatch"? I can't find any
>>> class called "RecordCache"...
>>>
>>> Cheers,
>>>
>>> Nick
>>>
>>> On Tue, 20 Jun 2023 at 19:42, John Roesler <vv...@apache.org> wrote:
>>>
>>>> Hi Nick,
>>>>
>>>> Thanks for picking this up again!
>>>>
>>>> I did have one new thought over the intervening months, which I'd like
>>>> your take on.
>>>>
>>>> What if, instead of using the RocksDB atomic write primitive at all, we
>>>> instead just:
>>>> 1. disable memtables entirely
>>>> 2. directly write the RecordCache into SST files when we flush
>>>> 3. atomically ingest the SST file(s) into RocksDB when we get the ACK
>>>> from the changelog (see
>>>>
>>>>
>> https://github.com/EighteenZi/rocksdb_wiki/blob/master/Creating-and-Ingesting-SST-files.md
>>>> and
>>>>
>>>>
>> https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/IngestExternalFileOptions.java
>>>> and
>>>>
>>>>
>> https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L1413-L1429
>>>> )
>>>> 4. track the changelog offsets either in another CF or the same CF with
>>>> a reserved key, either of which will make the changelog offset update
>>>> atomic with the file ingestions
>>>>
>>>> I suspect this'll have a number of benefits:
>>>> * writes to RocksDB will always be atomic
>>>> * we don't fragment memory between the RecordCache and the memtables
>>>> * RecordCache gives far higher performance than memtable for reads and
>>>> writes
>>>> * we don't need any new "transaction" concepts or memory bound configs
>>>>
>>>> What do you think?
>>>>
>>>> Thanks,
>>>> -John
>>>>
>>>> On 6/20/23 10:51, Nick Telford wrote:
>>>>> Hi Bruno,
>>>>>
>>>>> Thanks for reviewing the KIP. It's been a long road, I started working
>> on
>>>>> this more than a year ago, and most of the time in the last 6 months
>> has
>>>>> been spent on the "Atomic Checkpointing" stuff that's been benched, so
>>>> some
>>>>> of the reasoning behind some of my decisions have been lost, but I'll
>> do
>>>> my
>>>>> best to reconstruct them.
>>>>>
>>>>> 1.
>>>>> IIRC, this was the initial approach I tried. I don't remember the exact
>>>>> reasons I changed it to use a separate "view" of the StateStore that
>>>>> encapsulates the transaction, but I believe it had something to do with
>>>>> concurrent access to the StateStore from Interactive Query threads.
>> Reads
>>>>> from interactive queries need to be isolated from the currently ongoing
>>>>> transaction, both for consistency (so interactive queries don't observe
>>>>> changes that are subsequently rolled-back), but also to prevent
>> Iterators
>>>>> opened by an interactive query from being closed and invalidated by the
>>>>> StreamThread when it commits the transaction, which causes your
>>>> interactive
>>>>> queries to crash.
>>>>>
>>>>> Another reason I believe I implemented it this way was a separation of
>>>>> concerns. Recall that newTransaction() originally created an object of
>>>> type
>>>>> Transaction, not StateStore. My intent was to improve the type-safety
>> of
>>>>> the API, in an effort to ensure Transactions weren't used incorrectly.
>>>>> Unfortunately, this didn't pan out, but newTransaction() remained.
>>>>>
>>>>> Finally, this had the added benefit that implementations could easily
>> add
>>>>> support for transactions *without* re-writing their existing,
>>>>> non-transactional implementation. I think this can be a benefit both
>> for
>>>>> implementers of custom StateStores, but also for anyone extending
>>>>> RocksDbStore, as they can rely on the existing access methods working
>> how
>>>>> they expect them to.
>>>>>
>>>>> I'm not too happy with the way the current design has panned out, so
>> I'm
>>>>> open to ideas on how to improve it. Key to this is finding some way to
>>>>> ensure that reads from Interactive Query threads are properly isolated
>>>> from
>>>>> the transaction, *without* the performance overhead of checking which
>>>>> thread the method is being called from on every access.
>>>>>
>>>>> As for replacing flush() with commit() - I saw no reason to add this
>>>>> complexity to the KIP, unless there was a need to add arguments to the
>>>>> flush/commit method. This need arises with Atomic Checkpointing, but
>> that
>>>>> will be implemented separately, in a future KIP. Do you see a need for
>>>> some
>>>>> arguments to the flush/commit method that I've missed? Or were you
>> simply
>>>>> suggesting a rename?
>>>>>
>>>>> 2.
>>>>> This is simply due to the practical reason that isolationLevel() is
>>>> really
>>>>> a proxy for checking if the app is under EOS. The application
>>>> configuration
>>>>> is not provided to the constructor of StateStores, but it *is* provided
>>>> to
>>>>> init(), via StateStoreContext. For this reason, it seemed somewhat
>>>> natural
>>>>> to add it to StateStoreContext. I think this makes sense, since the
>>>>> IsolationLevel of all StateStores in an application *must* be the same,
>>>> and
>>>>> since those stores are all initialized with the same StateStoreContext,
>>>> it
>>>>> seems natural for that context to carry the desired IsolationLevel to
>>>> use.
>>>>>
>>>>> 3.
>>>>> Using IsolationLevel instead of just passing `boolean eosEnabled`, like
>>>>> much of the internals was an attempt to logically de-couple the
>>>> StateStore
>>>>> API from the internals of Kafka Streams. Technically, StateStores don't
>>>>> need to know/care what processing mode the KS app is using, all they
>> need
>>>>> to know is the isolation level expected of them.
>>>>>
>>>>> Having formal definitions for the expectations of the two required
>>>>> IsolationLevels allow implementers to implement transactional stores
>>>>> without having to dig through the internals of Kafka Streams and
>>>> understand
>>>>> exactly how they are used. The tight coupling between state stores and
>>>>> internal behaviour has actually significantly hindered my progress on
>>>> this
>>>>> KIP, and encouraged me to avoid increasing this logical coupling as
>> much
>>>> as
>>>>> possible.
>>>>>
>>>>> This also frees implementations to satisfy those requirements in any
>> way
>>>>> they choose. Transactions might not be the only/available approach to
>> an
>>>>> implementation, but they might have an alternative way to satisfy the
>>>>> isolation requirements. I admit that this point is more about
>> semantics,
>>>>> but "transactional" would need to be formally defined in order for
>>>>> implementers to provide a valid implementation, and these
>> IsolationLevels
>>>>> provide that formal definition.
>>>>>
>>>>> 4.
>>>>> I can remove them. I added them only as I planned to include them in
>> the
>>>>> org.apache.kafka.streams.state package, as a recommended base
>>>>> implementation for all StateStores, including those implemented by
>>>> users. I
>>>>> had assumed that anything in "public" packages, such as
>>>>> org.apache.kafka.streams.state, should be included in a KIP. Is that
>>>> wrong?
>>>>>
>>>>> 5.
>>>>> RocksDB provides no way to measure the actual size of a
>>>>> WriteBatch(WithIndex), so we're limited to tracking the sum total of
>> the
>>>>> size of keys + values that are written to the transaction. This
>> obviously
>>>>> under-estimates the actual memory usage, because WriteBatch no-doubt
>>>>> includes some record overheads, and WriteBatchWithIndex has to maintain
>>>> an
>>>>> index.
>>>>>
>>>>> Ideally, we could trivially add a method upstream to
>> WriteBatchInterface
>>>>> that provides the exact size of the batch, but that would require an
>>>>> upgrade of RocksDB, which won't happen soon. So for the time being,
>> we're
>>>>> stuck with an approximation, so I felt that the new method should
>> reflect
>>>>> that.
>>>>>
>>>>> Would you prefer the new method name ignores this constraint and that
>> we
>>>>> simply make the rocks measurement more accurate in the future?
>>>>>
>>>>> 6.
>>>>> Done
>>>>>
>>>>> 7.
>>>>> Very good point. The KIP already specifically calls out memory in the
>>>>> documentation of the config: "Maximum number of memory bytes to be used
>>>> to
>>>>> buffer uncommitted state-store records." - did you have something else
>> in
>>>>> mind?
>>>>>
>>>>> Should we also make this clearer by renaming the config property
>> itself?
>>>>> Perhaps to something like statestore.transaction.buffer.max.bytes?
>>>>>
>>>>> 8.
>>>>> OK, I can remove this. The intent here was to describe how Streams
>> itself
>>>>> will manage transaction roll-over etc. Presumably that means we also
>>>> don't
>>>>> need a description of how Streams will manage the commit of changelog
>>>>> transactions, state store transactions and checkpointing?
>>>>>
>>>>> 9.
>>>>> What do you mean by fail-over? Do you mean failing over an Active Task
>> to
>>>>> an instance already hosting a Standby Task?
>>>>>
>>>>> Thanks again and sorry for the essay of a response!
>>>>>
>>>>> Regards,
>>>>> Nick
>>>>>
>>>>> On Tue, 20 Jun 2023 at 10:49, Bruno Cadonna <ca...@apache.org>
>> wrote:
>>>>>
>>>>>> Hi Nick,
>>>>>>
>>>>>> Thanks for the updates!
>>>>>>
>>>>>> I really appreciate that you simplified the KIP by removing some
>>>>>> aspects. As I have already told you, I think the removed aspects are
>>>>>> also good ideas and we can discuss them on follow-up KIPs.
>>>>>>
>>>>>> Regarding the current KIP, I have the following feedback.
>>>>>>
>>>>>> 1.
>>>>>> Is there a good reason to add method newTransaction() to the
>> StateStore
>>>>>> interface? As far as I understand, the idea is that users of a state
>>>>>> store (transactional or not) call this method at start-up and after
>> each
>>>>>> commit. Since the call to newTransaction() is done in any case and I
>>>>>> think it would simplify the caller code if we just start a new
>>>>>> transaction after a commit in the implementation?
>>>>>> As far as I understand, you plan to commit the transaction in the
>>>>>> flush() method. I find the idea to replace flush() with commit()
>>>>>> presented in KIP-844 an elegant solution.
>>>>>>
>>>>>> 2.
>>>>>> Why is the method to query the isolation level added to the state
>> store
>>>>>> context?
>>>>>>
>>>>>> 3.
>>>>>> Do we need all the isolation level definitions? I think it is good to
>>>>>> know the guarantees of the transactionality of the state store.
>>>>>> However, currently, Streams guarantees that there will only be one
>>>>>> transaction that writes to the state store. Only the stream thread
>> that
>>>>>> executes the active task that owns the state store will write to the
>>>>>> state store. I think it should be enough to know if the state store is
>>>>>> transactional or not. So my proposal would be to just add a method on
>>>>>> the state store interface the returns if a state store is
>> transactional
>>>>>> or not by returning a boolean or an enum.
>>>>>>
>>>>>> 4.
>>>>>> I am wondering why AbstractTransaction and AbstractTransactionalStore
>>>>>> are part of the KIP. They look like implementation details that should
>>>>>> not be exposed in the public API.
>>>>>>
>>>>>> 5.
>>>>>> Why does StateStore#approximateNumUncommittedBytes() return an
>>>>>> approximate number of bytes?
>>>>>>
>>>>>> 6.
>>>>>> RocksDB is just one implementation of the state stores in Streams.
>>>>>> However, the issues regarding OOM errors might also apply to other
>>>>>> custom implementations. So in the KIP I would extract that part from
>>>>>> section "RocksDB Transaction". I would also move section "RocksDB
>>>>>> Transaction" to the end of section "Proposed Changes" and handle it as
>>>>>> an example implementation for a state store.
>>>>>>
>>>>>> 7.
>>>>>> Should statestore.uncommitted.max.bytes only limit the uncommitted
>> bytes
>>>>>> or the uncommitted bytes that reside in memory? In future, other
>>>>>> transactional state store implementations might implement a buffer for
>>>>>> uncommitted records that are able to spill records on disk. I think
>>>>>> statestore.uncommitted.max.bytes needs to limit the uncommitted bytes
>>>>>> irrespective if they reside in memory or disk. Since Streams will use
>>>>>> this config to decide if it needs to trigger a commit, state store
>>>>>> implementations that can spill to disk will never be able to spill to
>>>>>> disk. You would only need to change the doc of the config, if you
>> agree
>>>>>> with me.
>>>>>>
>>>>>> 8.
>>>>>> Section "Transaction Management" about the wrappers is rather a
>>>>>> implementation detail that should not be in the KIP.
>>>>>>
>>>>>> 9.
>>>>>> Could you add a section that describes how failover will work with the
>>>>>> transactional state stores? I think section "Error handling" is
>> already
>>>>>> a good start.
>>>>>>
>>>>>>
>>>>>> Best,
>>>>>> Bruno
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 15.05.23 11:04, Nick Telford wrote:
>>>>>>> Hi everyone,
>>>>>>>
>>>>>>> Quick update: I've added a new section to the KIP: "Offsets for
>>>> Consumer
>>>>>>> Rebalances", that outlines my solution to the problem that
>>>>>>> StreamsPartitionAssignor needs to read StateStore offsets even if
>>>> they're
>>>>>>> not currently open.
>>>>>>>
>>>>>>> Regards,
>>>>>>> Nick
>>>>>>>
>>>>>>> On Wed, 3 May 2023 at 11:34, Nick Telford <ni...@gmail.com>
>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Bruno,
>>>>>>>>
>>>>>>>> Thanks for reviewing my proposal.
>>>>>>>>
>>>>>>>> 1.
>>>>>>>> The main reason I added it was because it was easy to do. If we see
>> no
>>>>>>>> value in it, I can remove it.
>>>>>>>>
>>>>>>>> 2.
>>>>>>>> Global StateStores can have multiple partitions in their input
>> topics
>>>>>>>> (which function as their changelogs), so they would have more than
>> one
>>>>>>>> partition.
>>>>>>>>
>>>>>>>> 3.
>>>>>>>> That's a good point. At present, the only method it adds is
>>>>>>>> isolationLevel(), which is likely not necessary outside of
>>>> StateStores.
>>>>>>>> It *does* provide slightly different guarantees in the documentation
>>>> to
>>>>>>>> several of the methods (hence the overrides). I'm not sure if this
>> is
>>>>>>>> enough to warrant a new interface though.
>>>>>>>> I think the question that remains is whether this interface makes it
>>>>>>>> easier to implement custom transactional StateStores than if we were
>>>> to
>>>>>>>> remove it? Probably not.
>>>>>>>>
>>>>>>>> 4.
>>>>>>>> The main motivation for the Atomic Checkpointing is actually
>>>>>> performance.
>>>>>>>> My team has been testing out an implementation of this KIP without
>> it,
>>>>>> and
>>>>>>>> we had problems with RocksDB doing *much* more compaction, due to
>> the
>>>>>>>> significantly increased flush rate. It was enough of a problem that
>>>> (for
>>>>>>>> the time being), we had to revert back to Kafka Streams proper.
>>>>>>>> I think the best way to solve this, as you say, is to keep the
>>>>>> .checkpoint
>>>>>>>> files *in addition* to the offsets being stored within the store
>>>> itself.
>>>>>>>> Essentially, when closing StateStores, we force a memtable flush,
>> then
>>>>>>>> call getCommittedOffsets and write those out to the .checkpoint
>> file.
>>>>>>>> That would ensure the metadata is available to the
>>>>>>>> StreamsPartitionAssignor for all closed stores.
>>>>>>>> If there's a crash (no clean close), then we won't be able to
>>>> guarantee
>>>>>>>> which offsets were flushed to disk by RocksDB, so we'd need to open
>> (
>>>>>>>> init()), read offsets, and then close() those stores. But since this
>>>> is
>>>>>>>> the exception, and will only occur once (provided it doesn't crash
>>>> every
>>>>>>>> time!), I think the performance impact here would be acceptable.
>>>>>>>>
>>>>>>>> Thanks for the feedback, please let me know if you have any more
>>>>>> comments
>>>>>>>> or questions!
>>>>>>>>
>>>>>>>> I'm currently working on rebasing against trunk. This involves
>> adding
>>>>>>>> support for transactionality to VersionedStateStores. I will
>> probably
>>>>>> need
>>>>>>>> to revise my implementation for transactional "segmented" stores,
>> both
>>>>>> to
>>>>>>>> accommodate VersionedStateStore, and to clean up some other stuff.
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>> Nick
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org>
>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Nick,
>>>>>>>>>
>>>>>>>>> Thanks for the updates!
>>>>>>>>>
>>>>>>>>> I have a couple of questions/comments.
>>>>>>>>>
>>>>>>>>> 1.
>>>>>>>>> Why do you propose a configuration that involves max. bytes and
>> max.
>>>>>>>>> reords? I think we are mainly concerned about memory consumption
>>>>>> because
>>>>>>>>> we want to limit the off-heap memory used. I cannot think of a case
>>>>>>>>> where one would want to set the max. number of records.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 2.
>>>>>>>>> Why does
>>>>>>>>>
>>>>>>>>>       default void commit(final Map<TopicPartition, Long>
>>>>>> changelogOffsets) {
>>>>>>>>>           flush();
>>>>>>>>>       }
>>>>>>>>>
>>>>>>>>> take a map of partitions to changelog offsets?
>>>>>>>>> The mapping between state stores to partitions is a 1:1
>> relationship.
>>>>>>>>> Passing in a single changelog offset should suffice.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 3.
>>>>>>>>> Why do we need the Transaction interface? It should be possible to
>>>> hide
>>>>>>>>> beginning and committing a transactions withing the state store
>>>>>>>>> implementation, so that from outside the state store, it does not
>>>>>> matter
>>>>>>>>> whether the state store is transactional or not. What would be the
>>>>>>>>> advantage of using the Transaction interface?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 4.
>>>>>>>>> Regarding checkpointing offsets, I think we should keep the
>>>> checkpoint
>>>>>>>>> file in any case for the reason you mentioned about rebalancing.
>> Even
>>>>>> if
>>>>>>>>> that would not be an issue, I would propose to move the change to
>>>>>> offset
>>>>>>>>> management to a new KIP and to not add more complexity than needed
>> to
>>>>>>>>> this one. I would not be too concerned about the consistency
>>>> violation
>>>>>>>>> you mention. As far as I understand, with transactional state
>> stores
>>>>>>>>> Streams would write the checkpoint file during every commit even
>>>> under
>>>>>>>>> EOS. In the failure case you describe, Streams would restore the
>>>> state
>>>>>>>>> stores from the offsets found in the checkpoint file written during
>>>> the
>>>>>>>>> penultimate commit instead of during the last commit. Basically,
>>>>>> Streams
>>>>>>>>> would overwrite the records written to the state store between the
>>>> last
>>>>>>>>> two commits with the same records read from the changelogs. While I
>>>>>>>>> understand that this is wasteful, it is -- at the same time --
>>>>>>>>> acceptable and most importantly it does not break EOS.
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> Bruno
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 27.04.23 12:34, Nick Telford wrote:
>>>>>>>>>> Hi everyone,
>>>>>>>>>>
>>>>>>>>>> I find myself (again) considering removing the offset management
>>>> from
>>>>>>>>>> StateStores, and keeping the old checkpoint file system. The
>> reason
>>>> is
>>>>>>>>> that
>>>>>>>>>> the StreamPartitionAssignor directly reads checkpoint files in
>> order
>>>>>> to
>>>>>>>>>> determine which instance has the most up-to-date copy of the local
>>>>>>>>> state.
>>>>>>>>>> If we move offsets into the StateStore itself, then we will need
>> to
>>>>>>>>> open,
>>>>>>>>>> initialize, read offsets and then close each StateStore (that is
>> not
>>>>>>>>>> already assigned and open) for which we have *any* local state, on
>>>>>> every
>>>>>>>>>> rebalance.
>>>>>>>>>>
>>>>>>>>>> Generally, I don't think there are many "orphan" stores like this
>>>>>>>>> sitting
>>>>>>>>>> around on most instances, but even a few would introduce
>> additional
>>>>>>>>> latency
>>>>>>>>>> to an already somewhat lengthy rebalance procedure.
>>>>>>>>>>
>>>>>>>>>> I'm leaning towards Colt's (Slack) suggestion of just keeping
>> things
>>>>>> in
>>>>>>>>> the
>>>>>>>>>> checkpoint file(s) for now, and not worrying about the race. The
>>>>>>>>> downside
>>>>>>>>>> is that we wouldn't be able to remove the explicit RocksDB flush
>>>>>>>>> on-commit,
>>>>>>>>>> which likely hurts performance.
>>>>>>>>>>
>>>>>>>>>> If anyone has any thoughts or ideas on this subject, I would
>>>>>> appreciate
>>>>>>>>> it!
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>> Nick
>>>>>>>>>>
>>>>>>>>>> On Wed, 19 Apr 2023 at 15:05, Nick Telford <
>> nick.telford@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>
>>>>>>>>>>> The issue is that if there's a crash between 2 and 3, then you
>>>> still
>>>>>>>>> end
>>>>>>>>>>> up with inconsistent data in RocksDB. The only way to guarantee
>>>> that
>>>>>>>>> your
>>>>>>>>>>> checkpoint offsets and locally stored data are consistent with
>> each
>>>>>>>>> other
>>>>>>>>>>> are to atomically commit them, which can be achieved by having
>> the
>>>>>>>>> offsets
>>>>>>>>>>> stored in RocksDB.
>>>>>>>>>>>
>>>>>>>>>>> The offsets column family is likely to be extremely small (one
>>>>>>>>>>> per-changelog partition + one per Topology input partition for
>>>>>> regular
>>>>>>>>>>> stores, one per input partition for global stores). So the
>> overhead
>>>>>>>>> will be
>>>>>>>>>>> minimal.
>>>>>>>>>>>
>>>>>>>>>>> A major benefit of doing this is that we can remove the explicit
>>>>>> calls
>>>>>>>>> to
>>>>>>>>>>> db.flush(), which forcibly flushes memtables to disk on-commit.
>> It
>>>>>>>>> turns
>>>>>>>>>>> out, RocksDB memtable flushes are largely dictated by Kafka
>> Streams
>>>>>>>>>>> commits, *not* RocksDB configuration, which could be a major
>> source
>>>>>> of
>>>>>>>>>>> confusion. Atomic checkpointing makes it safe to remove these
>>>>>> explicit
>>>>>>>>>>> flushes, because it no longer matters exactly when RocksDB
>> flushes
>>>>>>>>> data to
>>>>>>>>>>> disk; since the data and corresponding checkpoint offsets will
>>>> always
>>>>>>>>> be
>>>>>>>>>>> flushed together, the local store is always in a consistent
>> state,
>>>>>> and
>>>>>>>>>>> on-restart, it can always safely resume restoration from the
>>>> on-disk
>>>>>>>>>>> offsets, restoring the small amount of data that hadn't been
>>>> flushed
>>>>>>>>> when
>>>>>>>>>>> the app exited/crashed.
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>> Nick
>>>>>>>>>>>
>>>>>>>>>>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io>
>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Nick,
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for your reply. Ack to A) and B).
>>>>>>>>>>>>
>>>>>>>>>>>> For item C), I see what you're referring to. Your proposed
>>>> solution
>>>>>>>>> will
>>>>>>>>>>>> work, so no need to change it. What I was suggesting was that it
>>>>>>>>> might be
>>>>>>>>>>>> possible to achieve this with only one column family. So long
>> as:
>>>>>>>>>>>>
>>>>>>>>>>>>         - No uncommitted records (i.e. not committed to the
>>>> changelog)
>>>>>> are
>>>>>>>>>>>>         *committed* to the state store, AND
>>>>>>>>>>>>         - The Checkpoint offset (which refers to the changelog
>>>> topic)
>>>>>> is
>>>>>>>>> less
>>>>>>>>>>>>         than or equal to the last written changelog offset in
>>>> rocksdb
>>>>>>>>>>>>
>>>>>>>>>>>> I don't see the need to do the full restoration from scratch. My
>>>>>>>>>>>> understanding was that prior to 844/892, full restorations were
>>>>>>>>> required
>>>>>>>>>>>> because there could be uncommitted records written to RocksDB;
>>>>>>>>> however,
>>>>>>>>>>>> given your use of RocksDB transactions, that can be avoided with
>>>> the
>>>>>>>>>>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
>>>>>>>>> transaction, 3)
>>>>>>>>>>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <
>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>
>>>>>>>>>>>>> A. I've done my best to de-couple the StateStore stuff from the
>>>>>> rest
>>>>>>>>> of
>>>>>>>>>>>> the
>>>>>>>>>>>>> Streams engine. The fact that there will be only one ongoing
>>>>>> (write)
>>>>>>>>>>>>> transaction at a time is not guaranteed by any API, and is
>> just a
>>>>>>>>>>>>> consequence of the way Streams operates. To that end, I tried
>> to
>>>>>>>>> ensure
>>>>>>>>>>>> the
>>>>>>>>>>>>> documentation and guarantees provided by the new APIs are
>>>>>>>>> independent of
>>>>>>>>>>>>> this incidental behaviour. In practice, you're right, this
>>>>>>>>> essentially
>>>>>>>>>>>>> refers to "interactive queries", which are technically "read
>>>>>>>>>>>> transactions",
>>>>>>>>>>>>> even if they don't actually use the transaction API to isolate
>>>>>>>>>>>> themselves.
>>>>>>>>>>>>>
>>>>>>>>>>>>> B. Yes, although not ideal. This is for backwards
>> compatibility,
>>>>>>>>>>>> because:
>>>>>>>>>>>>>          1) Existing custom StateStore implementations will
>>>> implement
>>>>>>>>>>>> flush(),
>>>>>>>>>>>>> and not commit(), but the Streams engine now calls commit(), so
>>>>>> those
>>>>>>>>>>>> calls
>>>>>>>>>>>>> need to be forwarded to flush() for these legacy stores.
>>>>>>>>>>>>>          2) Existing StateStore *users*, i.e. outside of the
>>>> Streams
>>>>>>>>> engine
>>>>>>>>>>>>> itself, may depend on explicitly calling flush(), so for these
>>>>>> cases,
>>>>>>>>>>>>> flush() needs to be redirected to call commit().
>>>>>>>>>>>>> If anyone has a better way to guarantee compatibility without
>>>>>>>>>>>> introducing
>>>>>>>>>>>>> this potential recursion loop, I'm open to changes!
>>>>>>>>>>>>>
>>>>>>>>>>>>> C. This is described in the "Atomic Checkpointing" section.
>>>> Offsets
>>>>>>>>> are
>>>>>>>>>>>>> stored in a separate RocksDB column family, which is guaranteed
>>>> to
>>>>>> be
>>>>>>>>>>>>> atomically flushed to disk with all other column families. The
>>>>>> issue
>>>>>>>>> of
>>>>>>>>>>>>> checkpoints being written to disk after commit causing
>>>>>> inconsistency
>>>>>>>>> if
>>>>>>>>>>>> it
>>>>>>>>>>>>> crashes in between is the reason why, under EOS, checkpoint
>> files
>>>>>> are
>>>>>>>>>>>> only
>>>>>>>>>>>>> written on clean shutdown. This is one of the major causes of
>>>> "full
>>>>>>>>>>>>> restorations", so moving the offsets into a place where they
>> can
>>>> be
>>>>>>>>>>>>> guaranteed to be atomically written with the data they
>> checkpoint
>>>>>>>>>>>> allows us
>>>>>>>>>>>>> to write the checkpoint offsets *on every commit*, not just on
>>>>>> clean
>>>>>>>>>>>>> shutdown.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <
>> colt@littlehorse.io>
>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thank you for continuing this work. I have a few minor
>>>> clarifying
>>>>>>>>>>>>>> questions.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> A) "Records written to any transaction are visible to all
>> other
>>>>>>>>>>>>>> transactions immediately." I am confused here—I thought there
>>>>>> could
>>>>>>>>>>>> only
>>>>>>>>>>>>> be
>>>>>>>>>>>>>> one transaction going on at a time for a given state store
>> given
>>>>>> the
>>>>>>>>>>>>>> threading model for processing records on a Task. Do you mean
>>>>>>>>>>>> Interactive
>>>>>>>>>>>>>> Queries by "other transactions"? (If so, then everything makes
>>>>>>>>> sense—I
>>>>>>>>>>>>>> thought that since IQ were read-only then they didn't count as
>>>>>>>>>>>>>> transactions).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> B) Is it intentional that the default implementations of the
>>>>>> flush()
>>>>>>>>>>>> and
>>>>>>>>>>>>>> commit() methods in the StateStore class refer to each other
>> in
>>>>>> some
>>>>>>>>>>>> sort
>>>>>>>>>>>>>> of unbounded recursion?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> C) How will the getCommittedOffset() method work? At first I
>>>>>> thought
>>>>>>>>>>>> the
>>>>>>>>>>>>>> way to do it would be using a special key in the RocksDB store
>>>> to
>>>>>>>>>>>> store
>>>>>>>>>>>>> the
>>>>>>>>>>>>>> offset, and committing that with the transaction. But upon
>>>> second
>>>>>>>>>>>>> thought,
>>>>>>>>>>>>>> since restoration from the changelog is an idempotent
>>>> procedure, I
>>>>>>>>>>>> think
>>>>>>>>>>>>> it
>>>>>>>>>>>>>> would be fine to 1) commit the RocksDB transaction and then 2)
>>>>>> write
>>>>>>>>>>>> the
>>>>>>>>>>>>>> offset to disk in a checkpoint file. If there is a crash
>> between
>>>>>> 1)
>>>>>>>>>>>> and
>>>>>>>>>>>>> 2),
>>>>>>>>>>>>>> I think the only downside is 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 <
>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I've updated the KIP to reflect the latest version of the
>>>> design:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> There are several changes in there that reflect feedback from
>>>>>> this
>>>>>>>>>>>>>> thread,
>>>>>>>>>>>>>>> and there's a new section and a bunch of interface changes
>>>>>> relating
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> Atomic Checkpointing, which is the final piece of the puzzle
>> to
>>>>>>>>>>>> making
>>>>>>>>>>>>>>> everything robust.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Let me know what you think!
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <
>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi Lucas,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks for looking over my KIP.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> A) The bound is per-instance, not per-Task. This was a typo
>> in
>>>>>> the
>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>> that I've now corrected. It was originally per-Task, but I
>>>>>>>>>>>> changed it
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> per-instance for exactly the reason you highlighted.
>>>>>>>>>>>>>>>> B) It's worth noting that transactionality is only enabled
>>>> under
>>>>>>>>>>>> EOS,
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> in the default mode of operation (ALOS), there should be no
>>>>>>>>>>>> change in
>>>>>>>>>>>>>>>> behavior at all. I think, under EOS, we can mitigate the
>>>> impact
>>>>>> on
>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>> sufficiently low default values for the memory bound
>>>>>>>>>>>> configuration. I
>>>>>>>>>>>>>>>> understand your hesitation to include a significant change
>> of
>>>>>>>>>>>>>> behaviour,
>>>>>>>>>>>>>>>> especially in a minor release, but I suspect that most users
>>>>>> will
>>>>>>>>>>>>>> prefer
>>>>>>>>>>>>>>>> the memory impact (under EOS) to the existing behaviour of
>>>>>>>>>>>> frequent
>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>> restorations! If this is a problem, the changes can wait
>> until
>>>>>> the
>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>> major release. I'll be running a patched version of streams
>> in
>>>>>>>>>>>>>> production
>>>>>>>>>>>>>>>> with these changes as soon as they're ready, so it won't
>>>> disrupt
>>>>>>>>>>>> me
>>>>>>>>>>>>> :-D
>>>>>>>>>>>>>>>> C) The main purpose of this sentence was just to note that
>>>> some
>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>>> will need to be made to the way Segments are handled in
>> order
>>>> to
>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>> they also benefit from transactions. At the time I wrote
>> it, I
>>>>>>>>>>>> hadn't
>>>>>>>>>>>>>>>> figured out the specific changes necessary, so it was
>>>>>> deliberately
>>>>>>>>>>>>>> vague.
>>>>>>>>>>>>>>>> This is the one outstanding problem I'm currently working
>> on,
>>>>>> and
>>>>>>>>>>>>> I'll
>>>>>>>>>>>>>>>> update this section with more detail once I have figured out
>>>> the
>>>>>>>>>>>>> exact
>>>>>>>>>>>>>>>> changes required.
>>>>>>>>>>>>>>>> D) newTransaction() provides the necessary isolation
>>>> guarantees.
>>>>>>>>>>>>> While
>>>>>>>>>>>>>>>> the RocksDB implementation of transactions doesn't
>> technically
>>>>>>>>>>>> *need*
>>>>>>>>>>>>>>>> read-only users to call newTransaction(), other
>>>> implementations
>>>>>>>>>>>>> (e.g. a
>>>>>>>>>>>>>>>> hypothetical PostgresStore) may require it. Calling
>>>>>>>>>>>> newTransaction()
>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>> no transaction is necessary is essentially free, as it will
>>>> just
>>>>>>>>>>>>> return
>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think it
>>>>>>>>>>>> should
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>> fairly obvious where the performance problems stem from:
>>>> writes
>>>>>>>>>>>> under
>>>>>>>>>>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with
>> the
>>>>>>>>>>>>>>>> tombstone/record flag, 1 to decode it from the
>>>> tombstone/record
>>>>>>>>>>>> flag,
>>>>>>>>>>>>>>> and 1
>>>>>>>>>>>>>>>> to copy the record from the "temporary" store to the "main"
>>>>>> store,
>>>>>>>>>>>>> when
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> transaction commits. The different approach taken by KIP-869
>>>>>>>>>>>> should
>>>>>>>>>>>>>>> perform
>>>>>>>>>>>>>>>> much better, as it avoids all these copies, and may actually
>>>>>>>>>>>> perform
>>>>>>>>>>>>>>>> slightly better than trunk, due to batched writes in RocksDB
>>>>>>>>>>>>> performing
>>>>>>>>>>>>>>>> better than non-batched writes.[1]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 1:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>
>>>>>>
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
>>>>>>>>>>>> lbrutschy@confluent.io
>>>>>>>>>>>>>>> .invalid>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I'm just starting to read up on the whole discussion about
>>>>>>>>>>>> KIP-892
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
>>>>>>>>>>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do have
>>>> some
>>>>>>>>>>>>>>>>> questions about the latest draft.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>       A) If I understand correctly, you propose to put a
>> bound
>>>> on
>>>>>> the
>>>>>>>>>>>>>>>>> (native) memory consumed by each task. However, I wonder if
>>>>>> this
>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> sufficient if we have temporary imbalances in the cluster.
>>>> For
>>>>>>>>>>>>>>>>> example, depending on the timing of rebalances during a
>>>> cluster
>>>>>>>>>>>>>>>>> restart, it could happen that a single streams node is
>>>>>> assigned a
>>>>>>>>>>>>> lot
>>>>>>>>>>>>>>>>> more tasks than expected. With your proposed change, this
>>>> would
>>>>>>>>>>>> mean
>>>>>>>>>>>>>>>>> that the memory required by this one node could be a
>> multiple
>>>>>> of
>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>> is required during normal operation. I wonder if it
>> wouldn't
>>>> be
>>>>>>>>>>>>> safer
>>>>>>>>>>>>>>>>> to put a global bound on the memory use, across all tasks.
>>>>>>>>>>>>>>>>>       B) Generally, the memory concerns still give me the
>>>> feeling
>>>>>>>>> that
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> should not be enabled by default for all users in a minor
>>>>>>>>>>>> release.
>>>>>>>>>>>>>>>>>       C) In section "Transaction Management": the sentence
>> "A
>>>>>> similar
>>>>>>>>>>>>>>>>> analogue will be created to automatically manage `Segment`
>>>>>>>>>>>>>>>>> transactions.". Maybe this is just me lacking some
>>>> background,
>>>>>>>>>>>> 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 you to find out why the KIP-844 PoC is 20x
>>>> slower
>>>>>>>>>>>> (e.g.
>>>>>>>>>>>>>>>>> by providing a flame graph), that would be quite
>> interesting.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>>>> Lucas
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I've updated the KIP with a more detailed design, which
>>>>>>>>>>>> reflects
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> implementation I've been working on:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>
>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> This new design should address the outstanding points
>>>> already
>>>>>>>>>>>> made
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Please let me know if there are areas that are unclear or
>>>> need
>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>>> clarification.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I have a (nearly) working implementation. I'm confident
>> that
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> remaining
>>>>>>>>>>>>>>>>>> work (making Segments behave) will not impact the
>> documented
>>>>>>>>>>>>> design.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <
>>>>>> colt@littlehorse.io
>>>>>>>>>>>>>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thank you for the reply; that makes sense. I was hoping
>>>> that,
>>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>>> reading
>>>>>>>>>>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
>>>>>>>>>>>> documented
>>>>>>>>>>>>>> API,
>>>>>>>>>>>>>>>>> maybe
>>>>>>>>>>>>>>>>>>> you *wouldn't* have to wait for the next major release to
>>>>>>>>>>>> make
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> change;
>>>>>>>>>>>>>>>>>>> but given 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 <
>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended
>> to
>>>>>>>>>>>> try
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> keep the
>>>>>>>>>>>>>>>>>>>> behaviour unchanged as much as possible, otherwise we'd
>>>>>>>>>>>> have
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> wait for
>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> major version release to land these changes.
>>>>>>>>>>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
>>>>>>>>>>>> guarantee,
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> typically longer commit intervals would be problematic
>>>> when
>>>>>>>>>>>>>>> reading
>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>> "committed" records.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I've been away for 5 days recovering from minor surgery,
>>>>>>>>>>>> but I
>>>>>>>>>>>>>>>>> spent a
>>>>>>>>>>>>>>>>>>>> considerable amount of that time working through ideas
>> for
>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>>> solutions in my head. I think your suggestion of keeping
>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>> as-is, but
>>>>>>>>>>>>>>>>>>>> buffering writes for EOS is the right path forwards,
>>>>>>>>>>>> although
>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>>>> solution that both expands on this, and provides for
>> some
>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>> formal
>>>>>>>>>>>>>>>>>>>> guarantees.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Essentially, adding support to KeyValueStores for
>>>>>>>>>>>>>> "Transactions",
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>> clearly defined IsolationLevels. Using "Read Committed"
>>>>>>>>>>>> when
>>>>>>>>>>>>>> under
>>>>>>>>>>>>>>>>> EOS,
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> "Read Uncommitted" under ALOS.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The nice thing about this approach is that it gives us
>>>> much
>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>> clearly
>>>>>>>>>>>>>>>>>>>> defined isolation behaviour that can be properly
>>>>>>>>>>>> documented to
>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>>>>> know what to expect.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I'm still working out the kinks in the design, and will
>>>>>>>>>>>> update
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>>>> I have something. The main struggle is trying to
>> implement
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>>>> making any major changes to the existing interfaces or
>>>>>>>>>>>>> breaking
>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>> implementations, because currently everything expects to
>>>>>>>>>>>>> operate
>>>>>>>>>>>>>>>>> directly
>>>>>>>>>>>>>>>>>>>> on a StateStore, and not a Transaction of that store. I
>>>>>>>>>>>> think
>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>> close, although sadly I won't be able to progress much
>>>>>>>>>>>> until
>>>>>>>>>>>>>> next
>>>>>>>>>>>>>>>>> week
>>>>>>>>>>>>>>>>>>> due
>>>>>>>>>>>>>>>>>>>> to some work commitments.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
>>>>>>>>>>>>> colt@littlehorse.io>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thank you for the explanation, and also for the updated
>>>>>>>>>>>>> KIP. I
>>>>>>>>>>>>>>> am
>>>>>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>>>>>> eager for this improvement to be released as it would
>>>>>>>>>>>>> greatly
>>>>>>>>>>>>>>>>> reduce
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> operational difficulties of EOS streams apps.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Two questions:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 10)
>>>>>>>>>>>>>>>>>>>>>> When reading records, we will use the
>>>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
>>>>>>>>>>>>>>>>>>>>>       and WriteBatchWithIndex#newIteratorWithBase
>>>> utilities in
>>>>>>>>>>>>>> order
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>>> that uncommitted writes are available to query.
>>>>>>>>>>>>>>>>>>>>> Why do extra work to enable the reading of uncommitted
>>>>>>>>>>>>> writes
>>>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>>>> IQ?
>>>>>>>>>>>>>>>>>>>>> Code complexity aside, reading uncommitted writes is,
>> in
>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>> opinion, a
>>>>>>>>>>>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have the
>>>>>>>>>>>>>>> guarantee
>>>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>>>>>> with EOS, IQ only reads committed records. In order to
>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>> dirty
>>>>>>>>>>>>>>>>>>> reads,
>>>>>>>>>>>>>>>>>>>>> one currently must query a standby replica (but this
>>>>>>>>>>>> still
>>>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>>>> fully
>>>>>>>>>>>>>>>>>>>>> guarantee monotonic reads).
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 20) Is it also necessary to enable this optimization on
>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>> stores?
>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need to
>>>>>>>>>>>>> restore
>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
>>>>>>>>>>>>> acceptable
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> accept
>>>>>>>>>>>>>>>>>>>>> that there may have been uncommitted writes on disk.
>> On a
>>>>>>>>>>>>> side
>>>>>>>>>>>>>>>>> note, if
>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>> enable this type of store on ALOS processors, the
>>>>>>>>>>>> community
>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
>>>>>>>>>>>>> longer
>>>>>>>>>>>>>>>>> include
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> StateStore management of checkpointing. This can be
>>>>>>>>>>>> added
>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>>>> to further optimize the consistency and performance of
>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>> stores.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I've also added a section discussing some of the
>>>>>>>>>>>> concerns
>>>>>>>>>>>>>>> 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 2022 at 16:36, Nick Telford <
>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I didn't do any profiling, but the 844
>>>>>>>>>>>> implementation:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>         - Writes uncommitted records to a temporary
>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>>>> instance
>>>>>>>>>>>>>>>>>>>>>>>            - Since tombstones need to be flagged, all
>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>> values are
>>>>>>>>>>>>>>>>>>>>>>>            prefixed with a value/tombstone marker.
>> This
>>>>>>>>>>>>>>>>> necessitates a
>>>>>>>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>>>>>> copy.
>>>>>>>>>>>>>>>>>>>>>>>         - On-commit, iterates all records in this
>>>>>>>>>>>> temporary
>>>>>>>>>>>>>>>>> instance and
>>>>>>>>>>>>>>>>>>>>>>>         writes them to the main RocksDB store.
>>>>>>>>>>>>>>>>>>>>>>>         - While iterating, the value/tombstone marker
>>>>>>>>>>>> needs
>>>>>>>>>>>>> to
>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>> parsed
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>         the real value extracted. This necessitates
>>>>>>>>>>>> another
>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>> copy.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> My guess is that the cost of iterating the temporary
>>>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> major factor, with the 2 extra memory copies
>>>>>>>>>>>> per-Record
>>>>>>>>>>>>>>>>>>> contributing
>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>> significant amount too.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
>>>>>>>>>>>>>>>>> colt@littlehorse.io>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Out of curiosity, why does the performance of the
>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>> degrade so
>>>>>>>>>>>>>>>>>>>>>>>> significantly with the 844 implementation? I
>>>>>>>>>>>> wouldn't
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>> too
>>>>>>>>>>>>>>>>>>>> surprised
>>>>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> 50-60% drop (caused by each record being written
>>>>>>>>>>>>> twice),
>>>>>>>>>>>>>>> but
>>>>>>>>>>>>>>>>> 96%
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>> extreme.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> The only thing I can think of which could create
>>>>>>>>>>>> such a
>>>>>>>>>>>>>>>>> bottleneck
>>>>>>>>>>>>>>>>>>>>> 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 Nick Telford <
>>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP to resolve all the points
>>>>>>>>>>>> that
>>>>>>>>>>>>>> have
>>>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>>>> raised
>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>> far, with one exception: the ALOS default commit
>>>>>>>>>>>>>> interval
>>>>>>>>>>>>>>>>> of 5
>>>>>>>>>>>>>>>>>>>>> minutes
>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>> likely to cause WriteBatchWithIndex memory to grow
>>>>>>>>>>>>> too
>>>>>>>>>>>>>>>>> large.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> There's a couple of different things I can think
>>>>>>>>>>>> of
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> solve
>>>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>         - We already have a memory/record limit in
>> the
>>>>>>>>>>>> KIP
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> prevent
>>>>>>>>>>>>>>>>>>>> OOM
>>>>>>>>>>>>>>>>>>>>>>>>>         errors. Should we choose a default value for
>>>>>>>>>>>>> these?
>>>>>>>>>>>>>> My
>>>>>>>>>>>>>>>>>>> concern
>>>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>         anything we choose might seem rather
>>>>>>>>>>>> arbitrary. We
>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>>>>>>>>>         its behaviour such that under ALOS, it only
>>>>>>>>>>>>> triggers
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>         StateStore, but under EOS, it triggers a
>>>>>>>>>>>> commit of
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>>>>>>>         - We could introduce a separate `
>>>>>>>>>>>>>>> checkpoint.interval.ms`
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>>>>>>>>>>         to commit the StateStores more frequently
>> than
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> general
>>>>>>>>>>>>>>>>>>>>>>>>>         commit.interval.ms? My concern here is that
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> semantics of
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>> config
>>>>
> 

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

Posted by Nick Telford <ni...@gmail.com>.
> Note that users can disable the cache, which would still be
ok, I think. We wouldn't ingest the SST files on every record, but just
append to them and only ingest them on commit, when we're already
waiting for acks and a RocksDB commit.

In this case, how would uncommitted records be read by joins?

On Tue, 20 Jun 2023, 20:51 John Roesler, <vv...@apache.org> wrote:

> Ah, sorry Nick,
>
> I just meant the regular heap based cache that we maintain in Streams. I
> see that it's not called "RecordCache" (my mistake).
>
> The actual cache is ThreadCache:
>
> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java
>
> Here's the example of how we use the cache in KeyValueStore:
>
> https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java
>
> It's basically just an on-heap Map of records that have not yet been
> written to the changelog or flushed into the underlying store. It gets
> flushed when the total cache size exceeds `cache.max.bytes.buffering` or
> the `commit.interval.ms` elapses.
>
> Speaking of those configs, another benefit to this idea is that we would
> no longer need to trigger extra commits based on the size of the ongoing
> transaction. Instead, we'd just preserve the existing cache-flush
> behavior. Note that users can disable the cache, which would still be
> ok, I think. We wouldn't ingest the SST files on every record, but just
> append to them and only ingest them on commit, when we're already
> waiting for acks and a RocksDB commit.
>
> Thanks,
> -John
>
> On 6/20/23 14:09, Nick Telford wrote:
> > Hi John,
> >
> > By "RecordCache", do you mean the RocksDB "WriteBatch"? I can't find any
> > class called "RecordCache"...
> >
> > Cheers,
> >
> > Nick
> >
> > On Tue, 20 Jun 2023 at 19:42, John Roesler <vv...@apache.org> wrote:
> >
> >> Hi Nick,
> >>
> >> Thanks for picking this up again!
> >>
> >> I did have one new thought over the intervening months, which I'd like
> >> your take on.
> >>
> >> What if, instead of using the RocksDB atomic write primitive at all, we
> >> instead just:
> >> 1. disable memtables entirely
> >> 2. directly write the RecordCache into SST files when we flush
> >> 3. atomically ingest the SST file(s) into RocksDB when we get the ACK
> >> from the changelog (see
> >>
> >>
> https://github.com/EighteenZi/rocksdb_wiki/blob/master/Creating-and-Ingesting-SST-files.md
> >> and
> >>
> >>
> https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/IngestExternalFileOptions.java
> >> and
> >>
> >>
> https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L1413-L1429
> >> )
> >> 4. track the changelog offsets either in another CF or the same CF with
> >> a reserved key, either of which will make the changelog offset update
> >> atomic with the file ingestions
> >>
> >> I suspect this'll have a number of benefits:
> >> * writes to RocksDB will always be atomic
> >> * we don't fragment memory between the RecordCache and the memtables
> >> * RecordCache gives far higher performance than memtable for reads and
> >> writes
> >> * we don't need any new "transaction" concepts or memory bound configs
> >>
> >> What do you think?
> >>
> >> Thanks,
> >> -John
> >>
> >> On 6/20/23 10:51, Nick Telford wrote:
> >>> Hi Bruno,
> >>>
> >>> Thanks for reviewing the KIP. It's been a long road, I started working
> on
> >>> this more than a year ago, and most of the time in the last 6 months
> has
> >>> been spent on the "Atomic Checkpointing" stuff that's been benched, so
> >> some
> >>> of the reasoning behind some of my decisions have been lost, but I'll
> do
> >> my
> >>> best to reconstruct them.
> >>>
> >>> 1.
> >>> IIRC, this was the initial approach I tried. I don't remember the exact
> >>> reasons I changed it to use a separate "view" of the StateStore that
> >>> encapsulates the transaction, but I believe it had something to do with
> >>> concurrent access to the StateStore from Interactive Query threads.
> Reads
> >>> from interactive queries need to be isolated from the currently ongoing
> >>> transaction, both for consistency (so interactive queries don't observe
> >>> changes that are subsequently rolled-back), but also to prevent
> Iterators
> >>> opened by an interactive query from being closed and invalidated by the
> >>> StreamThread when it commits the transaction, which causes your
> >> interactive
> >>> queries to crash.
> >>>
> >>> Another reason I believe I implemented it this way was a separation of
> >>> concerns. Recall that newTransaction() originally created an object of
> >> type
> >>> Transaction, not StateStore. My intent was to improve the type-safety
> of
> >>> the API, in an effort to ensure Transactions weren't used incorrectly.
> >>> Unfortunately, this didn't pan out, but newTransaction() remained.
> >>>
> >>> Finally, this had the added benefit that implementations could easily
> add
> >>> support for transactions *without* re-writing their existing,
> >>> non-transactional implementation. I think this can be a benefit both
> for
> >>> implementers of custom StateStores, but also for anyone extending
> >>> RocksDbStore, as they can rely on the existing access methods working
> how
> >>> they expect them to.
> >>>
> >>> I'm not too happy with the way the current design has panned out, so
> I'm
> >>> open to ideas on how to improve it. Key to this is finding some way to
> >>> ensure that reads from Interactive Query threads are properly isolated
> >> from
> >>> the transaction, *without* the performance overhead of checking which
> >>> thread the method is being called from on every access.
> >>>
> >>> As for replacing flush() with commit() - I saw no reason to add this
> >>> complexity to the KIP, unless there was a need to add arguments to the
> >>> flush/commit method. This need arises with Atomic Checkpointing, but
> that
> >>> will be implemented separately, in a future KIP. Do you see a need for
> >> some
> >>> arguments to the flush/commit method that I've missed? Or were you
> simply
> >>> suggesting a rename?
> >>>
> >>> 2.
> >>> This is simply due to the practical reason that isolationLevel() is
> >> really
> >>> a proxy for checking if the app is under EOS. The application
> >> configuration
> >>> is not provided to the constructor of StateStores, but it *is* provided
> >> to
> >>> init(), via StateStoreContext. For this reason, it seemed somewhat
> >> natural
> >>> to add it to StateStoreContext. I think this makes sense, since the
> >>> IsolationLevel of all StateStores in an application *must* be the same,
> >> and
> >>> since those stores are all initialized with the same StateStoreContext,
> >> it
> >>> seems natural for that context to carry the desired IsolationLevel to
> >> use.
> >>>
> >>> 3.
> >>> Using IsolationLevel instead of just passing `boolean eosEnabled`, like
> >>> much of the internals was an attempt to logically de-couple the
> >> StateStore
> >>> API from the internals of Kafka Streams. Technically, StateStores don't
> >>> need to know/care what processing mode the KS app is using, all they
> need
> >>> to know is the isolation level expected of them.
> >>>
> >>> Having formal definitions for the expectations of the two required
> >>> IsolationLevels allow implementers to implement transactional stores
> >>> without having to dig through the internals of Kafka Streams and
> >> understand
> >>> exactly how they are used. The tight coupling between state stores and
> >>> internal behaviour has actually significantly hindered my progress on
> >> this
> >>> KIP, and encouraged me to avoid increasing this logical coupling as
> much
> >> as
> >>> possible.
> >>>
> >>> This also frees implementations to satisfy those requirements in any
> way
> >>> they choose. Transactions might not be the only/available approach to
> an
> >>> implementation, but they might have an alternative way to satisfy the
> >>> isolation requirements. I admit that this point is more about
> semantics,
> >>> but "transactional" would need to be formally defined in order for
> >>> implementers to provide a valid implementation, and these
> IsolationLevels
> >>> provide that formal definition.
> >>>
> >>> 4.
> >>> I can remove them. I added them only as I planned to include them in
> the
> >>> org.apache.kafka.streams.state package, as a recommended base
> >>> implementation for all StateStores, including those implemented by
> >> users. I
> >>> had assumed that anything in "public" packages, such as
> >>> org.apache.kafka.streams.state, should be included in a KIP. Is that
> >> wrong?
> >>>
> >>> 5.
> >>> RocksDB provides no way to measure the actual size of a
> >>> WriteBatch(WithIndex), so we're limited to tracking the sum total of
> the
> >>> size of keys + values that are written to the transaction. This
> obviously
> >>> under-estimates the actual memory usage, because WriteBatch no-doubt
> >>> includes some record overheads, and WriteBatchWithIndex has to maintain
> >> an
> >>> index.
> >>>
> >>> Ideally, we could trivially add a method upstream to
> WriteBatchInterface
> >>> that provides the exact size of the batch, but that would require an
> >>> upgrade of RocksDB, which won't happen soon. So for the time being,
> we're
> >>> stuck with an approximation, so I felt that the new method should
> reflect
> >>> that.
> >>>
> >>> Would you prefer the new method name ignores this constraint and that
> we
> >>> simply make the rocks measurement more accurate in the future?
> >>>
> >>> 6.
> >>> Done
> >>>
> >>> 7.
> >>> Very good point. The KIP already specifically calls out memory in the
> >>> documentation of the config: "Maximum number of memory bytes to be used
> >> to
> >>> buffer uncommitted state-store records." - did you have something else
> in
> >>> mind?
> >>>
> >>> Should we also make this clearer by renaming the config property
> itself?
> >>> Perhaps to something like statestore.transaction.buffer.max.bytes?
> >>>
> >>> 8.
> >>> OK, I can remove this. The intent here was to describe how Streams
> itself
> >>> will manage transaction roll-over etc. Presumably that means we also
> >> don't
> >>> need a description of how Streams will manage the commit of changelog
> >>> transactions, state store transactions and checkpointing?
> >>>
> >>> 9.
> >>> What do you mean by fail-over? Do you mean failing over an Active Task
> to
> >>> an instance already hosting a Standby Task?
> >>>
> >>> Thanks again and sorry for the essay of a response!
> >>>
> >>> Regards,
> >>> Nick
> >>>
> >>> On Tue, 20 Jun 2023 at 10:49, Bruno Cadonna <ca...@apache.org>
> wrote:
> >>>
> >>>> Hi Nick,
> >>>>
> >>>> Thanks for the updates!
> >>>>
> >>>> I really appreciate that you simplified the KIP by removing some
> >>>> aspects. As I have already told you, I think the removed aspects are
> >>>> also good ideas and we can discuss them on follow-up KIPs.
> >>>>
> >>>> Regarding the current KIP, I have the following feedback.
> >>>>
> >>>> 1.
> >>>> Is there a good reason to add method newTransaction() to the
> StateStore
> >>>> interface? As far as I understand, the idea is that users of a state
> >>>> store (transactional or not) call this method at start-up and after
> each
> >>>> commit. Since the call to newTransaction() is done in any case and I
> >>>> think it would simplify the caller code if we just start a new
> >>>> transaction after a commit in the implementation?
> >>>> As far as I understand, you plan to commit the transaction in the
> >>>> flush() method. I find the idea to replace flush() with commit()
> >>>> presented in KIP-844 an elegant solution.
> >>>>
> >>>> 2.
> >>>> Why is the method to query the isolation level added to the state
> store
> >>>> context?
> >>>>
> >>>> 3.
> >>>> Do we need all the isolation level definitions? I think it is good to
> >>>> know the guarantees of the transactionality of the state store.
> >>>> However, currently, Streams guarantees that there will only be one
> >>>> transaction that writes to the state store. Only the stream thread
> that
> >>>> executes the active task that owns the state store will write to the
> >>>> state store. I think it should be enough to know if the state store is
> >>>> transactional or not. So my proposal would be to just add a method on
> >>>> the state store interface the returns if a state store is
> transactional
> >>>> or not by returning a boolean or an enum.
> >>>>
> >>>> 4.
> >>>> I am wondering why AbstractTransaction and AbstractTransactionalStore
> >>>> are part of the KIP. They look like implementation details that should
> >>>> not be exposed in the public API.
> >>>>
> >>>> 5.
> >>>> Why does StateStore#approximateNumUncommittedBytes() return an
> >>>> approximate number of bytes?
> >>>>
> >>>> 6.
> >>>> RocksDB is just one implementation of the state stores in Streams.
> >>>> However, the issues regarding OOM errors might also apply to other
> >>>> custom implementations. So in the KIP I would extract that part from
> >>>> section "RocksDB Transaction". I would also move section "RocksDB
> >>>> Transaction" to the end of section "Proposed Changes" and handle it as
> >>>> an example implementation for a state store.
> >>>>
> >>>> 7.
> >>>> Should statestore.uncommitted.max.bytes only limit the uncommitted
> bytes
> >>>> or the uncommitted bytes that reside in memory? In future, other
> >>>> transactional state store implementations might implement a buffer for
> >>>> uncommitted records that are able to spill records on disk. I think
> >>>> statestore.uncommitted.max.bytes needs to limit the uncommitted bytes
> >>>> irrespective if they reside in memory or disk. Since Streams will use
> >>>> this config to decide if it needs to trigger a commit, state store
> >>>> implementations that can spill to disk will never be able to spill to
> >>>> disk. You would only need to change the doc of the config, if you
> agree
> >>>> with me.
> >>>>
> >>>> 8.
> >>>> Section "Transaction Management" about the wrappers is rather a
> >>>> implementation detail that should not be in the KIP.
> >>>>
> >>>> 9.
> >>>> Could you add a section that describes how failover will work with the
> >>>> transactional state stores? I think section "Error handling" is
> already
> >>>> a good start.
> >>>>
> >>>>
> >>>> Best,
> >>>> Bruno
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> On 15.05.23 11:04, Nick Telford wrote:
> >>>>> Hi everyone,
> >>>>>
> >>>>> Quick update: I've added a new section to the KIP: "Offsets for
> >> Consumer
> >>>>> Rebalances", that outlines my solution to the problem that
> >>>>> StreamsPartitionAssignor needs to read StateStore offsets even if
> >> they're
> >>>>> not currently open.
> >>>>>
> >>>>> Regards,
> >>>>> Nick
> >>>>>
> >>>>> On Wed, 3 May 2023 at 11:34, Nick Telford <ni...@gmail.com>
> >>>> wrote:
> >>>>>
> >>>>>> Hi Bruno,
> >>>>>>
> >>>>>> Thanks for reviewing my proposal.
> >>>>>>
> >>>>>> 1.
> >>>>>> The main reason I added it was because it was easy to do. If we see
> no
> >>>>>> value in it, I can remove it.
> >>>>>>
> >>>>>> 2.
> >>>>>> Global StateStores can have multiple partitions in their input
> topics
> >>>>>> (which function as their changelogs), so they would have more than
> one
> >>>>>> partition.
> >>>>>>
> >>>>>> 3.
> >>>>>> That's a good point. At present, the only method it adds is
> >>>>>> isolationLevel(), which is likely not necessary outside of
> >> StateStores.
> >>>>>> It *does* provide slightly different guarantees in the documentation
> >> to
> >>>>>> several of the methods (hence the overrides). I'm not sure if this
> is
> >>>>>> enough to warrant a new interface though.
> >>>>>> I think the question that remains is whether this interface makes it
> >>>>>> easier to implement custom transactional StateStores than if we were
> >> to
> >>>>>> remove it? Probably not.
> >>>>>>
> >>>>>> 4.
> >>>>>> The main motivation for the Atomic Checkpointing is actually
> >>>> performance.
> >>>>>> My team has been testing out an implementation of this KIP without
> it,
> >>>> and
> >>>>>> we had problems with RocksDB doing *much* more compaction, due to
> the
> >>>>>> significantly increased flush rate. It was enough of a problem that
> >> (for
> >>>>>> the time being), we had to revert back to Kafka Streams proper.
> >>>>>> I think the best way to solve this, as you say, is to keep the
> >>>> .checkpoint
> >>>>>> files *in addition* to the offsets being stored within the store
> >> itself.
> >>>>>> Essentially, when closing StateStores, we force a memtable flush,
> then
> >>>>>> call getCommittedOffsets and write those out to the .checkpoint
> file.
> >>>>>> That would ensure the metadata is available to the
> >>>>>> StreamsPartitionAssignor for all closed stores.
> >>>>>> If there's a crash (no clean close), then we won't be able to
> >> guarantee
> >>>>>> which offsets were flushed to disk by RocksDB, so we'd need to open
> (
> >>>>>> init()), read offsets, and then close() those stores. But since this
> >> is
> >>>>>> the exception, and will only occur once (provided it doesn't crash
> >> every
> >>>>>> time!), I think the performance impact here would be acceptable.
> >>>>>>
> >>>>>> Thanks for the feedback, please let me know if you have any more
> >>>> comments
> >>>>>> or questions!
> >>>>>>
> >>>>>> I'm currently working on rebasing against trunk. This involves
> adding
> >>>>>> support for transactionality to VersionedStateStores. I will
> probably
> >>>> need
> >>>>>> to revise my implementation for transactional "segmented" stores,
> both
> >>>> to
> >>>>>> accommodate VersionedStateStore, and to clean up some other stuff.
> >>>>>>
> >>>>>> Regards,
> >>>>>> Nick
> >>>>>>
> >>>>>>
> >>>>>> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org>
> >> wrote:
> >>>>>>
> >>>>>>> Hi Nick,
> >>>>>>>
> >>>>>>> Thanks for the updates!
> >>>>>>>
> >>>>>>> I have a couple of questions/comments.
> >>>>>>>
> >>>>>>> 1.
> >>>>>>> Why do you propose a configuration that involves max. bytes and
> max.
> >>>>>>> reords? I think we are mainly concerned about memory consumption
> >>>> because
> >>>>>>> we want to limit the off-heap memory used. I cannot think of a case
> >>>>>>> where one would want to set the max. number of records.
> >>>>>>>
> >>>>>>>
> >>>>>>> 2.
> >>>>>>> Why does
> >>>>>>>
> >>>>>>>      default void commit(final Map<TopicPartition, Long>
> >>>> changelogOffsets) {
> >>>>>>>          flush();
> >>>>>>>      }
> >>>>>>>
> >>>>>>> take a map of partitions to changelog offsets?
> >>>>>>> The mapping between state stores to partitions is a 1:1
> relationship.
> >>>>>>> Passing in a single changelog offset should suffice.
> >>>>>>>
> >>>>>>>
> >>>>>>> 3.
> >>>>>>> Why do we need the Transaction interface? It should be possible to
> >> hide
> >>>>>>> beginning and committing a transactions withing the state store
> >>>>>>> implementation, so that from outside the state store, it does not
> >>>> matter
> >>>>>>> whether the state store is transactional or not. What would be the
> >>>>>>> advantage of using the Transaction interface?
> >>>>>>>
> >>>>>>>
> >>>>>>> 4.
> >>>>>>> Regarding checkpointing offsets, I think we should keep the
> >> checkpoint
> >>>>>>> file in any case for the reason you mentioned about rebalancing.
> Even
> >>>> if
> >>>>>>> that would not be an issue, I would propose to move the change to
> >>>> offset
> >>>>>>> management to a new KIP and to not add more complexity than needed
> to
> >>>>>>> this one. I would not be too concerned about the consistency
> >> violation
> >>>>>>> you mention. As far as I understand, with transactional state
> stores
> >>>>>>> Streams would write the checkpoint file during every commit even
> >> under
> >>>>>>> EOS. In the failure case you describe, Streams would restore the
> >> state
> >>>>>>> stores from the offsets found in the checkpoint file written during
> >> the
> >>>>>>> penultimate commit instead of during the last commit. Basically,
> >>>> Streams
> >>>>>>> would overwrite the records written to the state store between the
> >> last
> >>>>>>> two commits with the same records read from the changelogs. While I
> >>>>>>> understand that this is wasteful, it is -- at the same time --
> >>>>>>> acceptable and most importantly it does not break EOS.
> >>>>>>>
> >>>>>>> Best,
> >>>>>>> Bruno
> >>>>>>>
> >>>>>>>
> >>>>>>> On 27.04.23 12:34, Nick Telford wrote:
> >>>>>>>> Hi everyone,
> >>>>>>>>
> >>>>>>>> I find myself (again) considering removing the offset management
> >> from
> >>>>>>>> StateStores, and keeping the old checkpoint file system. The
> reason
> >> is
> >>>>>>> that
> >>>>>>>> the StreamPartitionAssignor directly reads checkpoint files in
> order
> >>>> to
> >>>>>>>> determine which instance has the most up-to-date copy of the local
> >>>>>>> state.
> >>>>>>>> If we move offsets into the StateStore itself, then we will need
> to
> >>>>>>> open,
> >>>>>>>> initialize, read offsets and then close each StateStore (that is
> not
> >>>>>>>> already assigned and open) for which we have *any* local state, on
> >>>> every
> >>>>>>>> rebalance.
> >>>>>>>>
> >>>>>>>> Generally, I don't think there are many "orphan" stores like this
> >>>>>>> sitting
> >>>>>>>> around on most instances, but even a few would introduce
> additional
> >>>>>>> latency
> >>>>>>>> to an already somewhat lengthy rebalance procedure.
> >>>>>>>>
> >>>>>>>> I'm leaning towards Colt's (Slack) suggestion of just keeping
> things
> >>>> in
> >>>>>>> the
> >>>>>>>> checkpoint file(s) for now, and not worrying about the race. The
> >>>>>>> downside
> >>>>>>>> is that we wouldn't be able to remove the explicit RocksDB flush
> >>>>>>> on-commit,
> >>>>>>>> which likely hurts performance.
> >>>>>>>>
> >>>>>>>> If anyone has any thoughts or ideas on this subject, I would
> >>>> appreciate
> >>>>>>> it!
> >>>>>>>>
> >>>>>>>> Regards,
> >>>>>>>> Nick
> >>>>>>>>
> >>>>>>>> On Wed, 19 Apr 2023 at 15:05, Nick Telford <
> nick.telford@gmail.com>
> >>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi Colt,
> >>>>>>>>>
> >>>>>>>>> The issue is that if there's a crash between 2 and 3, then you
> >> still
> >>>>>>> end
> >>>>>>>>> up with inconsistent data in RocksDB. The only way to guarantee
> >> that
> >>>>>>> your
> >>>>>>>>> checkpoint offsets and locally stored data are consistent with
> each
> >>>>>>> other
> >>>>>>>>> are to atomically commit them, which can be achieved by having
> the
> >>>>>>> offsets
> >>>>>>>>> stored in RocksDB.
> >>>>>>>>>
> >>>>>>>>> The offsets column family is likely to be extremely small (one
> >>>>>>>>> per-changelog partition + one per Topology input partition for
> >>>> regular
> >>>>>>>>> stores, one per input partition for global stores). So the
> overhead
> >>>>>>> will be
> >>>>>>>>> minimal.
> >>>>>>>>>
> >>>>>>>>> A major benefit of doing this is that we can remove the explicit
> >>>> calls
> >>>>>>> to
> >>>>>>>>> db.flush(), which forcibly flushes memtables to disk on-commit.
> It
> >>>>>>> turns
> >>>>>>>>> out, RocksDB memtable flushes are largely dictated by Kafka
> Streams
> >>>>>>>>> commits, *not* RocksDB configuration, which could be a major
> source
> >>>> of
> >>>>>>>>> confusion. Atomic checkpointing makes it safe to remove these
> >>>> explicit
> >>>>>>>>> flushes, because it no longer matters exactly when RocksDB
> flushes
> >>>>>>> data to
> >>>>>>>>> disk; since the data and corresponding checkpoint offsets will
> >> always
> >>>>>>> be
> >>>>>>>>> flushed together, the local store is always in a consistent
> state,
> >>>> and
> >>>>>>>>> on-restart, it can always safely resume restoration from the
> >> on-disk
> >>>>>>>>> offsets, restoring the small amount of data that hadn't been
> >> flushed
> >>>>>>> when
> >>>>>>>>> the app exited/crashed.
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>> Nick
> >>>>>>>>>
> >>>>>>>>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io>
> >>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Nick,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for your reply. Ack to A) and B).
> >>>>>>>>>>
> >>>>>>>>>> For item C), I see what you're referring to. Your proposed
> >> solution
> >>>>>>> will
> >>>>>>>>>> work, so no need to change it. What I was suggesting was that it
> >>>>>>> might be
> >>>>>>>>>> possible to achieve this with only one column family. So long
> as:
> >>>>>>>>>>
> >>>>>>>>>>        - No uncommitted records (i.e. not committed to the
> >> changelog)
> >>>> are
> >>>>>>>>>>        *committed* to the state store, AND
> >>>>>>>>>>        - The Checkpoint offset (which refers to the changelog
> >> topic)
> >>>> is
> >>>>>>> less
> >>>>>>>>>>        than or equal to the last written changelog offset in
> >> rocksdb
> >>>>>>>>>>
> >>>>>>>>>> I don't see the need to do the full restoration from scratch. My
> >>>>>>>>>> understanding was that prior to 844/892, full restorations were
> >>>>>>> required
> >>>>>>>>>> because there could be uncommitted records written to RocksDB;
> >>>>>>> however,
> >>>>>>>>>> given your use of RocksDB transactions, that can be avoided with
> >> the
> >>>>>>>>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
> >>>>>>> transaction, 3)
> >>>>>>>>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <
> >>>> nick.telford@gmail.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>
> >>>>>>>>>>> A. I've done my best to de-couple the StateStore stuff from the
> >>>> rest
> >>>>>>> of
> >>>>>>>>>> the
> >>>>>>>>>>> Streams engine. The fact that there will be only one ongoing
> >>>> (write)
> >>>>>>>>>>> transaction at a time is not guaranteed by any API, and is
> just a
> >>>>>>>>>>> consequence of the way Streams operates. To that end, I tried
> to
> >>>>>>> ensure
> >>>>>>>>>> the
> >>>>>>>>>>> documentation and guarantees provided by the new APIs are
> >>>>>>> independent of
> >>>>>>>>>>> this incidental behaviour. In practice, you're right, this
> >>>>>>> essentially
> >>>>>>>>>>> refers to "interactive queries", which are technically "read
> >>>>>>>>>> transactions",
> >>>>>>>>>>> even if they don't actually use the transaction API to isolate
> >>>>>>>>>> themselves.
> >>>>>>>>>>>
> >>>>>>>>>>> B. Yes, although not ideal. This is for backwards
> compatibility,
> >>>>>>>>>> because:
> >>>>>>>>>>>         1) Existing custom StateStore implementations will
> >> implement
> >>>>>>>>>> flush(),
> >>>>>>>>>>> and not commit(), but the Streams engine now calls commit(), so
> >>>> those
> >>>>>>>>>> calls
> >>>>>>>>>>> need to be forwarded to flush() for these legacy stores.
> >>>>>>>>>>>         2) Existing StateStore *users*, i.e. outside of the
> >> Streams
> >>>>>>> engine
> >>>>>>>>>>> itself, may depend on explicitly calling flush(), so for these
> >>>> cases,
> >>>>>>>>>>> flush() needs to be redirected to call commit().
> >>>>>>>>>>> If anyone has a better way to guarantee compatibility without
> >>>>>>>>>> introducing
> >>>>>>>>>>> this potential recursion loop, I'm open to changes!
> >>>>>>>>>>>
> >>>>>>>>>>> C. This is described in the "Atomic Checkpointing" section.
> >> Offsets
> >>>>>>> are
> >>>>>>>>>>> stored in a separate RocksDB column family, which is guaranteed
> >> to
> >>>> be
> >>>>>>>>>>> atomically flushed to disk with all other column families. The
> >>>> issue
> >>>>>>> of
> >>>>>>>>>>> checkpoints being written to disk after commit causing
> >>>> inconsistency
> >>>>>>> if
> >>>>>>>>>> it
> >>>>>>>>>>> crashes in between is the reason why, under EOS, checkpoint
> files
> >>>> are
> >>>>>>>>>> only
> >>>>>>>>>>> written on clean shutdown. This is one of the major causes of
> >> "full
> >>>>>>>>>>> restorations", so moving the offsets into a place where they
> can
> >> be
> >>>>>>>>>>> guaranteed to be atomically written with the data they
> checkpoint
> >>>>>>>>>> allows us
> >>>>>>>>>>> to write the checkpoint offsets *on every commit*, not just on
> >>>> clean
> >>>>>>>>>>> shutdown.
> >>>>>>>>>>>
> >>>>>>>>>>> Regards,
> >>>>>>>>>>> Nick
> >>>>>>>>>>>
> >>>>>>>>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <
> colt@littlehorse.io>
> >>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Nick,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thank you for continuing this work. I have a few minor
> >> clarifying
> >>>>>>>>>>>> questions.
> >>>>>>>>>>>>
> >>>>>>>>>>>> A) "Records written to any transaction are visible to all
> other
> >>>>>>>>>>>> transactions immediately." I am confused here—I thought there
> >>>> could
> >>>>>>>>>> only
> >>>>>>>>>>> be
> >>>>>>>>>>>> one transaction going on at a time for a given state store
> given
> >>>> the
> >>>>>>>>>>>> threading model for processing records on a Task. Do you mean
> >>>>>>>>>> Interactive
> >>>>>>>>>>>> Queries by "other transactions"? (If so, then everything makes
> >>>>>>> sense—I
> >>>>>>>>>>>> thought that since IQ were read-only then they didn't count as
> >>>>>>>>>>>> transactions).
> >>>>>>>>>>>>
> >>>>>>>>>>>> B) Is it intentional that the default implementations of the
> >>>> flush()
> >>>>>>>>>> and
> >>>>>>>>>>>> commit() methods in the StateStore class refer to each other
> in
> >>>> some
> >>>>>>>>>> sort
> >>>>>>>>>>>> of unbounded recursion?
> >>>>>>>>>>>>
> >>>>>>>>>>>> C) How will the getCommittedOffset() method work? At first I
> >>>> thought
> >>>>>>>>>> the
> >>>>>>>>>>>> way to do it would be using a special key in the RocksDB store
> >> to
> >>>>>>>>>> store
> >>>>>>>>>>> the
> >>>>>>>>>>>> offset, and committing that with the transaction. But upon
> >> second
> >>>>>>>>>>> thought,
> >>>>>>>>>>>> since restoration from the changelog is an idempotent
> >> procedure, I
> >>>>>>>>>> think
> >>>>>>>>>>> it
> >>>>>>>>>>>> would be fine to 1) commit the RocksDB transaction and then 2)
> >>>> write
> >>>>>>>>>> the
> >>>>>>>>>>>> offset to disk in a checkpoint file. If there is a crash
> between
> >>>> 1)
> >>>>>>>>>> and
> >>>>>>>>>>> 2),
> >>>>>>>>>>>> I think the only downside is 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 <
> >>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I've updated the KIP to reflect the latest version of the
> >> design:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> There are several changes in there that reflect feedback from
> >>>> this
> >>>>>>>>>>>> thread,
> >>>>>>>>>>>>> and there's a new section and a bunch of interface changes
> >>>> relating
> >>>>>>>>>> to
> >>>>>>>>>>>>> Atomic Checkpointing, which is the final piece of the puzzle
> to
> >>>>>>>>>> making
> >>>>>>>>>>>>> everything robust.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Let me know what you think!
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <
> >>>> nick.telford@gmail.com>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi Lucas,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Thanks for looking over my KIP.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> A) The bound is per-instance, not per-Task. This was a typo
> in
> >>>> the
> >>>>>>>>>>> KIP
> >>>>>>>>>>>>>> that I've now corrected. It was originally per-Task, but I
> >>>>>>>>>> changed it
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>> per-instance for exactly the reason you highlighted.
> >>>>>>>>>>>>>> B) It's worth noting that transactionality is only enabled
> >> under
> >>>>>>>>>> EOS,
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>> in the default mode of operation (ALOS), there should be no
> >>>>>>>>>> change in
> >>>>>>>>>>>>>> behavior at all. I think, under EOS, we can mitigate the
> >> impact
> >>>> on
> >>>>>>>>>>>> users
> >>>>>>>>>>>>> by
> >>>>>>>>>>>>>> sufficiently low default values for the memory bound
> >>>>>>>>>> configuration. I
> >>>>>>>>>>>>>> understand your hesitation to include a significant change
> of
> >>>>>>>>>>>> behaviour,
> >>>>>>>>>>>>>> especially in a minor release, but I suspect that most users
> >>>> will
> >>>>>>>>>>>> prefer
> >>>>>>>>>>>>>> the memory impact (under EOS) to the existing behaviour of
> >>>>>>>>>> frequent
> >>>>>>>>>>>> state
> >>>>>>>>>>>>>> restorations! If this is a problem, the changes can wait
> until
> >>>> the
> >>>>>>>>>>> next
> >>>>>>>>>>>>>> major release. I'll be running a patched version of streams
> in
> >>>>>>>>>>>> production
> >>>>>>>>>>>>>> with these changes as soon as they're ready, so it won't
> >> disrupt
> >>>>>>>>>> me
> >>>>>>>>>>> :-D
> >>>>>>>>>>>>>> C) The main purpose of this sentence was just to note that
> >> some
> >>>>>>>>>>> changes
> >>>>>>>>>>>>>> will need to be made to the way Segments are handled in
> order
> >> to
> >>>>>>>>>>> ensure
> >>>>>>>>>>>>>> they also benefit from transactions. At the time I wrote
> it, I
> >>>>>>>>>> hadn't
> >>>>>>>>>>>>>> figured out the specific changes necessary, so it was
> >>>> deliberately
> >>>>>>>>>>>> vague.
> >>>>>>>>>>>>>> This is the one outstanding problem I'm currently working
> on,
> >>>> and
> >>>>>>>>>>> I'll
> >>>>>>>>>>>>>> update this section with more detail once I have figured out
> >> the
> >>>>>>>>>>> exact
> >>>>>>>>>>>>>> changes required.
> >>>>>>>>>>>>>> D) newTransaction() provides the necessary isolation
> >> guarantees.
> >>>>>>>>>>> While
> >>>>>>>>>>>>>> the RocksDB implementation of transactions doesn't
> technically
> >>>>>>>>>> *need*
> >>>>>>>>>>>>>> read-only users to call newTransaction(), other
> >> implementations
> >>>>>>>>>>> (e.g. a
> >>>>>>>>>>>>>> hypothetical PostgresStore) may require it. Calling
> >>>>>>>>>> newTransaction()
> >>>>>>>>>>>> when
> >>>>>>>>>>>>>> no transaction is necessary is essentially free, as it will
> >> just
> >>>>>>>>>>> return
> >>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think it
> >>>>>>>>>> should
> >>>>>>>>>>> be
> >>>>>>>>>>>>>> fairly obvious where the performance problems stem from:
> >> writes
> >>>>>>>>>> under
> >>>>>>>>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with
> the
> >>>>>>>>>>>>>> tombstone/record flag, 1 to decode it from the
> >> tombstone/record
> >>>>>>>>>> flag,
> >>>>>>>>>>>>> and 1
> >>>>>>>>>>>>>> to copy the record from the "temporary" store to the "main"
> >>>> store,
> >>>>>>>>>>> when
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>> transaction commits. The different approach taken by KIP-869
> >>>>>>>>>> should
> >>>>>>>>>>>>> perform
> >>>>>>>>>>>>>> much better, as it avoids all these copies, and may actually
> >>>>>>>>>> perform
> >>>>>>>>>>>>>> slightly better than trunk, due to batched writes in RocksDB
> >>>>>>>>>>> performing
> >>>>>>>>>>>>>> better than non-batched writes.[1]
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 1:
> >>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>
> >>>>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
> >>>>>>>>>> lbrutschy@confluent.io
> >>>>>>>>>>>>> .invalid>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> I'm just starting to read up on the whole discussion about
> >>>>>>>>>> KIP-892
> >>>>>>>>>>> and
> >>>>>>>>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
> >>>>>>>>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do have
> >> some
> >>>>>>>>>>>>>>> questions about the latest draft.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>      A) If I understand correctly, you propose to put a
> bound
> >> on
> >>>> the
> >>>>>>>>>>>>>>> (native) memory consumed by each task. However, I wonder if
> >>>> this
> >>>>>>>>>> is
> >>>>>>>>>>>>>>> sufficient if we have temporary imbalances in the cluster.
> >> For
> >>>>>>>>>>>>>>> example, depending on the timing of rebalances during a
> >> cluster
> >>>>>>>>>>>>>>> restart, it could happen that a single streams node is
> >>>> assigned a
> >>>>>>>>>>> lot
> >>>>>>>>>>>>>>> more tasks than expected. With your proposed change, this
> >> would
> >>>>>>>>>> mean
> >>>>>>>>>>>>>>> that the memory required by this one node could be a
> multiple
> >>>> of
> >>>>>>>>>>> what
> >>>>>>>>>>>>>>> is required during normal operation. I wonder if it
> wouldn't
> >> be
> >>>>>>>>>>> safer
> >>>>>>>>>>>>>>> to put a global bound on the memory use, across all tasks.
> >>>>>>>>>>>>>>>      B) Generally, the memory concerns still give me the
> >> feeling
> >>>>>>> that
> >>>>>>>>>>> this
> >>>>>>>>>>>>>>> should not be enabled by default for all users in a minor
> >>>>>>>>>> release.
> >>>>>>>>>>>>>>>      C) In section "Transaction Management": the sentence
> "A
> >>>> similar
> >>>>>>>>>>>>>>> analogue will be created to automatically manage `Segment`
> >>>>>>>>>>>>>>> transactions.". Maybe this is just me lacking some
> >> background,
> >>>>>>>>>> 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 you to find out why the KIP-844 PoC is 20x
> >> slower
> >>>>>>>>>> (e.g.
> >>>>>>>>>>>>>>> by providing a flame graph), that would be quite
> interesting.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>>>> Lucas
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
> >>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I've updated the KIP with a more detailed design, which
> >>>>>>>>>> reflects
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> implementation I've been working on:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> This new design should address the outstanding points
> >> already
> >>>>>>>>>> made
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Please let me know if there are areas that are unclear or
> >> need
> >>>>>>>>>>> more
> >>>>>>>>>>>>>>>> clarification.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I have a (nearly) working implementation. I'm confident
> that
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> remaining
> >>>>>>>>>>>>>>>> work (making Segments behave) will not impact the
> documented
> >>>>>>>>>>> design.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <
> >>>> colt@littlehorse.io
> >>>>>>>>>>>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thank you for the reply; that makes sense. I was hoping
> >> that,
> >>>>>>>>>>>> since
> >>>>>>>>>>>>>>> reading
> >>>>>>>>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
> >>>>>>>>>> documented
> >>>>>>>>>>>> API,
> >>>>>>>>>>>>>>> maybe
> >>>>>>>>>>>>>>>>> you *wouldn't* have to wait for the next major release to
> >>>>>>>>>> make
> >>>>>>>>>>>> that
> >>>>>>>>>>>>>>> change;
> >>>>>>>>>>>>>>>>> but given 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 <
> >>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended
> to
> >>>>>>>>>> try
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>> keep the
> >>>>>>>>>>>>>>>>>> behaviour unchanged as much as possible, otherwise we'd
> >>>>>>>>>> have
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>> wait for
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>> major version release to land these changes.
> >>>>>>>>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
> >>>>>>>>>> guarantee,
> >>>>>>>>>>>> and
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> typically longer commit intervals would be problematic
> >> when
> >>>>>>>>>>>>> reading
> >>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>> "committed" records.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I've been away for 5 days recovering from minor surgery,
> >>>>>>>>>> but I
> >>>>>>>>>>>>>>> spent a
> >>>>>>>>>>>>>>>>>> considerable amount of that time working through ideas
> for
> >>>>>>>>>>>>> possible
> >>>>>>>>>>>>>>>>>> solutions in my head. I think your suggestion of keeping
> >>>>>>>>>> ALOS
> >>>>>>>>>>>>>>> as-is, but
> >>>>>>>>>>>>>>>>>> buffering writes for EOS is the right path forwards,
> >>>>>>>>>> although
> >>>>>>>>>>> I
> >>>>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>>>> solution that both expands on this, and provides for
> some
> >>>>>>>>>> more
> >>>>>>>>>>>>>>> formal
> >>>>>>>>>>>>>>>>>> guarantees.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Essentially, adding support to KeyValueStores for
> >>>>>>>>>>>> "Transactions",
> >>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>> clearly defined IsolationLevels. Using "Read Committed"
> >>>>>>>>>> when
> >>>>>>>>>>>> under
> >>>>>>>>>>>>>>> EOS,
> >>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> "Read Uncommitted" under ALOS.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The nice thing about this approach is that it gives us
> >> much
> >>>>>>>>>>> more
> >>>>>>>>>>>>>>> clearly
> >>>>>>>>>>>>>>>>>> defined isolation behaviour that can be properly
> >>>>>>>>>> documented to
> >>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>> users
> >>>>>>>>>>>>>>>>>> know what to expect.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I'm still working out the kinks in the design, and will
> >>>>>>>>>> update
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>>>> I have something. The main struggle is trying to
> implement
> >>>>>>>>>>> this
> >>>>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>>>> making any major changes to the existing interfaces or
> >>>>>>>>>>> breaking
> >>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>> implementations, because currently everything expects to
> >>>>>>>>>>> operate
> >>>>>>>>>>>>>>> directly
> >>>>>>>>>>>>>>>>>> on a StateStore, and not a Transaction of that store. I
> >>>>>>>>>> think
> >>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>>> close, although sadly I won't be able to progress much
> >>>>>>>>>> until
> >>>>>>>>>>>> next
> >>>>>>>>>>>>>>> week
> >>>>>>>>>>>>>>>>> due
> >>>>>>>>>>>>>>>>>> to some work commitments.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
> >>>>>>>>>>> colt@littlehorse.io>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thank you for the explanation, and also for the updated
> >>>>>>>>>>> KIP. I
> >>>>>>>>>>>>> am
> >>>>>>>>>>>>>>> quite
> >>>>>>>>>>>>>>>>>>> eager for this improvement to be released as it would
> >>>>>>>>>>> greatly
> >>>>>>>>>>>>>>> reduce
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> operational difficulties of EOS streams apps.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Two questions:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 10)
> >>>>>>>>>>>>>>>>>>>> When reading records, we will use the
> >>>>>>>>>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
> >>>>>>>>>>>>>>>>>>>      and WriteBatchWithIndex#newIteratorWithBase
> >> utilities in
> >>>>>>>>>>>> order
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>>> that uncommitted writes are available to query.
> >>>>>>>>>>>>>>>>>>> Why do extra work to enable the reading of uncommitted
> >>>>>>>>>>> writes
> >>>>>>>>>>>>>>> during
> >>>>>>>>>>>>>>>>> IQ?
> >>>>>>>>>>>>>>>>>>> Code complexity aside, reading uncommitted writes is,
> in
> >>>>>>>>>> my
> >>>>>>>>>>>>>>> opinion, a
> >>>>>>>>>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have the
> >>>>>>>>>>>>> guarantee
> >>>>>>>>>>>>>>> that,
> >>>>>>>>>>>>>>>>>>> with EOS, IQ only reads committed records. In order to
> >>>>>>>>>> avoid
> >>>>>>>>>>>>> dirty
> >>>>>>>>>>>>>>>>> reads,
> >>>>>>>>>>>>>>>>>>> one currently must query a standby replica (but this
> >>>>>>>>>> still
> >>>>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>>>> fully
> >>>>>>>>>>>>>>>>>>> guarantee monotonic reads).
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 20) Is it also necessary to enable this optimization on
> >>>>>>>>>> ALOS
> >>>>>>>>>>>>>>> stores?
> >>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need to
> >>>>>>>>>>> restore
> >>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
> >>>>>>>>>>> acceptable
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>> accept
> >>>>>>>>>>>>>>>>>>> that there may have been uncommitted writes on disk.
> On a
> >>>>>>>>>>> side
> >>>>>>>>>>>>>>> note, if
> >>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>> enable this type of store on ALOS processors, the
> >>>>>>>>>> community
> >>>>>>>>>>>>> would
> >>>>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
> >>>>>>>>>>> longer
> >>>>>>>>>>>>>>> include
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> StateStore management of checkpointing. This can be
> >>>>>>>>>> added
> >>>>>>>>>>>> as a
> >>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>>>> to further optimize the consistency and performance of
> >>>>>>>>>>> state
> >>>>>>>>>>>>>>> stores.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I've also added a section discussing some of the
> >>>>>>>>>> concerns
> >>>>>>>>>>>>> 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 2022 at 16:36, Nick Telford <
> >>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I didn't do any profiling, but the 844
> >>>>>>>>>> implementation:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>        - Writes uncommitted records to a temporary
> >>>>>>>>>> RocksDB
> >>>>>>>>>>>>>>> instance
> >>>>>>>>>>>>>>>>>>>>>           - Since tombstones need to be flagged, all
> >>>>>>>>>> record
> >>>>>>>>>>>>>>> values are
> >>>>>>>>>>>>>>>>>>>>>           prefixed with a value/tombstone marker.
> This
> >>>>>>>>>>>>>>> necessitates a
> >>>>>>>>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>>>> copy.
> >>>>>>>>>>>>>>>>>>>>>        - On-commit, iterates all records in this
> >>>>>>>>>> temporary
> >>>>>>>>>>>>>>> instance and
> >>>>>>>>>>>>>>>>>>>>>        writes them to the main RocksDB store.
> >>>>>>>>>>>>>>>>>>>>>        - While iterating, the value/tombstone marker
> >>>>>>>>>> needs
> >>>>>>>>>>> to
> >>>>>>>>>>>> be
> >>>>>>>>>>>>>>> parsed
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>        the real value extracted. This necessitates
> >>>>>>>>>> another
> >>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>> copy.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> My guess is that the cost of iterating the temporary
> >>>>>>>>>>>> RocksDB
> >>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> major factor, with the 2 extra memory copies
> >>>>>>>>>> per-Record
> >>>>>>>>>>>>>>>>> contributing
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>> significant amount too.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
> >>>>>>>>>>>>>>> colt@littlehorse.io>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Out of curiosity, why does the performance of the
> >>>>>>>>>> store
> >>>>>>>>>>>>>>> degrade so
> >>>>>>>>>>>>>>>>>>>>>> significantly with the 844 implementation? I
> >>>>>>>>>> wouldn't
> >>>>>>>>>>> be
> >>>>>>>>>>>>> too
> >>>>>>>>>>>>>>>>>> surprised
> >>>>>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>> 50-60% drop (caused by each record being written
> >>>>>>>>>>> twice),
> >>>>>>>>>>>>> but
> >>>>>>>>>>>>>>> 96%
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>> extreme.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> The only thing I can think of which could create
> >>>>>>>>>> such a
> >>>>>>>>>>>>>>> bottleneck
> >>>>>>>>>>>>>>>>>>> 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 Nick Telford <
> >>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP to resolve all the points
> >>>>>>>>>> that
> >>>>>>>>>>>> have
> >>>>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>>>> raised
> >>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>>> far, with one exception: the ALOS default commit
> >>>>>>>>>>>> interval
> >>>>>>>>>>>>>>> of 5
> >>>>>>>>>>>>>>>>>>> minutes
> >>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> likely to cause WriteBatchWithIndex memory to grow
> >>>>>>>>>>> too
> >>>>>>>>>>>>>>> large.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> There's a couple of different things I can think
> >>>>>>>>>> of
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>> solve
> >>>>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>        - We already have a memory/record limit in
> the
> >>>>>>>>>> KIP
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>> prevent
> >>>>>>>>>>>>>>>>>> OOM
> >>>>>>>>>>>>>>>>>>>>>>>        errors. Should we choose a default value for
> >>>>>>>>>>> these?
> >>>>>>>>>>>> My
> >>>>>>>>>>>>>>>>> concern
> >>>>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>        anything we choose might seem rather
> >>>>>>>>>> arbitrary. We
> >>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>>>>>>>        its behaviour such that under ALOS, it only
> >>>>>>>>>>> triggers
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>        StateStore, but under EOS, it triggers a
> >>>>>>>>>> commit of
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>>>>>>        - We could introduce a separate `
> >>>>>>>>>>>>> checkpoint.interval.ms`
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> allow
> >>>>>>>>>>>>>>>>>>>>>> ALOS
> >>>>>>>>>>>>>>>>>>>>>>>        to commit the StateStores more frequently
> than
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> general
> >>>>>>>>>>>>>>>>>>>>>>>        commit.interval.ms? My concern here is that
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> semantics of
> >>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> config
> >>

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

Posted by John Roesler <vv...@apache.org>.
Ah, sorry Nick,

I just meant the regular heap based cache that we maintain in Streams. I 
see that it's not called "RecordCache" (my mistake).

The actual cache is ThreadCache: 
https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/ThreadCache.java

Here's the example of how we use the cache in KeyValueStore:
https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java

It's basically just an on-heap Map of records that have not yet been 
written to the changelog or flushed into the underlying store. It gets 
flushed when the total cache size exceeds `cache.max.bytes.buffering` or 
the `commit.interval.ms` elapses.

Speaking of those configs, another benefit to this idea is that we would 
no longer need to trigger extra commits based on the size of the ongoing 
transaction. Instead, we'd just preserve the existing cache-flush 
behavior. Note that users can disable the cache, which would still be 
ok, I think. We wouldn't ingest the SST files on every record, but just 
append to them and only ingest them on commit, when we're already 
waiting for acks and a RocksDB commit.

Thanks,
-John

On 6/20/23 14:09, Nick Telford wrote:
> Hi John,
> 
> By "RecordCache", do you mean the RocksDB "WriteBatch"? I can't find any
> class called "RecordCache"...
> 
> Cheers,
> 
> Nick
> 
> On Tue, 20 Jun 2023 at 19:42, John Roesler <vv...@apache.org> wrote:
> 
>> Hi Nick,
>>
>> Thanks for picking this up again!
>>
>> I did have one new thought over the intervening months, which I'd like
>> your take on.
>>
>> What if, instead of using the RocksDB atomic write primitive at all, we
>> instead just:
>> 1. disable memtables entirely
>> 2. directly write the RecordCache into SST files when we flush
>> 3. atomically ingest the SST file(s) into RocksDB when we get the ACK
>> from the changelog (see
>>
>> https://github.com/EighteenZi/rocksdb_wiki/blob/master/Creating-and-Ingesting-SST-files.md
>> and
>>
>> https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/IngestExternalFileOptions.java
>> and
>>
>> https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L1413-L1429
>> )
>> 4. track the changelog offsets either in another CF or the same CF with
>> a reserved key, either of which will make the changelog offset update
>> atomic with the file ingestions
>>
>> I suspect this'll have a number of benefits:
>> * writes to RocksDB will always be atomic
>> * we don't fragment memory between the RecordCache and the memtables
>> * RecordCache gives far higher performance than memtable for reads and
>> writes
>> * we don't need any new "transaction" concepts or memory bound configs
>>
>> What do you think?
>>
>> Thanks,
>> -John
>>
>> On 6/20/23 10:51, Nick Telford wrote:
>>> Hi Bruno,
>>>
>>> Thanks for reviewing the KIP. It's been a long road, I started working on
>>> this more than a year ago, and most of the time in the last 6 months has
>>> been spent on the "Atomic Checkpointing" stuff that's been benched, so
>> some
>>> of the reasoning behind some of my decisions have been lost, but I'll do
>> my
>>> best to reconstruct them.
>>>
>>> 1.
>>> IIRC, this was the initial approach I tried. I don't remember the exact
>>> reasons I changed it to use a separate "view" of the StateStore that
>>> encapsulates the transaction, but I believe it had something to do with
>>> concurrent access to the StateStore from Interactive Query threads. Reads
>>> from interactive queries need to be isolated from the currently ongoing
>>> transaction, both for consistency (so interactive queries don't observe
>>> changes that are subsequently rolled-back), but also to prevent Iterators
>>> opened by an interactive query from being closed and invalidated by the
>>> StreamThread when it commits the transaction, which causes your
>> interactive
>>> queries to crash.
>>>
>>> Another reason I believe I implemented it this way was a separation of
>>> concerns. Recall that newTransaction() originally created an object of
>> type
>>> Transaction, not StateStore. My intent was to improve the type-safety of
>>> the API, in an effort to ensure Transactions weren't used incorrectly.
>>> Unfortunately, this didn't pan out, but newTransaction() remained.
>>>
>>> Finally, this had the added benefit that implementations could easily add
>>> support for transactions *without* re-writing their existing,
>>> non-transactional implementation. I think this can be a benefit both for
>>> implementers of custom StateStores, but also for anyone extending
>>> RocksDbStore, as they can rely on the existing access methods working how
>>> they expect them to.
>>>
>>> I'm not too happy with the way the current design has panned out, so I'm
>>> open to ideas on how to improve it. Key to this is finding some way to
>>> ensure that reads from Interactive Query threads are properly isolated
>> from
>>> the transaction, *without* the performance overhead of checking which
>>> thread the method is being called from on every access.
>>>
>>> As for replacing flush() with commit() - I saw no reason to add this
>>> complexity to the KIP, unless there was a need to add arguments to the
>>> flush/commit method. This need arises with Atomic Checkpointing, but that
>>> will be implemented separately, in a future KIP. Do you see a need for
>> some
>>> arguments to the flush/commit method that I've missed? Or were you simply
>>> suggesting a rename?
>>>
>>> 2.
>>> This is simply due to the practical reason that isolationLevel() is
>> really
>>> a proxy for checking if the app is under EOS. The application
>> configuration
>>> is not provided to the constructor of StateStores, but it *is* provided
>> to
>>> init(), via StateStoreContext. For this reason, it seemed somewhat
>> natural
>>> to add it to StateStoreContext. I think this makes sense, since the
>>> IsolationLevel of all StateStores in an application *must* be the same,
>> and
>>> since those stores are all initialized with the same StateStoreContext,
>> it
>>> seems natural for that context to carry the desired IsolationLevel to
>> use.
>>>
>>> 3.
>>> Using IsolationLevel instead of just passing `boolean eosEnabled`, like
>>> much of the internals was an attempt to logically de-couple the
>> StateStore
>>> API from the internals of Kafka Streams. Technically, StateStores don't
>>> need to know/care what processing mode the KS app is using, all they need
>>> to know is the isolation level expected of them.
>>>
>>> Having formal definitions for the expectations of the two required
>>> IsolationLevels allow implementers to implement transactional stores
>>> without having to dig through the internals of Kafka Streams and
>> understand
>>> exactly how they are used. The tight coupling between state stores and
>>> internal behaviour has actually significantly hindered my progress on
>> this
>>> KIP, and encouraged me to avoid increasing this logical coupling as much
>> as
>>> possible.
>>>
>>> This also frees implementations to satisfy those requirements in any way
>>> they choose. Transactions might not be the only/available approach to an
>>> implementation, but they might have an alternative way to satisfy the
>>> isolation requirements. I admit that this point is more about semantics,
>>> but "transactional" would need to be formally defined in order for
>>> implementers to provide a valid implementation, and these IsolationLevels
>>> provide that formal definition.
>>>
>>> 4.
>>> I can remove them. I added them only as I planned to include them in the
>>> org.apache.kafka.streams.state package, as a recommended base
>>> implementation for all StateStores, including those implemented by
>> users. I
>>> had assumed that anything in "public" packages, such as
>>> org.apache.kafka.streams.state, should be included in a KIP. Is that
>> wrong?
>>>
>>> 5.
>>> RocksDB provides no way to measure the actual size of a
>>> WriteBatch(WithIndex), so we're limited to tracking the sum total of the
>>> size of keys + values that are written to the transaction. This obviously
>>> under-estimates the actual memory usage, because WriteBatch no-doubt
>>> includes some record overheads, and WriteBatchWithIndex has to maintain
>> an
>>> index.
>>>
>>> Ideally, we could trivially add a method upstream to WriteBatchInterface
>>> that provides the exact size of the batch, but that would require an
>>> upgrade of RocksDB, which won't happen soon. So for the time being, we're
>>> stuck with an approximation, so I felt that the new method should reflect
>>> that.
>>>
>>> Would you prefer the new method name ignores this constraint and that we
>>> simply make the rocks measurement more accurate in the future?
>>>
>>> 6.
>>> Done
>>>
>>> 7.
>>> Very good point. The KIP already specifically calls out memory in the
>>> documentation of the config: "Maximum number of memory bytes to be used
>> to
>>> buffer uncommitted state-store records." - did you have something else in
>>> mind?
>>>
>>> Should we also make this clearer by renaming the config property itself?
>>> Perhaps to something like statestore.transaction.buffer.max.bytes?
>>>
>>> 8.
>>> OK, I can remove this. The intent here was to describe how Streams itself
>>> will manage transaction roll-over etc. Presumably that means we also
>> don't
>>> need a description of how Streams will manage the commit of changelog
>>> transactions, state store transactions and checkpointing?
>>>
>>> 9.
>>> What do you mean by fail-over? Do you mean failing over an Active Task to
>>> an instance already hosting a Standby Task?
>>>
>>> Thanks again and sorry for the essay of a response!
>>>
>>> Regards,
>>> Nick
>>>
>>> On Tue, 20 Jun 2023 at 10:49, Bruno Cadonna <ca...@apache.org> wrote:
>>>
>>>> Hi Nick,
>>>>
>>>> Thanks for the updates!
>>>>
>>>> I really appreciate that you simplified the KIP by removing some
>>>> aspects. As I have already told you, I think the removed aspects are
>>>> also good ideas and we can discuss them on follow-up KIPs.
>>>>
>>>> Regarding the current KIP, I have the following feedback.
>>>>
>>>> 1.
>>>> Is there a good reason to add method newTransaction() to the StateStore
>>>> interface? As far as I understand, the idea is that users of a state
>>>> store (transactional or not) call this method at start-up and after each
>>>> commit. Since the call to newTransaction() is done in any case and I
>>>> think it would simplify the caller code if we just start a new
>>>> transaction after a commit in the implementation?
>>>> As far as I understand, you plan to commit the transaction in the
>>>> flush() method. I find the idea to replace flush() with commit()
>>>> presented in KIP-844 an elegant solution.
>>>>
>>>> 2.
>>>> Why is the method to query the isolation level added to the state store
>>>> context?
>>>>
>>>> 3.
>>>> Do we need all the isolation level definitions? I think it is good to
>>>> know the guarantees of the transactionality of the state store.
>>>> However, currently, Streams guarantees that there will only be one
>>>> transaction that writes to the state store. Only the stream thread that
>>>> executes the active task that owns the state store will write to the
>>>> state store. I think it should be enough to know if the state store is
>>>> transactional or not. So my proposal would be to just add a method on
>>>> the state store interface the returns if a state store is transactional
>>>> or not by returning a boolean or an enum.
>>>>
>>>> 4.
>>>> I am wondering why AbstractTransaction and AbstractTransactionalStore
>>>> are part of the KIP. They look like implementation details that should
>>>> not be exposed in the public API.
>>>>
>>>> 5.
>>>> Why does StateStore#approximateNumUncommittedBytes() return an
>>>> approximate number of bytes?
>>>>
>>>> 6.
>>>> RocksDB is just one implementation of the state stores in Streams.
>>>> However, the issues regarding OOM errors might also apply to other
>>>> custom implementations. So in the KIP I would extract that part from
>>>> section "RocksDB Transaction". I would also move section "RocksDB
>>>> Transaction" to the end of section "Proposed Changes" and handle it as
>>>> an example implementation for a state store.
>>>>
>>>> 7.
>>>> Should statestore.uncommitted.max.bytes only limit the uncommitted bytes
>>>> or the uncommitted bytes that reside in memory? In future, other
>>>> transactional state store implementations might implement a buffer for
>>>> uncommitted records that are able to spill records on disk. I think
>>>> statestore.uncommitted.max.bytes needs to limit the uncommitted bytes
>>>> irrespective if they reside in memory or disk. Since Streams will use
>>>> this config to decide if it needs to trigger a commit, state store
>>>> implementations that can spill to disk will never be able to spill to
>>>> disk. You would only need to change the doc of the config, if you agree
>>>> with me.
>>>>
>>>> 8.
>>>> Section "Transaction Management" about the wrappers is rather a
>>>> implementation detail that should not be in the KIP.
>>>>
>>>> 9.
>>>> Could you add a section that describes how failover will work with the
>>>> transactional state stores? I think section "Error handling" is already
>>>> a good start.
>>>>
>>>>
>>>> Best,
>>>> Bruno
>>>>
>>>>
>>>>
>>>>
>>>> On 15.05.23 11:04, Nick Telford wrote:
>>>>> Hi everyone,
>>>>>
>>>>> Quick update: I've added a new section to the KIP: "Offsets for
>> Consumer
>>>>> Rebalances", that outlines my solution to the problem that
>>>>> StreamsPartitionAssignor needs to read StateStore offsets even if
>> they're
>>>>> not currently open.
>>>>>
>>>>> Regards,
>>>>> Nick
>>>>>
>>>>> On Wed, 3 May 2023 at 11:34, Nick Telford <ni...@gmail.com>
>>>> wrote:
>>>>>
>>>>>> Hi Bruno,
>>>>>>
>>>>>> Thanks for reviewing my proposal.
>>>>>>
>>>>>> 1.
>>>>>> The main reason I added it was because it was easy to do. If we see no
>>>>>> value in it, I can remove it.
>>>>>>
>>>>>> 2.
>>>>>> Global StateStores can have multiple partitions in their input topics
>>>>>> (which function as their changelogs), so they would have more than one
>>>>>> partition.
>>>>>>
>>>>>> 3.
>>>>>> That's a good point. At present, the only method it adds is
>>>>>> isolationLevel(), which is likely not necessary outside of
>> StateStores.
>>>>>> It *does* provide slightly different guarantees in the documentation
>> to
>>>>>> several of the methods (hence the overrides). I'm not sure if this is
>>>>>> enough to warrant a new interface though.
>>>>>> I think the question that remains is whether this interface makes it
>>>>>> easier to implement custom transactional StateStores than if we were
>> to
>>>>>> remove it? Probably not.
>>>>>>
>>>>>> 4.
>>>>>> The main motivation for the Atomic Checkpointing is actually
>>>> performance.
>>>>>> My team has been testing out an implementation of this KIP without it,
>>>> and
>>>>>> we had problems with RocksDB doing *much* more compaction, due to the
>>>>>> significantly increased flush rate. It was enough of a problem that
>> (for
>>>>>> the time being), we had to revert back to Kafka Streams proper.
>>>>>> I think the best way to solve this, as you say, is to keep the
>>>> .checkpoint
>>>>>> files *in addition* to the offsets being stored within the store
>> itself.
>>>>>> Essentially, when closing StateStores, we force a memtable flush, then
>>>>>> call getCommittedOffsets and write those out to the .checkpoint file.
>>>>>> That would ensure the metadata is available to the
>>>>>> StreamsPartitionAssignor for all closed stores.
>>>>>> If there's a crash (no clean close), then we won't be able to
>> guarantee
>>>>>> which offsets were flushed to disk by RocksDB, so we'd need to open (
>>>>>> init()), read offsets, and then close() those stores. But since this
>> is
>>>>>> the exception, and will only occur once (provided it doesn't crash
>> every
>>>>>> time!), I think the performance impact here would be acceptable.
>>>>>>
>>>>>> Thanks for the feedback, please let me know if you have any more
>>>> comments
>>>>>> or questions!
>>>>>>
>>>>>> I'm currently working on rebasing against trunk. This involves adding
>>>>>> support for transactionality to VersionedStateStores. I will probably
>>>> need
>>>>>> to revise my implementation for transactional "segmented" stores, both
>>>> to
>>>>>> accommodate VersionedStateStore, and to clean up some other stuff.
>>>>>>
>>>>>> Regards,
>>>>>> Nick
>>>>>>
>>>>>>
>>>>>> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org>
>> wrote:
>>>>>>
>>>>>>> Hi Nick,
>>>>>>>
>>>>>>> Thanks for the updates!
>>>>>>>
>>>>>>> I have a couple of questions/comments.
>>>>>>>
>>>>>>> 1.
>>>>>>> Why do you propose a configuration that involves max. bytes and max.
>>>>>>> reords? I think we are mainly concerned about memory consumption
>>>> because
>>>>>>> we want to limit the off-heap memory used. I cannot think of a case
>>>>>>> where one would want to set the max. number of records.
>>>>>>>
>>>>>>>
>>>>>>> 2.
>>>>>>> Why does
>>>>>>>
>>>>>>>      default void commit(final Map<TopicPartition, Long>
>>>> changelogOffsets) {
>>>>>>>          flush();
>>>>>>>      }
>>>>>>>
>>>>>>> take a map of partitions to changelog offsets?
>>>>>>> The mapping between state stores to partitions is a 1:1 relationship.
>>>>>>> Passing in a single changelog offset should suffice.
>>>>>>>
>>>>>>>
>>>>>>> 3.
>>>>>>> Why do we need the Transaction interface? It should be possible to
>> hide
>>>>>>> beginning and committing a transactions withing the state store
>>>>>>> implementation, so that from outside the state store, it does not
>>>> matter
>>>>>>> whether the state store is transactional or not. What would be the
>>>>>>> advantage of using the Transaction interface?
>>>>>>>
>>>>>>>
>>>>>>> 4.
>>>>>>> Regarding checkpointing offsets, I think we should keep the
>> checkpoint
>>>>>>> file in any case for the reason you mentioned about rebalancing. Even
>>>> if
>>>>>>> that would not be an issue, I would propose to move the change to
>>>> offset
>>>>>>> management to a new KIP and to not add more complexity than needed to
>>>>>>> this one. I would not be too concerned about the consistency
>> violation
>>>>>>> you mention. As far as I understand, with transactional state stores
>>>>>>> Streams would write the checkpoint file during every commit even
>> under
>>>>>>> EOS. In the failure case you describe, Streams would restore the
>> state
>>>>>>> stores from the offsets found in the checkpoint file written during
>> the
>>>>>>> penultimate commit instead of during the last commit. Basically,
>>>> Streams
>>>>>>> would overwrite the records written to the state store between the
>> last
>>>>>>> two commits with the same records read from the changelogs. While I
>>>>>>> understand that this is wasteful, it is -- at the same time --
>>>>>>> acceptable and most importantly it does not break EOS.
>>>>>>>
>>>>>>> Best,
>>>>>>> Bruno
>>>>>>>
>>>>>>>
>>>>>>> On 27.04.23 12:34, Nick Telford wrote:
>>>>>>>> Hi everyone,
>>>>>>>>
>>>>>>>> I find myself (again) considering removing the offset management
>> from
>>>>>>>> StateStores, and keeping the old checkpoint file system. The reason
>> is
>>>>>>> that
>>>>>>>> the StreamPartitionAssignor directly reads checkpoint files in order
>>>> to
>>>>>>>> determine which instance has the most up-to-date copy of the local
>>>>>>> state.
>>>>>>>> If we move offsets into the StateStore itself, then we will need to
>>>>>>> open,
>>>>>>>> initialize, read offsets and then close each StateStore (that is not
>>>>>>>> already assigned and open) for which we have *any* local state, on
>>>> every
>>>>>>>> rebalance.
>>>>>>>>
>>>>>>>> Generally, I don't think there are many "orphan" stores like this
>>>>>>> sitting
>>>>>>>> around on most instances, but even a few would introduce additional
>>>>>>> latency
>>>>>>>> to an already somewhat lengthy rebalance procedure.
>>>>>>>>
>>>>>>>> I'm leaning towards Colt's (Slack) suggestion of just keeping things
>>>> in
>>>>>>> the
>>>>>>>> checkpoint file(s) for now, and not worrying about the race. The
>>>>>>> downside
>>>>>>>> is that we wouldn't be able to remove the explicit RocksDB flush
>>>>>>> on-commit,
>>>>>>>> which likely hurts performance.
>>>>>>>>
>>>>>>>> If anyone has any thoughts or ideas on this subject, I would
>>>> appreciate
>>>>>>> it!
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>> Nick
>>>>>>>>
>>>>>>>> On Wed, 19 Apr 2023 at 15:05, Nick Telford <ni...@gmail.com>
>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Colt,
>>>>>>>>>
>>>>>>>>> The issue is that if there's a crash between 2 and 3, then you
>> still
>>>>>>> end
>>>>>>>>> up with inconsistent data in RocksDB. The only way to guarantee
>> that
>>>>>>> your
>>>>>>>>> checkpoint offsets and locally stored data are consistent with each
>>>>>>> other
>>>>>>>>> are to atomically commit them, which can be achieved by having the
>>>>>>> offsets
>>>>>>>>> stored in RocksDB.
>>>>>>>>>
>>>>>>>>> The offsets column family is likely to be extremely small (one
>>>>>>>>> per-changelog partition + one per Topology input partition for
>>>> regular
>>>>>>>>> stores, one per input partition for global stores). So the overhead
>>>>>>> will be
>>>>>>>>> minimal.
>>>>>>>>>
>>>>>>>>> A major benefit of doing this is that we can remove the explicit
>>>> calls
>>>>>>> to
>>>>>>>>> db.flush(), which forcibly flushes memtables to disk on-commit. It
>>>>>>> turns
>>>>>>>>> out, RocksDB memtable flushes are largely dictated by Kafka Streams
>>>>>>>>> commits, *not* RocksDB configuration, which could be a major source
>>>> of
>>>>>>>>> confusion. Atomic checkpointing makes it safe to remove these
>>>> explicit
>>>>>>>>> flushes, because it no longer matters exactly when RocksDB flushes
>>>>>>> data to
>>>>>>>>> disk; since the data and corresponding checkpoint offsets will
>> always
>>>>>>> be
>>>>>>>>> flushed together, the local store is always in a consistent state,
>>>> and
>>>>>>>>> on-restart, it can always safely resume restoration from the
>> on-disk
>>>>>>>>> offsets, restoring the small amount of data that hadn't been
>> flushed
>>>>>>> when
>>>>>>>>> the app exited/crashed.
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>> Nick
>>>>>>>>>
>>>>>>>>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io>
>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Nick,
>>>>>>>>>>
>>>>>>>>>> Thanks for your reply. Ack to A) and B).
>>>>>>>>>>
>>>>>>>>>> For item C), I see what you're referring to. Your proposed
>> solution
>>>>>>> will
>>>>>>>>>> work, so no need to change it. What I was suggesting was that it
>>>>>>> might be
>>>>>>>>>> possible to achieve this with only one column family. So long as:
>>>>>>>>>>
>>>>>>>>>>        - No uncommitted records (i.e. not committed to the
>> changelog)
>>>> are
>>>>>>>>>>        *committed* to the state store, AND
>>>>>>>>>>        - The Checkpoint offset (which refers to the changelog
>> topic)
>>>> is
>>>>>>> less
>>>>>>>>>>        than or equal to the last written changelog offset in
>> rocksdb
>>>>>>>>>>
>>>>>>>>>> I don't see the need to do the full restoration from scratch. My
>>>>>>>>>> understanding was that prior to 844/892, full restorations were
>>>>>>> required
>>>>>>>>>> because there could be uncommitted records written to RocksDB;
>>>>>>> however,
>>>>>>>>>> given your use of RocksDB transactions, that can be avoided with
>> the
>>>>>>>>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
>>>>>>> transaction, 3)
>>>>>>>>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <
>>>> nick.telford@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>
>>>>>>>>>>> A. I've done my best to de-couple the StateStore stuff from the
>>>> rest
>>>>>>> of
>>>>>>>>>> the
>>>>>>>>>>> Streams engine. The fact that there will be only one ongoing
>>>> (write)
>>>>>>>>>>> transaction at a time is not guaranteed by any API, and is just a
>>>>>>>>>>> consequence of the way Streams operates. To that end, I tried to
>>>>>>> ensure
>>>>>>>>>> the
>>>>>>>>>>> documentation and guarantees provided by the new APIs are
>>>>>>> independent of
>>>>>>>>>>> this incidental behaviour. In practice, you're right, this
>>>>>>> essentially
>>>>>>>>>>> refers to "interactive queries", which are technically "read
>>>>>>>>>> transactions",
>>>>>>>>>>> even if they don't actually use the transaction API to isolate
>>>>>>>>>> themselves.
>>>>>>>>>>>
>>>>>>>>>>> B. Yes, although not ideal. This is for backwards compatibility,
>>>>>>>>>> because:
>>>>>>>>>>>         1) Existing custom StateStore implementations will
>> implement
>>>>>>>>>> flush(),
>>>>>>>>>>> and not commit(), but the Streams engine now calls commit(), so
>>>> those
>>>>>>>>>> calls
>>>>>>>>>>> need to be forwarded to flush() for these legacy stores.
>>>>>>>>>>>         2) Existing StateStore *users*, i.e. outside of the
>> Streams
>>>>>>> engine
>>>>>>>>>>> itself, may depend on explicitly calling flush(), so for these
>>>> cases,
>>>>>>>>>>> flush() needs to be redirected to call commit().
>>>>>>>>>>> If anyone has a better way to guarantee compatibility without
>>>>>>>>>> introducing
>>>>>>>>>>> this potential recursion loop, I'm open to changes!
>>>>>>>>>>>
>>>>>>>>>>> C. This is described in the "Atomic Checkpointing" section.
>> Offsets
>>>>>>> are
>>>>>>>>>>> stored in a separate RocksDB column family, which is guaranteed
>> to
>>>> be
>>>>>>>>>>> atomically flushed to disk with all other column families. The
>>>> issue
>>>>>>> of
>>>>>>>>>>> checkpoints being written to disk after commit causing
>>>> inconsistency
>>>>>>> if
>>>>>>>>>> it
>>>>>>>>>>> crashes in between is the reason why, under EOS, checkpoint files
>>>> are
>>>>>>>>>> only
>>>>>>>>>>> written on clean shutdown. This is one of the major causes of
>> "full
>>>>>>>>>>> restorations", so moving the offsets into a place where they can
>> be
>>>>>>>>>>> guaranteed to be atomically written with the data they checkpoint
>>>>>>>>>> allows us
>>>>>>>>>>> to write the checkpoint offsets *on every commit*, not just on
>>>> clean
>>>>>>>>>>> shutdown.
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>> Nick
>>>>>>>>>>>
>>>>>>>>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io>
>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Nick,
>>>>>>>>>>>>
>>>>>>>>>>>> Thank you for continuing this work. I have a few minor
>> clarifying
>>>>>>>>>>>> questions.
>>>>>>>>>>>>
>>>>>>>>>>>> A) "Records written to any transaction are visible to all other
>>>>>>>>>>>> transactions immediately." I am confused here—I thought there
>>>> could
>>>>>>>>>> only
>>>>>>>>>>> be
>>>>>>>>>>>> one transaction going on at a time for a given state store given
>>>> the
>>>>>>>>>>>> threading model for processing records on a Task. Do you mean
>>>>>>>>>> Interactive
>>>>>>>>>>>> Queries by "other transactions"? (If so, then everything makes
>>>>>>> sense—I
>>>>>>>>>>>> thought that since IQ were read-only then they didn't count as
>>>>>>>>>>>> transactions).
>>>>>>>>>>>>
>>>>>>>>>>>> B) Is it intentional that the default implementations of the
>>>> flush()
>>>>>>>>>> and
>>>>>>>>>>>> commit() methods in the StateStore class refer to each other in
>>>> some
>>>>>>>>>> sort
>>>>>>>>>>>> of unbounded recursion?
>>>>>>>>>>>>
>>>>>>>>>>>> C) How will the getCommittedOffset() method work? At first I
>>>> thought
>>>>>>>>>> the
>>>>>>>>>>>> way to do it would be using a special key in the RocksDB store
>> to
>>>>>>>>>> store
>>>>>>>>>>> the
>>>>>>>>>>>> offset, and committing that with the transaction. But upon
>> second
>>>>>>>>>>> thought,
>>>>>>>>>>>> since restoration from the changelog is an idempotent
>> procedure, I
>>>>>>>>>> think
>>>>>>>>>>> it
>>>>>>>>>>>> would be fine to 1) commit the RocksDB transaction and then 2)
>>>> write
>>>>>>>>>> the
>>>>>>>>>>>> offset to disk in a checkpoint file. If there is a crash between
>>>> 1)
>>>>>>>>>> and
>>>>>>>>>>> 2),
>>>>>>>>>>>> I think the only downside is 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 <
>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>
>>>>>>>>>>>>> I've updated the KIP to reflect the latest version of the
>> design:
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>
>>>>>>>>>>>>> There are several changes in there that reflect feedback from
>>>> this
>>>>>>>>>>>> thread,
>>>>>>>>>>>>> and there's a new section and a bunch of interface changes
>>>> relating
>>>>>>>>>> to
>>>>>>>>>>>>> Atomic Checkpointing, which is the final piece of the puzzle to
>>>>>>>>>> making
>>>>>>>>>>>>> everything robust.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Let me know what you think!
>>>>>>>>>>>>>
>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <
>>>> nick.telford@gmail.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Lucas,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks for looking over my KIP.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> A) The bound is per-instance, not per-Task. This was a typo in
>>>> the
>>>>>>>>>>> KIP
>>>>>>>>>>>>>> that I've now corrected. It was originally per-Task, but I
>>>>>>>>>> changed it
>>>>>>>>>>>> to
>>>>>>>>>>>>>> per-instance for exactly the reason you highlighted.
>>>>>>>>>>>>>> B) It's worth noting that transactionality is only enabled
>> under
>>>>>>>>>> EOS,
>>>>>>>>>>>> and
>>>>>>>>>>>>>> in the default mode of operation (ALOS), there should be no
>>>>>>>>>> change in
>>>>>>>>>>>>>> behavior at all. I think, under EOS, we can mitigate the
>> impact
>>>> on
>>>>>>>>>>>> users
>>>>>>>>>>>>> by
>>>>>>>>>>>>>> sufficiently low default values for the memory bound
>>>>>>>>>> configuration. I
>>>>>>>>>>>>>> understand your hesitation to include a significant change of
>>>>>>>>>>>> behaviour,
>>>>>>>>>>>>>> especially in a minor release, but I suspect that most users
>>>> will
>>>>>>>>>>>> prefer
>>>>>>>>>>>>>> the memory impact (under EOS) to the existing behaviour of
>>>>>>>>>> frequent
>>>>>>>>>>>> state
>>>>>>>>>>>>>> restorations! If this is a problem, the changes can wait until
>>>> the
>>>>>>>>>>> next
>>>>>>>>>>>>>> major release. I'll be running a patched version of streams in
>>>>>>>>>>>> production
>>>>>>>>>>>>>> with these changes as soon as they're ready, so it won't
>> disrupt
>>>>>>>>>> me
>>>>>>>>>>> :-D
>>>>>>>>>>>>>> C) The main purpose of this sentence was just to note that
>> some
>>>>>>>>>>> changes
>>>>>>>>>>>>>> will need to be made to the way Segments are handled in order
>> to
>>>>>>>>>>> ensure
>>>>>>>>>>>>>> they also benefit from transactions. At the time I wrote it, I
>>>>>>>>>> hadn't
>>>>>>>>>>>>>> figured out the specific changes necessary, so it was
>>>> deliberately
>>>>>>>>>>>> vague.
>>>>>>>>>>>>>> This is the one outstanding problem I'm currently working on,
>>>> and
>>>>>>>>>>> I'll
>>>>>>>>>>>>>> update this section with more detail once I have figured out
>> the
>>>>>>>>>>> exact
>>>>>>>>>>>>>> changes required.
>>>>>>>>>>>>>> D) newTransaction() provides the necessary isolation
>> guarantees.
>>>>>>>>>>> While
>>>>>>>>>>>>>> the RocksDB implementation of transactions doesn't technically
>>>>>>>>>> *need*
>>>>>>>>>>>>>> read-only users to call newTransaction(), other
>> implementations
>>>>>>>>>>> (e.g. a
>>>>>>>>>>>>>> hypothetical PostgresStore) may require it. Calling
>>>>>>>>>> newTransaction()
>>>>>>>>>>>> when
>>>>>>>>>>>>>> no transaction is necessary is essentially free, as it will
>> just
>>>>>>>>>>> return
>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think it
>>>>>>>>>> should
>>>>>>>>>>> be
>>>>>>>>>>>>>> fairly obvious where the performance problems stem from:
>> writes
>>>>>>>>>> under
>>>>>>>>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with the
>>>>>>>>>>>>>> tombstone/record flag, 1 to decode it from the
>> tombstone/record
>>>>>>>>>> flag,
>>>>>>>>>>>>> and 1
>>>>>>>>>>>>>> to copy the record from the "temporary" store to the "main"
>>>> store,
>>>>>>>>>>> when
>>>>>>>>>>>>> the
>>>>>>>>>>>>>> transaction commits. The different approach taken by KIP-869
>>>>>>>>>> should
>>>>>>>>>>>>> perform
>>>>>>>>>>>>>> much better, as it avoids all these copies, and may actually
>>>>>>>>>> perform
>>>>>>>>>>>>>> slightly better than trunk, due to batched writes in RocksDB
>>>>>>>>>>> performing
>>>>>>>>>>>>>> better than non-batched writes.[1]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 1:
>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>
>>>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
>>>>>>>>>> lbrutschy@confluent.io
>>>>>>>>>>>>> .invalid>
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I'm just starting to read up on the whole discussion about
>>>>>>>>>> KIP-892
>>>>>>>>>>> and
>>>>>>>>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
>>>>>>>>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do have
>> some
>>>>>>>>>>>>>>> questions about the latest draft.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>      A) If I understand correctly, you propose to put a bound
>> on
>>>> the
>>>>>>>>>>>>>>> (native) memory consumed by each task. However, I wonder if
>>>> this
>>>>>>>>>> is
>>>>>>>>>>>>>>> sufficient if we have temporary imbalances in the cluster.
>> For
>>>>>>>>>>>>>>> example, depending on the timing of rebalances during a
>> cluster
>>>>>>>>>>>>>>> restart, it could happen that a single streams node is
>>>> assigned a
>>>>>>>>>>> lot
>>>>>>>>>>>>>>> more tasks than expected. With your proposed change, this
>> would
>>>>>>>>>> mean
>>>>>>>>>>>>>>> that the memory required by this one node could be a multiple
>>>> of
>>>>>>>>>>> what
>>>>>>>>>>>>>>> is required during normal operation. I wonder if it wouldn't
>> be
>>>>>>>>>>> safer
>>>>>>>>>>>>>>> to put a global bound on the memory use, across all tasks.
>>>>>>>>>>>>>>>      B) Generally, the memory concerns still give me the
>> feeling
>>>>>>> that
>>>>>>>>>>> this
>>>>>>>>>>>>>>> should not be enabled by default for all users in a minor
>>>>>>>>>> release.
>>>>>>>>>>>>>>>      C) In section "Transaction Management": the sentence "A
>>>> similar
>>>>>>>>>>>>>>> analogue will be created to automatically manage `Segment`
>>>>>>>>>>>>>>> transactions.". Maybe this is just me lacking some
>> background,
>>>>>>>>>> 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 you to find out why the KIP-844 PoC is 20x
>> slower
>>>>>>>>>> (e.g.
>>>>>>>>>>>>>>> by providing a flame graph), that would be quite interesting.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>> Lucas
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I've updated the KIP with a more detailed design, which
>>>>>>>>>> reflects
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> implementation I've been working on:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> This new design should address the outstanding points
>> already
>>>>>>>>>> made
>>>>>>>>>>>> in
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Please let me know if there are areas that are unclear or
>> need
>>>>>>>>>>> more
>>>>>>>>>>>>>>>> clarification.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I have a (nearly) working implementation. I'm confident that
>>>>>>>>>> the
>>>>>>>>>>>>>>> remaining
>>>>>>>>>>>>>>>> work (making Segments behave) will not impact the documented
>>>>>>>>>>> design.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <
>>>> colt@littlehorse.io
>>>>>>>>>>>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thank you for the reply; that makes sense. I was hoping
>> that,
>>>>>>>>>>>> since
>>>>>>>>>>>>>>> reading
>>>>>>>>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
>>>>>>>>>> documented
>>>>>>>>>>>> API,
>>>>>>>>>>>>>>> maybe
>>>>>>>>>>>>>>>>> you *wouldn't* have to wait for the next major release to
>>>>>>>>>> make
>>>>>>>>>>>> that
>>>>>>>>>>>>>>> change;
>>>>>>>>>>>>>>>>> but given 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 <
>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended to
>>>>>>>>>> try
>>>>>>>>>>> to
>>>>>>>>>>>>>>> keep the
>>>>>>>>>>>>>>>>>> behaviour unchanged as much as possible, otherwise we'd
>>>>>>>>>> have
>>>>>>>>>>> to
>>>>>>>>>>>>>>> wait for
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> major version release to land these changes.
>>>>>>>>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
>>>>>>>>>> guarantee,
>>>>>>>>>>>> and
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> typically longer commit intervals would be problematic
>> when
>>>>>>>>>>>>> reading
>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>> "committed" records.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I've been away for 5 days recovering from minor surgery,
>>>>>>>>>> but I
>>>>>>>>>>>>>>> spent a
>>>>>>>>>>>>>>>>>> considerable amount of that time working through ideas for
>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>> solutions in my head. I think your suggestion of keeping
>>>>>>>>>> ALOS
>>>>>>>>>>>>>>> as-is, but
>>>>>>>>>>>>>>>>>> buffering writes for EOS is the right path forwards,
>>>>>>>>>> although
>>>>>>>>>>> I
>>>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>> solution that both expands on this, and provides for some
>>>>>>>>>> more
>>>>>>>>>>>>>>> formal
>>>>>>>>>>>>>>>>>> guarantees.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Essentially, adding support to KeyValueStores for
>>>>>>>>>>>> "Transactions",
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>> clearly defined IsolationLevels. Using "Read Committed"
>>>>>>>>>> when
>>>>>>>>>>>> under
>>>>>>>>>>>>>>> EOS,
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> "Read Uncommitted" under ALOS.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The nice thing about this approach is that it gives us
>> much
>>>>>>>>>>> more
>>>>>>>>>>>>>>> clearly
>>>>>>>>>>>>>>>>>> defined isolation behaviour that can be properly
>>>>>>>>>> documented to
>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>>> know what to expect.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I'm still working out the kinks in the design, and will
>>>>>>>>>> update
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>>>> I have something. The main struggle is trying to implement
>>>>>>>>>>> this
>>>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>>>> making any major changes to the existing interfaces or
>>>>>>>>>>> breaking
>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>> implementations, because currently everything expects to
>>>>>>>>>>> operate
>>>>>>>>>>>>>>> directly
>>>>>>>>>>>>>>>>>> on a StateStore, and not a Transaction of that store. I
>>>>>>>>>> think
>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>> close, although sadly I won't be able to progress much
>>>>>>>>>> until
>>>>>>>>>>>> next
>>>>>>>>>>>>>>> week
>>>>>>>>>>>>>>>>> due
>>>>>>>>>>>>>>>>>> to some work commitments.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
>>>>>>>>>>> colt@littlehorse.io>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thank you for the explanation, and also for the updated
>>>>>>>>>>> KIP. I
>>>>>>>>>>>>> am
>>>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>>>> eager for this improvement to be released as it would
>>>>>>>>>>> greatly
>>>>>>>>>>>>>>> reduce
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> operational difficulties of EOS streams apps.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Two questions:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 10)
>>>>>>>>>>>>>>>>>>>> When reading records, we will use the
>>>>>>>>>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
>>>>>>>>>>>>>>>>>>>      and WriteBatchWithIndex#newIteratorWithBase
>> utilities in
>>>>>>>>>>>> order
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>> that uncommitted writes are available to query.
>>>>>>>>>>>>>>>>>>> Why do extra work to enable the reading of uncommitted
>>>>>>>>>>> writes
>>>>>>>>>>>>>>> during
>>>>>>>>>>>>>>>>> IQ?
>>>>>>>>>>>>>>>>>>> Code complexity aside, reading uncommitted writes is, in
>>>>>>>>>> my
>>>>>>>>>>>>>>> opinion, a
>>>>>>>>>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have the
>>>>>>>>>>>>> guarantee
>>>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>>>> with EOS, IQ only reads committed records. In order to
>>>>>>>>>> avoid
>>>>>>>>>>>>> dirty
>>>>>>>>>>>>>>>>> reads,
>>>>>>>>>>>>>>>>>>> one currently must query a standby replica (but this
>>>>>>>>>> still
>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>>> fully
>>>>>>>>>>>>>>>>>>> guarantee monotonic reads).
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 20) Is it also necessary to enable this optimization on
>>>>>>>>>> ALOS
>>>>>>>>>>>>>>> stores?
>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need to
>>>>>>>>>>> restore
>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
>>>>>>>>>>> acceptable
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> accept
>>>>>>>>>>>>>>>>>>> that there may have been uncommitted writes on disk. On a
>>>>>>>>>>> side
>>>>>>>>>>>>>>> note, if
>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>> enable this type of store on ALOS processors, the
>>>>>>>>>> community
>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
>>>>>>>>>>> longer
>>>>>>>>>>>>>>> include
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> StateStore management of checkpointing. This can be
>>>>>>>>>> added
>>>>>>>>>>>> as a
>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>>>> to further optimize the consistency and performance of
>>>>>>>>>>> state
>>>>>>>>>>>>>>> stores.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I've also added a section discussing some of the
>>>>>>>>>> concerns
>>>>>>>>>>>>> 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 2022 at 16:36, Nick Telford <
>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I didn't do any profiling, but the 844
>>>>>>>>>> implementation:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>        - Writes uncommitted records to a temporary
>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>> instance
>>>>>>>>>>>>>>>>>>>>>           - Since tombstones need to be flagged, all
>>>>>>>>>> record
>>>>>>>>>>>>>>> values are
>>>>>>>>>>>>>>>>>>>>>           prefixed with a value/tombstone marker. This
>>>>>>>>>>>>>>> necessitates a
>>>>>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>>>> copy.
>>>>>>>>>>>>>>>>>>>>>        - On-commit, iterates all records in this
>>>>>>>>>> temporary
>>>>>>>>>>>>>>> instance and
>>>>>>>>>>>>>>>>>>>>>        writes them to the main RocksDB store.
>>>>>>>>>>>>>>>>>>>>>        - While iterating, the value/tombstone marker
>>>>>>>>>> needs
>>>>>>>>>>> to
>>>>>>>>>>>> be
>>>>>>>>>>>>>>> parsed
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>        the real value extracted. This necessitates
>>>>>>>>>> another
>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>> copy.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> My guess is that the cost of iterating the temporary
>>>>>>>>>>>> RocksDB
>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> major factor, with the 2 extra memory copies
>>>>>>>>>> per-Record
>>>>>>>>>>>>>>>>> contributing
>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> significant amount too.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
>>>>>>>>>>>>>>> colt@littlehorse.io>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Out of curiosity, why does the performance of the
>>>>>>>>>> store
>>>>>>>>>>>>>>> degrade so
>>>>>>>>>>>>>>>>>>>>>> significantly with the 844 implementation? I
>>>>>>>>>> wouldn't
>>>>>>>>>>> be
>>>>>>>>>>>>> too
>>>>>>>>>>>>>>>>>> surprised
>>>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>> 50-60% drop (caused by each record being written
>>>>>>>>>>> twice),
>>>>>>>>>>>>> but
>>>>>>>>>>>>>>> 96%
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> extreme.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> The only thing I can think of which could create
>>>>>>>>>> such a
>>>>>>>>>>>>>>> bottleneck
>>>>>>>>>>>>>>>>>>> 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 Nick Telford <
>>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I've updated the KIP to resolve all the points
>>>>>>>>>> that
>>>>>>>>>>>> have
>>>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>>>> raised
>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>> far, with one exception: the ALOS default commit
>>>>>>>>>>>> interval
>>>>>>>>>>>>>>> of 5
>>>>>>>>>>>>>>>>>>> minutes
>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> likely to cause WriteBatchWithIndex memory to grow
>>>>>>>>>>> too
>>>>>>>>>>>>>>> large.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> There's a couple of different things I can think
>>>>>>>>>> of
>>>>>>>>>>> to
>>>>>>>>>>>>>>> solve
>>>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>        - We already have a memory/record limit in the
>>>>>>>>>> KIP
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> prevent
>>>>>>>>>>>>>>>>>> OOM
>>>>>>>>>>>>>>>>>>>>>>>        errors. Should we choose a default value for
>>>>>>>>>>> these?
>>>>>>>>>>>> My
>>>>>>>>>>>>>>>>> concern
>>>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>        anything we choose might seem rather
>>>>>>>>>> arbitrary. We
>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>>>>>>>        its behaviour such that under ALOS, it only
>>>>>>>>>>> triggers
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>        StateStore, but under EOS, it triggers a
>>>>>>>>>> commit of
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>>>>>        - We could introduce a separate `
>>>>>>>>>>>>> checkpoint.interval.ms`
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>>>>>>>>        to commit the StateStores more frequently than
>>>>>>>>>> the
>>>>>>>>>>>>>>> general
>>>>>>>>>>>>>>>>>>>>>>>        commit.interval.ms? My concern here is that
>>>>>>>>>> the
>>>>>>>>>>>>>>> semantics of
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>>>>>        would depend 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 <
>>>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hi Alex,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the feedback.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I've updated the discussion of OOM issues by
>>>>>>>>>>>> describing
>>>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>>> we'll
>>>>>>>>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>>>>>>> it. Here's the new text:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> To mitigate this, we will automatically force a
>>>>>>>>>>> Task
>>>>>>>>>>>>>>> commit if
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> total
>>>>>>>>>>>>>>>>>>>>>>>>> uncommitted records returned by
>>>>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedEntries()
>>>>>>>>>>>>> exceeds a
>>>>>>>>>>>>>>>>>>> threshold,
>>>>>>>>>>>>>>>>>>>>>>>>> configured by
>>>>>>>>>>>> max.uncommitted.state.entries.per.task;
>>>>>>>>>>>>>>> or the
>>>>>>>>>>>>>>>>>>> total
>>>>>>>>>>>>>>>>>>>>>>>>> memory used for buffering uncommitted records
>>>>>>>>>>>> returned
>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
>>>>>>>>>>> exceeds
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> threshold
>>>>>>>>>>>>>>>>>>>>>>>>> configured by
>>>>>>>>>>> max.uncommitted.state.bytes.per.task.
>>>>>>>>>>>>>>> This will
>>>>>>>>>>>>>>>>>>>> roughly
>>>>>>>>>>>>>>>>>>>>>>>>> bound the memory required per-Task for
>>>>>>>>>> buffering
>>>>>>>>>>>>>>> uncommitted
>>>>>>>>>>>>>>>>>>>> records,
>>>>>>>>>>>>>>>>>>>>>>>>> irrespective of the commit.interval.ms, and
>>>>>>>>>> will
>>>>>>>>>>>>>>> effectively
>>>>>>>>>>>>>>>>>>> bound
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> number of records that will need to be
>>>>>>>>>> restored in
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> event
>>>>>>>>>>>>>>>>>> of a
>>>>>>>>>>>>>>>>>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> These limits will be checked in
>>>>>>>>>> StreamTask#process
>>>>>>>>>>>> and
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> premature
>>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>>> will be requested via Task#requestCommit().
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Note that these new methods provide default
>>>>>>>>>>>>>>> implementations
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>>>>>>> existing custom stores and non-transactional
>>>>>>>>>>> stores
>>>>>>>>>>>>>>> (e.g.
>>>>>>>>>>>>>>>>>>>>>>>>> InMemoryKeyValueStore) do not force any early
>>>>>>>>>>>> commits.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I've chosen to have the StateStore expose
>>>>>>>>>>>>> approximations
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>>>> buffer
>>>>>>>>>>>>>>>>>>>>>>>> size/count instead of opaquely requesting a
>>>>>>>>>> commit
>>>>>>>>>>> in
>>>>>>>>>>>>>>> order to
>>>>>>>>>>>>>>>>>>>>>> delegate
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> decision making to the Task itself. This enables
>>>>>>>>>>>> Tasks
>>>>>>>>>>>>>>> to look
>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>> *all*
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> their StateStores, and determine whether an
>>>>>>>>>> early
>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> necessary.
>>>>>>>>>>>>>>>>>>>>>>>> Notably, it enables pre-Task thresholds,
>>>>>>>>>> instead of
>>>>>>>>>>>>>>> per-Store,
>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>>>> prevents Tasks with many StateStores from using
>>>>>>>>>>> much
>>>>>>>>>>>>> more
>>>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>> Tasks
>>>>>>>>>>>>>>>>>>>>>>>> with one StateStore. This makes sense, since
>>>>>>>>>>> commits
>>>>>>>>>>>>> are
>>>>>>>>>>>>>>> done
>>>>>>>>>>>>>>>>>>>> by-Task,
>>>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>> by-Store.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Prizes* for anyone who can come up with a better
>>>>>>>>>>> name
>>>>>>>>>>>>>>> for the
>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>>>>>> properties!
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for pointing out the potential
>>>>>>>>>> performance
>>>>>>>>>>>>> issues
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> WBWI.
>>>>>>>>>>>>>>>>>>>> From
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> benchmarks that user posted[1], it looks like
>>>>>>>>>> WBWI
>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>> performs
>>>>>>>>>>>>>>>>>>>>>>>> considerably better than individual puts, which
>>>>>>>>>> is
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>>>> design,
>>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>> I'd actually expect a performance boost from
>>>>>>>>>> WBWI,
>>>>>>>>>>>> just
>>>>>>>>>>>>>>> not as
>>>>>>>>>>>>>>>>>>> great
>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>> we'd get from a plain WriteBatch. This does
>>>>>>>>>> suggest
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>>>> optimization would be to use a regular
>>>>>>>>>> WriteBatch
>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> restoration
>>>>>>>>>>>>>>>>>>>> (in
>>>>>>>>>>>>>>>>>>>>>>>> RocksDBStore#restoreBatch), since we know that
>>>>>>>>>>> those
>>>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>> never
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>> queried before they're committed.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 1:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>
>>>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> * Just kidding, no prizes, sadly.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Wed, 23 Nov 2022 at 12:28, Alexander
>>>>>>>>>> Sorokoumov
>>>>>>>>>>>>>>>>>>>>>>>> <as...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 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 to keep the current
>>>>>>>>>>>> checkpointing
>>>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>>>> fallback
>>>>>>>>>>>>>>>>>>>>>>>>> mechanism. We want to keep existing users'
>>>>>>>>>>> workflows
>>>>>>>>>>>>>>> intact
>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>> can. A
>>>>>>>>>>>>>>>>>>>>>>>>> non-intrusive way would be to add a separate
>>>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>>>> method,
>>>>>>>>>>>>>>>>>>>> say,
>>>>>>>>>>>>>>>>>>>>>>>>> StateStore#managesCheckpointing(), that
>>>>>>>>>> controls
>>>>>>>>>>>>>>> whether the
>>>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>> implementation owns checkpointing.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I think that a solution to the transactional
>>>>>>>>>>> writes
>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>> address
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> OOMEs. One possible way to address that is to
>>>>>>>>>> wire
>>>>>>>>>>>>>>>>> StateStore's
>>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>>> request by adding, say, StateStore#commitNeeded
>>>>>>>>>>> that
>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> checked
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>> StreamTask#commitNeeded via the corresponding
>>>>>>>>>>>>>>>>>>>> ProcessorStateManager.
>>>>>>>>>>>>>>>>>>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>>>> that change, RocksDBStore will have to track
>>>>>>>>>> the
>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>>>> size
>>>>>>>>>>>>>>>>>>>>>>>>> and request a commit when the size goes over a
>>>>>>>>>>>>>>> (configurable)
>>>>>>>>>>>>>>>>>>>>>> threshold.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> AFAIU WriteBatchWithIndex might perform
>>>>>>>>>>>> significantly
>>>>>>>>>>>>>>> slower
>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>>>> non-txn
>>>>>>>>>>>>>>>>>>>>>>>>> puts as the batch size grows [1]. We should
>>>>>>>>>> have a
>>>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> fall
>>>>>>>>>>>>>>>>>>>>>>>>> back to the current behavior (and/or disable
>>>>>>>>>> txn
>>>>>>>>>>>>> stores
>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> ALOS)
>>>>>>>>>>>>>>>>>>>>>> unless
>>>>>>>>>>>>>>>>>>>>>>>>> the benchmarks show negligible overhead for
>>>>>>>>>> longer
>>>>>>>>>>>>>>> commits /
>>>>>>>>>>>>>>>>>>>>>>> large-enough
>>>>>>>>>>>>>>>>>>>>>>>>> batch sizes.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> If you prefer to keep the KIP smaller, I would
>>>>>>>>>>>> rather
>>>>>>>>>>>>>>> cut out
>>>>>>>>>>>>>>>>>>>>>>>>> state-store-managed checkpointing rather than
>>>>>>>>>>> proper
>>>>>>>>>>>>>>> OOMe
>>>>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>> being able to switch to non-txn behavior. The
>>>>>>>>>>>>>>> checkpointing
>>>>>>>>>>>>>>>>> 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 configuration parts as well as review
>>>>>>>>>> and
>>>>>>>>>>>>> test
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> patch.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>> Alex
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>>>> https://github.com/facebook/rocksdb/issues/608
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
>>>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the review and feedback!
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 1. Custom Stores: I've been mulling over this
>>>>>>>>>>>>> problem
>>>>>>>>>>>>>>>>> myself.
>>>>>>>>>>>>>>>>>>> As
>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>> stands,
>>>>>>>>>>>>>>>>>>>>>>>>>> custom stores would essentially lose
>>>>>>>>>>> checkpointing
>>>>>>>>>>>>>>> with no
>>>>>>>>>>>>>>>>>>>>>> indication
>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>> they're expected to make changes, besides a
>>>>>>>>>> line
>>>>>>>>>>>> in
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> release
>>>>>>>>>>>>>>>>>>>>>>> notes. I
>>>>>>>>>>>>>>>>>>>>>>>>>> agree that the best solution would be to
>>>>>>>>>>> provide a
>>>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>> checkpoints
>>>>>>>>>>>>>>>>>>>>>>>>>> to a file. The one thing I would change is
>>>>>>>>>> that
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> checkpointing
>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>> store-local file, instead of a per-Task file.
>>>>>>>>>>> This
>>>>>>>>>>>>>>> way the
>>>>>>>>>>>>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>>>> technically owns its own checkpointing (via a
>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>>>>>> implementation),
>>>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> the StateManager/Task execution engine
>>>>>>>>>> doesn't
>>>>>>>>>>>> need
>>>>>>>>>>>>>>> to know
>>>>>>>>>>>>>>>>>>>>>> anything
>>>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>>>> checkpointing, which greatly simplifies some
>>>>>>>>>> of
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> logic.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 2. OOME errors: The main reasons why I didn't
>>>>>>>>>>>>> explore
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> this is
>>>>>>>>>>>>>>>>>>>>>>>>>> a) to keep this KIP as simple as possible,
>>>>>>>>>> and
>>>>>>>>>>> b)
>>>>>>>>>>>>>>> because
>>>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>>>> how to signal that a Task should commit
>>>>>>>>>>>> prematurely.
>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>>>> confident
>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>> possible, and I think it's worth adding a
>>>>>>>>>>> section
>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>>>>>> Besides
>>>>>>>>>>>>>>>>>>>>>>>>>> my proposal to force an early commit once
>>>>>>>>>> memory
>>>>>>>>>>>>> usage
>>>>>>>>>>>>>>>>>> reaches
>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>> threshold,
>>>>>>>>>>>>>>>>>>>>>>>>>> is there any other approach that you might
>>>>>>>>>>> suggest
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> tackling
>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>> problem?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 3. ALOS: I can add in an explicit paragraph,
>>>>>>>>>> but
>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>> assumption
>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>> since transactional behaviour comes at
>>>>>>>>>> little/no
>>>>>>>>>>>>>>> cost, that
>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>> available by default on all stores,
>>>>>>>>>> irrespective
>>>>>>>>>>>> of
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>>>> mode.
>>>>>>>>>>>>>>>>>>>>>>>>>> While ALOS doesn't use transactions, the Task
>>>>>>>>>>>> itself
>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>> "commits",
>>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>> the behaviour should be correct under ALOS
>>>>>>>>>> too.
>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>> convinced
>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>>> worth having both
>>>>>>>>>>> transactional/non-transactional
>>>>>>>>>>>>>>> stores
>>>>>>>>>>>>>>>>>>>>>> available, as
>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>> would considerably increase the complexity of
>>>>>>>>>>> the
>>>>>>>>>>>>>>> codebase,
>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>> very
>>>>>>>>>>>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>>>>>>>>>>>> benefit.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> 4. Method deprecation: Are you referring to
>>>>>>>>>>>>>>>>>>>>>> StateStore#getPosition()?
>>>>>>>>>>>>>>>>>>>>>>>>> As I
>>>>>>>>>>>>>>>>>>>>>>>>>> understand it, Position contains the
>>>>>>>>>> position of
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> *source*
>>>>>>>>>>>>>>>>>>>>>> topics,
>>>>>>>>>>>>>>>>>>>>>>>>>> whereas the commit offsets would be the
>>>>>>>>>>>> *changelog*
>>>>>>>>>>>>>>>>> offsets.
>>>>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>>>> necessary to retain the Position data, as
>>>>>>>>>> well
>>>>>>>>>>> as
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> changelog
>>>>>>>>>>>>>>>>>>>>>>> offsets.
>>>>>>>>>>>>>>>>>>>>>>>>>> What I meant in the KIP is that Position
>>>>>>>>>> offsets
>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>>>>> stored
>>>>>>>>>>>>>>>>>>>>>>>>> in a
>>>>>>>>>>>>>>>>>>>>>>>>>> file, and since we can atomically store
>>>>>>>>>> metadata
>>>>>>>>>>>>>>> along with
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>>>>>> batch we commit to RocksDB, we can move our
>>>>>>>>>>>> Position
>>>>>>>>>>>>>>>>> offsets
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>> metadata too, and gain the same transactional
>>>>>>>>>>>>>>> guarantees
>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>>>> changelog offsets, ensuring that the Position
>>>>>>>>>>>>> offsets
>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>> consistent
>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>> the records that are read from the database.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, 22 Nov 2022 at 16:25, John Roesler <
>>>>>>>>>>>>>>>>>>> vvcephei@apache.org>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for publishing this alternative,
>>>>>>>>>> Nick!
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> The benchmark you mentioned in the KIP-844
>>>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>>>> compelling reason to revisit the built-in
>>>>>>>>>>>>>>>>> transactionality
>>>>>>>>>>>>>>>>>>>>>>> mechanism.
>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>> also appreciate you analysis, showing that
>>>>>>>>>> for
>>>>>>>>>>>>> most
>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>> cases,
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>>>> batch approach should be just fine.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> There are a couple of points that would
>>>>>>>>>> hold
>>>>>>>>>>> me
>>>>>>>>>>>>>>> back from
>>>>>>>>>>>>>>>>>>>>>> approving
>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP right now:
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 1. Loss of coverage for custom stores.
>>>>>>>>>>>>>>>>>>>>>>>>>>> The fact that you can plug in a
>>>>>>>>>> (relatively)
>>>>>>>>>>>>> simple
>>>>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> XStateStore interfaces and automagically
>>>>>>>>>> get a
>>>>>>>>>>>>>>>>> distributed
>>>>>>>>>>>>>>>>>>>>>> database
>>>>>>>>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>> it is a significant benefit of Kafka
>>>>>>>>>> Streams.
>>>>>>>>>>>> I'd
>>>>>>>>>>>>>>> hate to
>>>>>>>>>>>>>>>>>>> lose
>>>>>>>>>>>>>>>>>>>>>> it,
>>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>>>> would be better to spend some time and
>>>>>>>>>> come up
>>>>>>>>>>>>> with
>>>>>>>>>>>>>>> a way
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> preserve
>>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>> property. For example, can we provide a
>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>>>> `commit(..)` that re-implements the
>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>> checkpoint-file
>>>>>>>>>>>>>>>>>>>>>>>>> approach? Or
>>>>>>>>>>>>>>>>>>>>>>>>>>> perhaps add an `isTransactional()` flag to
>>>>>>>>>> the
>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>>>> that the runtime can decide whether to
>>>>>>>>>>> continue
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> manage
>>>>>>>>>>>>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>>>>>>>>>>>>> files
>>>>>>>>>>>>>>>>>>>>>>>>>>> vs delegating transactionality to the
>>>>>>>>>> stores?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 2. Guarding against OOME
>>>>>>>>>>>>>>>>>>>>>>>>>>> I appreciate your analysis, but I don't
>>>>>>>>>> think
>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>> sufficient
>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>>>> we will solve the memory problem later if
>>>>>>>>>> it
>>>>>>>>>>>>> becomes
>>>>>>>>>>>>>>>>>>> necessary.
>>>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>>>> experience leading to that situation would
>>>>>>>>>> be
>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>> bad:
>>>>>>>>>>>>>>>>>>>> Imagine,
>>>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>>>> upgrade to AK 3.next, your tests pass, so
>>>>>>>>>> you
>>>>>>>>>>>>>>> deploy to
>>>>>>>>>>>>>>>>>>>>>> production.
>>>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>>>> night, you get paged because your app is
>>>>>>>>>> now
>>>>>>>>>>>>>>> crashing
>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>> OOMEs. As
>>>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>>> all OOMEs, you'll have a really hard time
>>>>>>>>>>>> finding
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> root
>>>>>>>>>>>>>>>>>>>> cause,
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>>> once
>>>>>>>>>>>>>>>>>>>>>>>>>>> you do, you won't have a clear path to
>>>>>>>>>> resolve
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> issue.
>>>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>>>> tune down the commit interval and cache
>>>>>>>>>> buffer
>>>>>>>>>>>>> size
>>>>>>>>>>>>>>> until
>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>> stop
>>>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>>>> crashes.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> FYI, I know of multiple cases where people
>>>>>>>>>> run
>>>>>>>>>>>> EOS
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>>>> larger
>>>>>>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>>>>> intervals to get better batching than the
>>>>>>>>>>>> default,
>>>>>>>>>>>>>>> so I
>>>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>> pathological case would be as rare as you
>>>>>>>>>>>> suspect.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Given that we already have the rudiments
>>>>>>>>>> of an
>>>>>>>>>>>>> idea
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>> prevent this downside, we should take the
>>>>>>>>>> time
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> design
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>> solution.
>>>>>>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>>>> owe
>>>>>>>>>>>>>>>>>>>>>>>>>>> it to our users to ensure that awesome new
>>>>>>>>>>>>> features
>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>>> bitter
>>>>>>>>>>>>>>>>>>>>>>>>>>> pills unless we can't avoid it.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 3. ALOS mode.
>>>>>>>>>>>>>>>>>>>>>>>>>>> On the other hand, I didn't see an
>>>>>>>>>> indication
>>>>>>>>>>> of
>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>> stores
>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>>>> handled under ALOS (aka non-EOS) mode.
>>>>>>>>>>>>>>> Theoretically, the
>>>>>>>>>>>>>>>>>>>>>>>>>> transactionality
>>>>>>>>>>>>>>>>>>>>>>>>>>> of the store and the processing mode are
>>>>>>>>>>>>>>> orthogonal. A
>>>>>>>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>>>> would serve ALOS just as well as a
>>>>>>>>>>>>>>> non-transactional one
>>>>>>>>>>>>>>>>>> (if
>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>> better).
>>>>>>>>>>>>>>>>>>>>>>>>>>> Under ALOS, though, the default commit
>>>>>>>>>>> interval
>>>>>>>>>>>> is
>>>>>>>>>>>>>>> five
>>>>>>>>>>>>>>>>>>>> minutes,
>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> memory issue is far more pressing.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> As I see it, we have several options to
>>>>>>>>>>> resolve
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> point.
>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>>>> demonstrate that transactional stores work
>>>>>>>>>>> just
>>>>>>>>>>>>>>> fine for
>>>>>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>>>>>>> and we
>>>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>>> therefore just swap over unconditionally.
>>>>>>>>>> We
>>>>>>>>>>>> could
>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>>>> disable
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> transactional mechanism under ALOS so that
>>>>>>>>>>>> stores
>>>>>>>>>>>>>>> operate
>>>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>> they do today when run in ALOS mode.
>>>>>>>>>> Finally,
>>>>>>>>>>> we
>>>>>>>>>>>>>>> could do
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP-844 and make transactional stores
>>>>>>>>>> opt-in
>>>>>>>>>>>> (it'd
>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> extra opt-in mechanism, but it's a good
>>>>>>>>>>>>>>>>>> get-out-of-jail-free
>>>>>>>>>>>>>>>>>>>>>> card).
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> 4. (minor point) Deprecation of methods
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> You mentioned that the new `commit` 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?
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for the KIP! It's really nice
>>>>>>>>>>> that
>>>>>>>>>>>>> you
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> Alex
>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>> chance to collaborate on both directions so
>>>>>>>>>>> that
>>>>>>>>>>>>> we
>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> best
>>>>>>>>>>>>>>>>>>>>>>>>>>> outcome for Streams and its users.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> On 2022/11/21 15:02:15 Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>
>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>>>>>>>>>>>>>> ,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> so that it can be discussed/reviewed
>>>>>>>>>>>> separately
>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>>>> KIP-844.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Alex: I'm especially interested in what
>>>>>>>>>> you
>>>>>>>>>>>>> think!
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> I have a nearly complete implementation
>>>>>>>>>> of
>>>>>>>>>>> the
>>>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>>>>>> outlined in
>>>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>>>> KIP, please let me know if you'd like me
>>>>>>>>>> to
>>>>>>>>>>>> push
>>>>>>>>>>>>>>> them
>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>> review
>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>>>> advance
>>>>>>>>>>>>>>>>>>>>>>>>>>>> of a vote.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 

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

Posted by Nick Telford <ni...@gmail.com>.
Hi John,

I think you're referring to the "record cache" that's provided by the
ThreadCache class?

1-3.
I was hoping to (eventually) remove the "flush-on-commit" behaviour from
RocksDbStore, so that RocksDB can choose when to flush memtables, enabling
users to tailor RocksDB performance to their workload. Explicitly flushing
the Record Cache to files instead would entail either flushing on every
commit, or the current behaviour, of flushing on every commit provided at
least 10K records have been processed. Compared with RocksDB-managed
memtable flushing, this is very inflexible. If we pursue this design, I
highly recommend replacing the hard-coded 10K limit with something
configurable so that users can tune flush behaviour for their workloads.

4.
Tracking the changelog offsets in another CF and atomically updating it
with the main CFs is orthogonal, I think, as it can be done when using
memtables provided the "Atomic Flush" feature of RocksDB is enabled. This
is something I'd originally planned for this KIP, but we're trying to pull
out into a later KIP to make things more manageable.

> * we don't fragment memory between the RecordCache and the memtables
I think by memory fragmentation, you mean duplication, because we're
caching the records both in the (on-heap) Record Cache and the RocksDB
memtables? This is a good point that I hadn't considered before. Wouldn't a
simpler solution be to just disable the record cache for RocksDB stores (by
default), and let the memtables do the caching? Although I guess that would
reduce read performance, which could be especially important for joins.

> * RecordCache gives far higher performance than memtable for reads and
writes
I'll concede this point. The JNI boundary plus RocksDB record encoding will
likely make it impossible to ever match the Record Cache on throughput.

> * we don't need any new "transaction" concepts or memory bound configs
Maybe. Unless I'm mistaken, the Record Cache only retains the most recently
written value for a key, which would mean that Interactive Queries would
always observe new record values *before* they're committed to the
changelog. While this is the current behaviour, it's also a violation of
consistency, because successive IQ could observe a regression of a value,
due to an error writing to the changelog (e.g. a changelog transaction
rollback or a timeout). This is something that KIP-892 aims to improve on,
as the current design would ensure that records are only observed by IQ
*after* they have been committed to the Kafka changelog.

That said, it definitely sounds *feasible*.

Regards,

Nick

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

Posted by Nick Telford <ni...@gmail.com>.
Hi John,

By "RecordCache", do you mean the RocksDB "WriteBatch"? I can't find any
class called "RecordCache"...

Cheers,

Nick

On Tue, 20 Jun 2023 at 19:42, John Roesler <vv...@apache.org> wrote:

> Hi Nick,
>
> Thanks for picking this up again!
>
> I did have one new thought over the intervening months, which I'd like
> your take on.
>
> What if, instead of using the RocksDB atomic write primitive at all, we
> instead just:
> 1. disable memtables entirely
> 2. directly write the RecordCache into SST files when we flush
> 3. atomically ingest the SST file(s) into RocksDB when we get the ACK
> from the changelog (see
>
> https://github.com/EighteenZi/rocksdb_wiki/blob/master/Creating-and-Ingesting-SST-files.md
> and
>
> https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/IngestExternalFileOptions.java
> and
>
> https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L1413-L1429
> )
> 4. track the changelog offsets either in another CF or the same CF with
> a reserved key, either of which will make the changelog offset update
> atomic with the file ingestions
>
> I suspect this'll have a number of benefits:
> * writes to RocksDB will always be atomic
> * we don't fragment memory between the RecordCache and the memtables
> * RecordCache gives far higher performance than memtable for reads and
> writes
> * we don't need any new "transaction" concepts or memory bound configs
>
> What do you think?
>
> Thanks,
> -John
>
> On 6/20/23 10:51, Nick Telford wrote:
> > Hi Bruno,
> >
> > Thanks for reviewing the KIP. It's been a long road, I started working on
> > this more than a year ago, and most of the time in the last 6 months has
> > been spent on the "Atomic Checkpointing" stuff that's been benched, so
> some
> > of the reasoning behind some of my decisions have been lost, but I'll do
> my
> > best to reconstruct them.
> >
> > 1.
> > IIRC, this was the initial approach I tried. I don't remember the exact
> > reasons I changed it to use a separate "view" of the StateStore that
> > encapsulates the transaction, but I believe it had something to do with
> > concurrent access to the StateStore from Interactive Query threads. Reads
> > from interactive queries need to be isolated from the currently ongoing
> > transaction, both for consistency (so interactive queries don't observe
> > changes that are subsequently rolled-back), but also to prevent Iterators
> > opened by an interactive query from being closed and invalidated by the
> > StreamThread when it commits the transaction, which causes your
> interactive
> > queries to crash.
> >
> > Another reason I believe I implemented it this way was a separation of
> > concerns. Recall that newTransaction() originally created an object of
> type
> > Transaction, not StateStore. My intent was to improve the type-safety of
> > the API, in an effort to ensure Transactions weren't used incorrectly.
> > Unfortunately, this didn't pan out, but newTransaction() remained.
> >
> > Finally, this had the added benefit that implementations could easily add
> > support for transactions *without* re-writing their existing,
> > non-transactional implementation. I think this can be a benefit both for
> > implementers of custom StateStores, but also for anyone extending
> > RocksDbStore, as they can rely on the existing access methods working how
> > they expect them to.
> >
> > I'm not too happy with the way the current design has panned out, so I'm
> > open to ideas on how to improve it. Key to this is finding some way to
> > ensure that reads from Interactive Query threads are properly isolated
> from
> > the transaction, *without* the performance overhead of checking which
> > thread the method is being called from on every access.
> >
> > As for replacing flush() with commit() - I saw no reason to add this
> > complexity to the KIP, unless there was a need to add arguments to the
> > flush/commit method. This need arises with Atomic Checkpointing, but that
> > will be implemented separately, in a future KIP. Do you see a need for
> some
> > arguments to the flush/commit method that I've missed? Or were you simply
> > suggesting a rename?
> >
> > 2.
> > This is simply due to the practical reason that isolationLevel() is
> really
> > a proxy for checking if the app is under EOS. The application
> configuration
> > is not provided to the constructor of StateStores, but it *is* provided
> to
> > init(), via StateStoreContext. For this reason, it seemed somewhat
> natural
> > to add it to StateStoreContext. I think this makes sense, since the
> > IsolationLevel of all StateStores in an application *must* be the same,
> and
> > since those stores are all initialized with the same StateStoreContext,
> it
> > seems natural for that context to carry the desired IsolationLevel to
> use.
> >
> > 3.
> > Using IsolationLevel instead of just passing `boolean eosEnabled`, like
> > much of the internals was an attempt to logically de-couple the
> StateStore
> > API from the internals of Kafka Streams. Technically, StateStores don't
> > need to know/care what processing mode the KS app is using, all they need
> > to know is the isolation level expected of them.
> >
> > Having formal definitions for the expectations of the two required
> > IsolationLevels allow implementers to implement transactional stores
> > without having to dig through the internals of Kafka Streams and
> understand
> > exactly how they are used. The tight coupling between state stores and
> > internal behaviour has actually significantly hindered my progress on
> this
> > KIP, and encouraged me to avoid increasing this logical coupling as much
> as
> > possible.
> >
> > This also frees implementations to satisfy those requirements in any way
> > they choose. Transactions might not be the only/available approach to an
> > implementation, but they might have an alternative way to satisfy the
> > isolation requirements. I admit that this point is more about semantics,
> > but "transactional" would need to be formally defined in order for
> > implementers to provide a valid implementation, and these IsolationLevels
> > provide that formal definition.
> >
> > 4.
> > I can remove them. I added them only as I planned to include them in the
> > org.apache.kafka.streams.state package, as a recommended base
> > implementation for all StateStores, including those implemented by
> users. I
> > had assumed that anything in "public" packages, such as
> > org.apache.kafka.streams.state, should be included in a KIP. Is that
> wrong?
> >
> > 5.
> > RocksDB provides no way to measure the actual size of a
> > WriteBatch(WithIndex), so we're limited to tracking the sum total of the
> > size of keys + values that are written to the transaction. This obviously
> > under-estimates the actual memory usage, because WriteBatch no-doubt
> > includes some record overheads, and WriteBatchWithIndex has to maintain
> an
> > index.
> >
> > Ideally, we could trivially add a method upstream to WriteBatchInterface
> > that provides the exact size of the batch, but that would require an
> > upgrade of RocksDB, which won't happen soon. So for the time being, we're
> > stuck with an approximation, so I felt that the new method should reflect
> > that.
> >
> > Would you prefer the new method name ignores this constraint and that we
> > simply make the rocks measurement more accurate in the future?
> >
> > 6.
> > Done
> >
> > 7.
> > Very good point. The KIP already specifically calls out memory in the
> > documentation of the config: "Maximum number of memory bytes to be used
> to
> > buffer uncommitted state-store records." - did you have something else in
> > mind?
> >
> > Should we also make this clearer by renaming the config property itself?
> > Perhaps to something like statestore.transaction.buffer.max.bytes?
> >
> > 8.
> > OK, I can remove this. The intent here was to describe how Streams itself
> > will manage transaction roll-over etc. Presumably that means we also
> don't
> > need a description of how Streams will manage the commit of changelog
> > transactions, state store transactions and checkpointing?
> >
> > 9.
> > What do you mean by fail-over? Do you mean failing over an Active Task to
> > an instance already hosting a Standby Task?
> >
> > Thanks again and sorry for the essay of a response!
> >
> > Regards,
> > Nick
> >
> > On Tue, 20 Jun 2023 at 10:49, Bruno Cadonna <ca...@apache.org> wrote:
> >
> >> Hi Nick,
> >>
> >> Thanks for the updates!
> >>
> >> I really appreciate that you simplified the KIP by removing some
> >> aspects. As I have already told you, I think the removed aspects are
> >> also good ideas and we can discuss them on follow-up KIPs.
> >>
> >> Regarding the current KIP, I have the following feedback.
> >>
> >> 1.
> >> Is there a good reason to add method newTransaction() to the StateStore
> >> interface? As far as I understand, the idea is that users of a state
> >> store (transactional or not) call this method at start-up and after each
> >> commit. Since the call to newTransaction() is done in any case and I
> >> think it would simplify the caller code if we just start a new
> >> transaction after a commit in the implementation?
> >> As far as I understand, you plan to commit the transaction in the
> >> flush() method. I find the idea to replace flush() with commit()
> >> presented in KIP-844 an elegant solution.
> >>
> >> 2.
> >> Why is the method to query the isolation level added to the state store
> >> context?
> >>
> >> 3.
> >> Do we need all the isolation level definitions? I think it is good to
> >> know the guarantees of the transactionality of the state store.
> >> However, currently, Streams guarantees that there will only be one
> >> transaction that writes to the state store. Only the stream thread that
> >> executes the active task that owns the state store will write to the
> >> state store. I think it should be enough to know if the state store is
> >> transactional or not. So my proposal would be to just add a method on
> >> the state store interface the returns if a state store is transactional
> >> or not by returning a boolean or an enum.
> >>
> >> 4.
> >> I am wondering why AbstractTransaction and AbstractTransactionalStore
> >> are part of the KIP. They look like implementation details that should
> >> not be exposed in the public API.
> >>
> >> 5.
> >> Why does StateStore#approximateNumUncommittedBytes() return an
> >> approximate number of bytes?
> >>
> >> 6.
> >> RocksDB is just one implementation of the state stores in Streams.
> >> However, the issues regarding OOM errors might also apply to other
> >> custom implementations. So in the KIP I would extract that part from
> >> section "RocksDB Transaction". I would also move section "RocksDB
> >> Transaction" to the end of section "Proposed Changes" and handle it as
> >> an example implementation for a state store.
> >>
> >> 7.
> >> Should statestore.uncommitted.max.bytes only limit the uncommitted bytes
> >> or the uncommitted bytes that reside in memory? In future, other
> >> transactional state store implementations might implement a buffer for
> >> uncommitted records that are able to spill records on disk. I think
> >> statestore.uncommitted.max.bytes needs to limit the uncommitted bytes
> >> irrespective if they reside in memory or disk. Since Streams will use
> >> this config to decide if it needs to trigger a commit, state store
> >> implementations that can spill to disk will never be able to spill to
> >> disk. You would only need to change the doc of the config, if you agree
> >> with me.
> >>
> >> 8.
> >> Section "Transaction Management" about the wrappers is rather a
> >> implementation detail that should not be in the KIP.
> >>
> >> 9.
> >> Could you add a section that describes how failover will work with the
> >> transactional state stores? I think section "Error handling" is already
> >> a good start.
> >>
> >>
> >> Best,
> >> Bruno
> >>
> >>
> >>
> >>
> >> On 15.05.23 11:04, Nick Telford wrote:
> >>> Hi everyone,
> >>>
> >>> Quick update: I've added a new section to the KIP: "Offsets for
> Consumer
> >>> Rebalances", that outlines my solution to the problem that
> >>> StreamsPartitionAssignor needs to read StateStore offsets even if
> they're
> >>> not currently open.
> >>>
> >>> Regards,
> >>> Nick
> >>>
> >>> On Wed, 3 May 2023 at 11:34, Nick Telford <ni...@gmail.com>
> >> wrote:
> >>>
> >>>> Hi Bruno,
> >>>>
> >>>> Thanks for reviewing my proposal.
> >>>>
> >>>> 1.
> >>>> The main reason I added it was because it was easy to do. If we see no
> >>>> value in it, I can remove it.
> >>>>
> >>>> 2.
> >>>> Global StateStores can have multiple partitions in their input topics
> >>>> (which function as their changelogs), so they would have more than one
> >>>> partition.
> >>>>
> >>>> 3.
> >>>> That's a good point. At present, the only method it adds is
> >>>> isolationLevel(), which is likely not necessary outside of
> StateStores.
> >>>> It *does* provide slightly different guarantees in the documentation
> to
> >>>> several of the methods (hence the overrides). I'm not sure if this is
> >>>> enough to warrant a new interface though.
> >>>> I think the question that remains is whether this interface makes it
> >>>> easier to implement custom transactional StateStores than if we were
> to
> >>>> remove it? Probably not.
> >>>>
> >>>> 4.
> >>>> The main motivation for the Atomic Checkpointing is actually
> >> performance.
> >>>> My team has been testing out an implementation of this KIP without it,
> >> and
> >>>> we had problems with RocksDB doing *much* more compaction, due to the
> >>>> significantly increased flush rate. It was enough of a problem that
> (for
> >>>> the time being), we had to revert back to Kafka Streams proper.
> >>>> I think the best way to solve this, as you say, is to keep the
> >> .checkpoint
> >>>> files *in addition* to the offsets being stored within the store
> itself.
> >>>> Essentially, when closing StateStores, we force a memtable flush, then
> >>>> call getCommittedOffsets and write those out to the .checkpoint file.
> >>>> That would ensure the metadata is available to the
> >>>> StreamsPartitionAssignor for all closed stores.
> >>>> If there's a crash (no clean close), then we won't be able to
> guarantee
> >>>> which offsets were flushed to disk by RocksDB, so we'd need to open (
> >>>> init()), read offsets, and then close() those stores. But since this
> is
> >>>> the exception, and will only occur once (provided it doesn't crash
> every
> >>>> time!), I think the performance impact here would be acceptable.
> >>>>
> >>>> Thanks for the feedback, please let me know if you have any more
> >> comments
> >>>> or questions!
> >>>>
> >>>> I'm currently working on rebasing against trunk. This involves adding
> >>>> support for transactionality to VersionedStateStores. I will probably
> >> need
> >>>> to revise my implementation for transactional "segmented" stores, both
> >> to
> >>>> accommodate VersionedStateStore, and to clean up some other stuff.
> >>>>
> >>>> Regards,
> >>>> Nick
> >>>>
> >>>>
> >>>> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org>
> wrote:
> >>>>
> >>>>> Hi Nick,
> >>>>>
> >>>>> Thanks for the updates!
> >>>>>
> >>>>> I have a couple of questions/comments.
> >>>>>
> >>>>> 1.
> >>>>> Why do you propose a configuration that involves max. bytes and max.
> >>>>> reords? I think we are mainly concerned about memory consumption
> >> because
> >>>>> we want to limit the off-heap memory used. I cannot think of a case
> >>>>> where one would want to set the max. number of records.
> >>>>>
> >>>>>
> >>>>> 2.
> >>>>> Why does
> >>>>>
> >>>>>     default void commit(final Map<TopicPartition, Long>
> >> changelogOffsets) {
> >>>>>         flush();
> >>>>>     }
> >>>>>
> >>>>> take a map of partitions to changelog offsets?
> >>>>> The mapping between state stores to partitions is a 1:1 relationship.
> >>>>> Passing in a single changelog offset should suffice.
> >>>>>
> >>>>>
> >>>>> 3.
> >>>>> Why do we need the Transaction interface? It should be possible to
> hide
> >>>>> beginning and committing a transactions withing the state store
> >>>>> implementation, so that from outside the state store, it does not
> >> matter
> >>>>> whether the state store is transactional or not. What would be the
> >>>>> advantage of using the Transaction interface?
> >>>>>
> >>>>>
> >>>>> 4.
> >>>>> Regarding checkpointing offsets, I think we should keep the
> checkpoint
> >>>>> file in any case for the reason you mentioned about rebalancing. Even
> >> if
> >>>>> that would not be an issue, I would propose to move the change to
> >> offset
> >>>>> management to a new KIP and to not add more complexity than needed to
> >>>>> this one. I would not be too concerned about the consistency
> violation
> >>>>> you mention. As far as I understand, with transactional state stores
> >>>>> Streams would write the checkpoint file during every commit even
> under
> >>>>> EOS. In the failure case you describe, Streams would restore the
> state
> >>>>> stores from the offsets found in the checkpoint file written during
> the
> >>>>> penultimate commit instead of during the last commit. Basically,
> >> Streams
> >>>>> would overwrite the records written to the state store between the
> last
> >>>>> two commits with the same records read from the changelogs. While I
> >>>>> understand that this is wasteful, it is -- at the same time --
> >>>>> acceptable and most importantly it does not break EOS.
> >>>>>
> >>>>> Best,
> >>>>> Bruno
> >>>>>
> >>>>>
> >>>>> On 27.04.23 12:34, Nick Telford wrote:
> >>>>>> Hi everyone,
> >>>>>>
> >>>>>> I find myself (again) considering removing the offset management
> from
> >>>>>> StateStores, and keeping the old checkpoint file system. The reason
> is
> >>>>> that
> >>>>>> the StreamPartitionAssignor directly reads checkpoint files in order
> >> to
> >>>>>> determine which instance has the most up-to-date copy of the local
> >>>>> state.
> >>>>>> If we move offsets into the StateStore itself, then we will need to
> >>>>> open,
> >>>>>> initialize, read offsets and then close each StateStore (that is not
> >>>>>> already assigned and open) for which we have *any* local state, on
> >> every
> >>>>>> rebalance.
> >>>>>>
> >>>>>> Generally, I don't think there are many "orphan" stores like this
> >>>>> sitting
> >>>>>> around on most instances, but even a few would introduce additional
> >>>>> latency
> >>>>>> to an already somewhat lengthy rebalance procedure.
> >>>>>>
> >>>>>> I'm leaning towards Colt's (Slack) suggestion of just keeping things
> >> in
> >>>>> the
> >>>>>> checkpoint file(s) for now, and not worrying about the race. The
> >>>>> downside
> >>>>>> is that we wouldn't be able to remove the explicit RocksDB flush
> >>>>> on-commit,
> >>>>>> which likely hurts performance.
> >>>>>>
> >>>>>> If anyone has any thoughts or ideas on this subject, I would
> >> appreciate
> >>>>> it!
> >>>>>>
> >>>>>> Regards,
> >>>>>> Nick
> >>>>>>
> >>>>>> On Wed, 19 Apr 2023 at 15:05, Nick Telford <ni...@gmail.com>
> >>>>> wrote:
> >>>>>>
> >>>>>>> Hi Colt,
> >>>>>>>
> >>>>>>> The issue is that if there's a crash between 2 and 3, then you
> still
> >>>>> end
> >>>>>>> up with inconsistent data in RocksDB. The only way to guarantee
> that
> >>>>> your
> >>>>>>> checkpoint offsets and locally stored data are consistent with each
> >>>>> other
> >>>>>>> are to atomically commit them, which can be achieved by having the
> >>>>> offsets
> >>>>>>> stored in RocksDB.
> >>>>>>>
> >>>>>>> The offsets column family is likely to be extremely small (one
> >>>>>>> per-changelog partition + one per Topology input partition for
> >> regular
> >>>>>>> stores, one per input partition for global stores). So the overhead
> >>>>> will be
> >>>>>>> minimal.
> >>>>>>>
> >>>>>>> A major benefit of doing this is that we can remove the explicit
> >> calls
> >>>>> to
> >>>>>>> db.flush(), which forcibly flushes memtables to disk on-commit. It
> >>>>> turns
> >>>>>>> out, RocksDB memtable flushes are largely dictated by Kafka Streams
> >>>>>>> commits, *not* RocksDB configuration, which could be a major source
> >> of
> >>>>>>> confusion. Atomic checkpointing makes it safe to remove these
> >> explicit
> >>>>>>> flushes, because it no longer matters exactly when RocksDB flushes
> >>>>> data to
> >>>>>>> disk; since the data and corresponding checkpoint offsets will
> always
> >>>>> be
> >>>>>>> flushed together, the local store is always in a consistent state,
> >> and
> >>>>>>> on-restart, it can always safely resume restoration from the
> on-disk
> >>>>>>> offsets, restoring the small amount of data that hadn't been
> flushed
> >>>>> when
> >>>>>>> the app exited/crashed.
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>> Nick
> >>>>>>>
> >>>>>>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io>
> >>>>> wrote:
> >>>>>>>
> >>>>>>>> Nick,
> >>>>>>>>
> >>>>>>>> Thanks for your reply. Ack to A) and B).
> >>>>>>>>
> >>>>>>>> For item C), I see what you're referring to. Your proposed
> solution
> >>>>> will
> >>>>>>>> work, so no need to change it. What I was suggesting was that it
> >>>>> might be
> >>>>>>>> possible to achieve this with only one column family. So long as:
> >>>>>>>>
> >>>>>>>>       - No uncommitted records (i.e. not committed to the
> changelog)
> >> are
> >>>>>>>>       *committed* to the state store, AND
> >>>>>>>>       - The Checkpoint offset (which refers to the changelog
> topic)
> >> is
> >>>>> less
> >>>>>>>>       than or equal to the last written changelog offset in
> rocksdb
> >>>>>>>>
> >>>>>>>> I don't see the need to do the full restoration from scratch. My
> >>>>>>>> understanding was that prior to 844/892, full restorations were
> >>>>> required
> >>>>>>>> because there could be uncommitted records written to RocksDB;
> >>>>> however,
> >>>>>>>> given your use of RocksDB transactions, that can be avoided with
> the
> >>>>>>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
> >>>>> transaction, 3)
> >>>>>>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <
> >> nick.telford@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi Colt,
> >>>>>>>>>
> >>>>>>>>> A. I've done my best to de-couple the StateStore stuff from the
> >> rest
> >>>>> of
> >>>>>>>> the
> >>>>>>>>> Streams engine. The fact that there will be only one ongoing
> >> (write)
> >>>>>>>>> transaction at a time is not guaranteed by any API, and is just a
> >>>>>>>>> consequence of the way Streams operates. To that end, I tried to
> >>>>> ensure
> >>>>>>>> the
> >>>>>>>>> documentation and guarantees provided by the new APIs are
> >>>>> independent of
> >>>>>>>>> this incidental behaviour. In practice, you're right, this
> >>>>> essentially
> >>>>>>>>> refers to "interactive queries", which are technically "read
> >>>>>>>> transactions",
> >>>>>>>>> even if they don't actually use the transaction API to isolate
> >>>>>>>> themselves.
> >>>>>>>>>
> >>>>>>>>> B. Yes, although not ideal. This is for backwards compatibility,
> >>>>>>>> because:
> >>>>>>>>>        1) Existing custom StateStore implementations will
> implement
> >>>>>>>> flush(),
> >>>>>>>>> and not commit(), but the Streams engine now calls commit(), so
> >> those
> >>>>>>>> calls
> >>>>>>>>> need to be forwarded to flush() for these legacy stores.
> >>>>>>>>>        2) Existing StateStore *users*, i.e. outside of the
> Streams
> >>>>> engine
> >>>>>>>>> itself, may depend on explicitly calling flush(), so for these
> >> cases,
> >>>>>>>>> flush() needs to be redirected to call commit().
> >>>>>>>>> If anyone has a better way to guarantee compatibility without
> >>>>>>>> introducing
> >>>>>>>>> this potential recursion loop, I'm open to changes!
> >>>>>>>>>
> >>>>>>>>> C. This is described in the "Atomic Checkpointing" section.
> Offsets
> >>>>> are
> >>>>>>>>> stored in a separate RocksDB column family, which is guaranteed
> to
> >> be
> >>>>>>>>> atomically flushed to disk with all other column families. The
> >> issue
> >>>>> of
> >>>>>>>>> checkpoints being written to disk after commit causing
> >> inconsistency
> >>>>> if
> >>>>>>>> it
> >>>>>>>>> crashes in between is the reason why, under EOS, checkpoint files
> >> are
> >>>>>>>> only
> >>>>>>>>> written on clean shutdown. This is one of the major causes of
> "full
> >>>>>>>>> restorations", so moving the offsets into a place where they can
> be
> >>>>>>>>> guaranteed to be atomically written with the data they checkpoint
> >>>>>>>> allows us
> >>>>>>>>> to write the checkpoint offsets *on every commit*, not just on
> >> clean
> >>>>>>>>> shutdown.
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>> Nick
> >>>>>>>>>
> >>>>>>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io>
> >>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Nick,
> >>>>>>>>>>
> >>>>>>>>>> Thank you for continuing this work. I have a few minor
> clarifying
> >>>>>>>>>> questions.
> >>>>>>>>>>
> >>>>>>>>>> A) "Records written to any transaction are visible to all other
> >>>>>>>>>> transactions immediately." I am confused here—I thought there
> >> could
> >>>>>>>> only
> >>>>>>>>> be
> >>>>>>>>>> one transaction going on at a time for a given state store given
> >> the
> >>>>>>>>>> threading model for processing records on a Task. Do you mean
> >>>>>>>> Interactive
> >>>>>>>>>> Queries by "other transactions"? (If so, then everything makes
> >>>>> sense—I
> >>>>>>>>>> thought that since IQ were read-only then they didn't count as
> >>>>>>>>>> transactions).
> >>>>>>>>>>
> >>>>>>>>>> B) Is it intentional that the default implementations of the
> >> flush()
> >>>>>>>> and
> >>>>>>>>>> commit() methods in the StateStore class refer to each other in
> >> some
> >>>>>>>> sort
> >>>>>>>>>> of unbounded recursion?
> >>>>>>>>>>
> >>>>>>>>>> C) How will the getCommittedOffset() method work? At first I
> >> thought
> >>>>>>>> the
> >>>>>>>>>> way to do it would be using a special key in the RocksDB store
> to
> >>>>>>>> store
> >>>>>>>>> the
> >>>>>>>>>> offset, and committing that with the transaction. But upon
> second
> >>>>>>>>> thought,
> >>>>>>>>>> since restoration from the changelog is an idempotent
> procedure, I
> >>>>>>>> think
> >>>>>>>>> it
> >>>>>>>>>> would be fine to 1) commit the RocksDB transaction and then 2)
> >> write
> >>>>>>>> the
> >>>>>>>>>> offset to disk in a checkpoint file. If there is a crash between
> >> 1)
> >>>>>>>> and
> >>>>>>>>> 2),
> >>>>>>>>>> I think the only downside is 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 <
> >>>>> nick.telford@gmail.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>
> >>>>>>>>>>> I've updated the KIP to reflect the latest version of the
> design:
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>
> >>>>>>>>>>> There are several changes in there that reflect feedback from
> >> this
> >>>>>>>>>> thread,
> >>>>>>>>>>> and there's a new section and a bunch of interface changes
> >> relating
> >>>>>>>> to
> >>>>>>>>>>> Atomic Checkpointing, which is the final piece of the puzzle to
> >>>>>>>> making
> >>>>>>>>>>> everything robust.
> >>>>>>>>>>>
> >>>>>>>>>>> Let me know what you think!
> >>>>>>>>>>>
> >>>>>>>>>>> Regards,
> >>>>>>>>>>> Nick
> >>>>>>>>>>>
> >>>>>>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <
> >> nick.telford@gmail.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Hi Lucas,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks for looking over my KIP.
> >>>>>>>>>>>>
> >>>>>>>>>>>> A) The bound is per-instance, not per-Task. This was a typo in
> >> the
> >>>>>>>>> KIP
> >>>>>>>>>>>> that I've now corrected. It was originally per-Task, but I
> >>>>>>>> changed it
> >>>>>>>>>> to
> >>>>>>>>>>>> per-instance for exactly the reason you highlighted.
> >>>>>>>>>>>> B) It's worth noting that transactionality is only enabled
> under
> >>>>>>>> EOS,
> >>>>>>>>>> and
> >>>>>>>>>>>> in the default mode of operation (ALOS), there should be no
> >>>>>>>> change in
> >>>>>>>>>>>> behavior at all. I think, under EOS, we can mitigate the
> impact
> >> on
> >>>>>>>>>> users
> >>>>>>>>>>> by
> >>>>>>>>>>>> sufficiently low default values for the memory bound
> >>>>>>>> configuration. I
> >>>>>>>>>>>> understand your hesitation to include a significant change of
> >>>>>>>>>> behaviour,
> >>>>>>>>>>>> especially in a minor release, but I suspect that most users
> >> will
> >>>>>>>>>> prefer
> >>>>>>>>>>>> the memory impact (under EOS) to the existing behaviour of
> >>>>>>>> frequent
> >>>>>>>>>> state
> >>>>>>>>>>>> restorations! If this is a problem, the changes can wait until
> >> the
> >>>>>>>>> next
> >>>>>>>>>>>> major release. I'll be running a patched version of streams in
> >>>>>>>>>> production
> >>>>>>>>>>>> with these changes as soon as they're ready, so it won't
> disrupt
> >>>>>>>> me
> >>>>>>>>> :-D
> >>>>>>>>>>>> C) The main purpose of this sentence was just to note that
> some
> >>>>>>>>> changes
> >>>>>>>>>>>> will need to be made to the way Segments are handled in order
> to
> >>>>>>>>> ensure
> >>>>>>>>>>>> they also benefit from transactions. At the time I wrote it, I
> >>>>>>>> hadn't
> >>>>>>>>>>>> figured out the specific changes necessary, so it was
> >> deliberately
> >>>>>>>>>> vague.
> >>>>>>>>>>>> This is the one outstanding problem I'm currently working on,
> >> and
> >>>>>>>>> I'll
> >>>>>>>>>>>> update this section with more detail once I have figured out
> the
> >>>>>>>>> exact
> >>>>>>>>>>>> changes required.
> >>>>>>>>>>>> D) newTransaction() provides the necessary isolation
> guarantees.
> >>>>>>>>> While
> >>>>>>>>>>>> the RocksDB implementation of transactions doesn't technically
> >>>>>>>> *need*
> >>>>>>>>>>>> read-only users to call newTransaction(), other
> implementations
> >>>>>>>>> (e.g. a
> >>>>>>>>>>>> hypothetical PostgresStore) may require it. Calling
> >>>>>>>> newTransaction()
> >>>>>>>>>> when
> >>>>>>>>>>>> no transaction is necessary is essentially free, as it will
> just
> >>>>>>>>> return
> >>>>>>>>>>>> this.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think it
> >>>>>>>> should
> >>>>>>>>> be
> >>>>>>>>>>>> fairly obvious where the performance problems stem from:
> writes
> >>>>>>>> under
> >>>>>>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with the
> >>>>>>>>>>>> tombstone/record flag, 1 to decode it from the
> tombstone/record
> >>>>>>>> flag,
> >>>>>>>>>>> and 1
> >>>>>>>>>>>> to copy the record from the "temporary" store to the "main"
> >> store,
> >>>>>>>>> when
> >>>>>>>>>>> the
> >>>>>>>>>>>> transaction commits. The different approach taken by KIP-869
> >>>>>>>> should
> >>>>>>>>>>> perform
> >>>>>>>>>>>> much better, as it avoids all these copies, and may actually
> >>>>>>>> perform
> >>>>>>>>>>>> slightly better than trunk, due to batched writes in RocksDB
> >>>>>>>>> performing
> >>>>>>>>>>>> better than non-batched writes.[1]
> >>>>>>>>>>>>
> >>>>>>>>>>>> Regards,
> >>>>>>>>>>>> Nick
> >>>>>>>>>>>>
> >>>>>>>>>>>> 1:
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>
> >> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
> >>>>>>>> lbrutschy@confluent.io
> >>>>>>>>>>> .invalid>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I'm just starting to read up on the whole discussion about
> >>>>>>>> KIP-892
> >>>>>>>>> and
> >>>>>>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
> >>>>>>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do have
> some
> >>>>>>>>>>>>> questions about the latest draft.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>     A) If I understand correctly, you propose to put a bound
> on
> >> the
> >>>>>>>>>>>>> (native) memory consumed by each task. However, I wonder if
> >> this
> >>>>>>>> is
> >>>>>>>>>>>>> sufficient if we have temporary imbalances in the cluster.
> For
> >>>>>>>>>>>>> example, depending on the timing of rebalances during a
> cluster
> >>>>>>>>>>>>> restart, it could happen that a single streams node is
> >> assigned a
> >>>>>>>>> lot
> >>>>>>>>>>>>> more tasks than expected. With your proposed change, this
> would
> >>>>>>>> mean
> >>>>>>>>>>>>> that the memory required by this one node could be a multiple
> >> of
> >>>>>>>>> what
> >>>>>>>>>>>>> is required during normal operation. I wonder if it wouldn't
> be
> >>>>>>>>> safer
> >>>>>>>>>>>>> to put a global bound on the memory use, across all tasks.
> >>>>>>>>>>>>>     B) Generally, the memory concerns still give me the
> feeling
> >>>>> that
> >>>>>>>>> this
> >>>>>>>>>>>>> should not be enabled by default for all users in a minor
> >>>>>>>> release.
> >>>>>>>>>>>>>     C) In section "Transaction Management": the sentence "A
> >> similar
> >>>>>>>>>>>>> analogue will be created to automatically manage `Segment`
> >>>>>>>>>>>>> transactions.". Maybe this is just me lacking some
> background,
> >>>>>>>> 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 you to find out why the KIP-844 PoC is 20x
> slower
> >>>>>>>> (e.g.
> >>>>>>>>>>>>> by providing a flame graph), that would be quite interesting.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Cheers,
> >>>>>>>>>>>>> Lucas
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
> >>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I've updated the KIP with a more detailed design, which
> >>>>>>>> reflects
> >>>>>>>>> the
> >>>>>>>>>>>>>> implementation I've been working on:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> This new design should address the outstanding points
> already
> >>>>>>>> made
> >>>>>>>>>> in
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>> thread.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Please let me know if there are areas that are unclear or
> need
> >>>>>>>>> more
> >>>>>>>>>>>>>> clarification.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I have a (nearly) working implementation. I'm confident that
> >>>>>>>> the
> >>>>>>>>>>>>> remaining
> >>>>>>>>>>>>>> work (making Segments behave) will not impact the documented
> >>>>>>>>> design.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <
> >> colt@littlehorse.io
> >>>>>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thank you for the reply; that makes sense. I was hoping
> that,
> >>>>>>>>>> since
> >>>>>>>>>>>>> reading
> >>>>>>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
> >>>>>>>> documented
> >>>>>>>>>> API,
> >>>>>>>>>>>>> maybe
> >>>>>>>>>>>>>>> you *wouldn't* have to wait for the next major release to
> >>>>>>>> make
> >>>>>>>>>> that
> >>>>>>>>>>>>> change;
> >>>>>>>>>>>>>>> but given 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 <
> >>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended to
> >>>>>>>> try
> >>>>>>>>> to
> >>>>>>>>>>>>> keep the
> >>>>>>>>>>>>>>>> behaviour unchanged as much as possible, otherwise we'd
> >>>>>>>> have
> >>>>>>>>> to
> >>>>>>>>>>>>> wait for
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>> major version release to land these changes.
> >>>>>>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
> >>>>>>>> guarantee,
> >>>>>>>>>> and
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> typically longer commit intervals would be problematic
> when
> >>>>>>>>>>> reading
> >>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>> "committed" records.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I've been away for 5 days recovering from minor surgery,
> >>>>>>>> but I
> >>>>>>>>>>>>> spent a
> >>>>>>>>>>>>>>>> considerable amount of that time working through ideas for
> >>>>>>>>>>> possible
> >>>>>>>>>>>>>>>> solutions in my head. I think your suggestion of keeping
> >>>>>>>> ALOS
> >>>>>>>>>>>>> as-is, but
> >>>>>>>>>>>>>>>> buffering writes for EOS is the right path forwards,
> >>>>>>>> although
> >>>>>>>>> I
> >>>>>>>>>>>>> have a
> >>>>>>>>>>>>>>>> solution that both expands on this, and provides for some
> >>>>>>>> more
> >>>>>>>>>>>>> formal
> >>>>>>>>>>>>>>>> guarantees.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Essentially, adding support to KeyValueStores for
> >>>>>>>>>> "Transactions",
> >>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>> clearly defined IsolationLevels. Using "Read Committed"
> >>>>>>>> when
> >>>>>>>>>> under
> >>>>>>>>>>>>> EOS,
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>> "Read Uncommitted" under ALOS.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> The nice thing about this approach is that it gives us
> much
> >>>>>>>>> more
> >>>>>>>>>>>>> clearly
> >>>>>>>>>>>>>>>> defined isolation behaviour that can be properly
> >>>>>>>> documented to
> >>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>> users
> >>>>>>>>>>>>>>>> know what to expect.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I'm still working out the kinks in the design, and will
> >>>>>>>> update
> >>>>>>>>>> the
> >>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>> when
> >>>>>>>>>>>>>>>> I have something. The main struggle is trying to implement
> >>>>>>>>> this
> >>>>>>>>>>>>> without
> >>>>>>>>>>>>>>>> making any major changes to the existing interfaces or
> >>>>>>>>> breaking
> >>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>> implementations, because currently everything expects to
> >>>>>>>>> operate
> >>>>>>>>>>>>> directly
> >>>>>>>>>>>>>>>> on a StateStore, and not a Transaction of that store. I
> >>>>>>>> think
> >>>>>>>>>> I'm
> >>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>> close, although sadly I won't be able to progress much
> >>>>>>>> until
> >>>>>>>>>> next
> >>>>>>>>>>>>> week
> >>>>>>>>>>>>>>> due
> >>>>>>>>>>>>>>>> to some work commitments.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
> >>>>>>>>> colt@littlehorse.io>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thank you for the explanation, and also for the updated
> >>>>>>>>> KIP. I
> >>>>>>>>>>> am
> >>>>>>>>>>>>> quite
> >>>>>>>>>>>>>>>>> eager for this improvement to be released as it would
> >>>>>>>>> greatly
> >>>>>>>>>>>>> reduce
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> operational difficulties of EOS streams apps.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Two questions:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 10)
> >>>>>>>>>>>>>>>>>> When reading records, we will use the
> >>>>>>>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
> >>>>>>>>>>>>>>>>>     and WriteBatchWithIndex#newIteratorWithBase
> utilities in
> >>>>>>>>>> order
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>> that uncommitted writes are available to query.
> >>>>>>>>>>>>>>>>> Why do extra work to enable the reading of uncommitted
> >>>>>>>>> writes
> >>>>>>>>>>>>> during
> >>>>>>>>>>>>>>> IQ?
> >>>>>>>>>>>>>>>>> Code complexity aside, reading uncommitted writes is, in
> >>>>>>>> my
> >>>>>>>>>>>>> opinion, a
> >>>>>>>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have the
> >>>>>>>>>>> guarantee
> >>>>>>>>>>>>> that,
> >>>>>>>>>>>>>>>>> with EOS, IQ only reads committed records. In order to
> >>>>>>>> avoid
> >>>>>>>>>>> dirty
> >>>>>>>>>>>>>>> reads,
> >>>>>>>>>>>>>>>>> one currently must query a standby replica (but this
> >>>>>>>> still
> >>>>>>>>>>> doesn't
> >>>>>>>>>>>>>>> fully
> >>>>>>>>>>>>>>>>> guarantee monotonic reads).
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 20) Is it also necessary to enable this optimization on
> >>>>>>>> ALOS
> >>>>>>>>>>>>> stores?
> >>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need to
> >>>>>>>>> restore
> >>>>>>>>>>>>> state
> >>>>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
> >>>>>>>>> acceptable
> >>>>>>>>>> to
> >>>>>>>>>>>>> accept
> >>>>>>>>>>>>>>>>> that there may have been uncommitted writes on disk. On a
> >>>>>>>>> side
> >>>>>>>>>>>>> note, if
> >>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>> enable this type of store on ALOS processors, the
> >>>>>>>> community
> >>>>>>>>>>> would
> >>>>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
> >>>>>>>>> longer
> >>>>>>>>>>>>> include
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> StateStore management of checkpointing. This can be
> >>>>>>>> added
> >>>>>>>>>> as a
> >>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>>>> to further optimize the consistency and performance of
> >>>>>>>>> state
> >>>>>>>>>>>>> stores.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I've also added a section discussing some of the
> >>>>>>>> concerns
> >>>>>>>>>>> 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 2022 at 16:36, Nick Telford <
> >>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I didn't do any profiling, but the 844
> >>>>>>>> implementation:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>       - Writes uncommitted records to a temporary
> >>>>>>>> RocksDB
> >>>>>>>>>>>>> instance
> >>>>>>>>>>>>>>>>>>>          - Since tombstones need to be flagged, all
> >>>>>>>> record
> >>>>>>>>>>>>> values are
> >>>>>>>>>>>>>>>>>>>          prefixed with a value/tombstone marker. This
> >>>>>>>>>>>>> necessitates a
> >>>>>>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>> copy.
> >>>>>>>>>>>>>>>>>>>       - On-commit, iterates all records in this
> >>>>>>>> temporary
> >>>>>>>>>>>>> instance and
> >>>>>>>>>>>>>>>>>>>       writes them to the main RocksDB store.
> >>>>>>>>>>>>>>>>>>>       - While iterating, the value/tombstone marker
> >>>>>>>> needs
> >>>>>>>>> to
> >>>>>>>>>> be
> >>>>>>>>>>>>> parsed
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>       the real value extracted. This necessitates
> >>>>>>>> another
> >>>>>>>>>>> memory
> >>>>>>>>>>>>> copy.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> My guess is that the cost of iterating the temporary
> >>>>>>>>>> RocksDB
> >>>>>>>>>>>>> store
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> major factor, with the 2 extra memory copies
> >>>>>>>> per-Record
> >>>>>>>>>>>>>>> contributing
> >>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> significant amount too.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
> >>>>>>>>>>>>> colt@littlehorse.io>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Out of curiosity, why does the performance of the
> >>>>>>>> store
> >>>>>>>>>>>>> degrade so
> >>>>>>>>>>>>>>>>>>>> significantly with the 844 implementation? I
> >>>>>>>> wouldn't
> >>>>>>>>> be
> >>>>>>>>>>> too
> >>>>>>>>>>>>>>>> surprised
> >>>>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> 50-60% drop (caused by each record being written
> >>>>>>>>> twice),
> >>>>>>>>>>> but
> >>>>>>>>>>>>> 96%
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>> extreme.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The only thing I can think of which could create
> >>>>>>>> such a
> >>>>>>>>>>>>> bottleneck
> >>>>>>>>>>>>>>>>> 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 Nick Telford <
> >>>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I've updated the KIP to resolve all the points
> >>>>>>>> that
> >>>>>>>>>> have
> >>>>>>>>>>>>> been
> >>>>>>>>>>>>>>>> raised
> >>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>> far, with one exception: the ALOS default commit
> >>>>>>>>>> interval
> >>>>>>>>>>>>> of 5
> >>>>>>>>>>>>>>>>> minutes
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> likely to cause WriteBatchWithIndex memory to grow
> >>>>>>>>> too
> >>>>>>>>>>>>> large.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> There's a couple of different things I can think
> >>>>>>>> of
> >>>>>>>>> to
> >>>>>>>>>>>>> solve
> >>>>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>       - We already have a memory/record limit in the
> >>>>>>>> KIP
> >>>>>>>>>> to
> >>>>>>>>>>>>> prevent
> >>>>>>>>>>>>>>>> OOM
> >>>>>>>>>>>>>>>>>>>>>       errors. Should we choose a default value for
> >>>>>>>>> these?
> >>>>>>>>>> My
> >>>>>>>>>>>>>>> concern
> >>>>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>       anything we choose might seem rather
> >>>>>>>> arbitrary. We
> >>>>>>>>>>> could
> >>>>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>>>>>       its behaviour such that under ALOS, it only
> >>>>>>>>> triggers
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>       StateStore, but under EOS, it triggers a
> >>>>>>>> commit of
> >>>>>>>>>> the
> >>>>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>>>>       - We could introduce a separate `
> >>>>>>>>>>> checkpoint.interval.ms`
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>> allow
> >>>>>>>>>>>>>>>>>>>> ALOS
> >>>>>>>>>>>>>>>>>>>>>       to commit the StateStores more frequently than
> >>>>>>>> the
> >>>>>>>>>>>>> general
> >>>>>>>>>>>>>>>>>>>>>       commit.interval.ms? My concern here is that
> >>>>>>>> the
> >>>>>>>>>>>>> semantics of
> >>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> config
> >>>>>>>>>>>>>>>>>>>>>       would depend 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 <
> >>>>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hi Alex,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks for the feedback.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I've updated the discussion of OOM issues by
> >>>>>>>>>> describing
> >>>>>>>>>>>>> how
> >>>>>>>>>>>>>>>> we'll
> >>>>>>>>>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>>>>>>> it. Here's the new text:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> To mitigate this, we will automatically force a
> >>>>>>>>> Task
> >>>>>>>>>>>>> commit if
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> total
> >>>>>>>>>>>>>>>>>>>>>>> uncommitted records returned by
> >>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedEntries()
> >>>>>>>>>>> exceeds a
> >>>>>>>>>>>>>>>>> threshold,
> >>>>>>>>>>>>>>>>>>>>>>> configured by
> >>>>>>>>>> max.uncommitted.state.entries.per.task;
> >>>>>>>>>>>>> or the
> >>>>>>>>>>>>>>>>> total
> >>>>>>>>>>>>>>>>>>>>>>> memory used for buffering uncommitted records
> >>>>>>>>>> returned
> >>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
> >>>>>>>>> exceeds
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> threshold
> >>>>>>>>>>>>>>>>>>>>>>> configured by
> >>>>>>>>> max.uncommitted.state.bytes.per.task.
> >>>>>>>>>>>>> This will
> >>>>>>>>>>>>>>>>>> roughly
> >>>>>>>>>>>>>>>>>>>>>>> bound the memory required per-Task for
> >>>>>>>> buffering
> >>>>>>>>>>>>> uncommitted
> >>>>>>>>>>>>>>>>>> records,
> >>>>>>>>>>>>>>>>>>>>>>> irrespective of the commit.interval.ms, and
> >>>>>>>> will
> >>>>>>>>>>>>> effectively
> >>>>>>>>>>>>>>>>> bound
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> number of records that will need to be
> >>>>>>>> restored in
> >>>>>>>>>> the
> >>>>>>>>>>>>> event
> >>>>>>>>>>>>>>>> of a
> >>>>>>>>>>>>>>>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> These limits will be checked in
> >>>>>>>> StreamTask#process
> >>>>>>>>>> and
> >>>>>>>>>>> a
> >>>>>>>>>>>>>>>> premature
> >>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>> will be requested via Task#requestCommit().
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Note that these new methods provide default
> >>>>>>>>>>>>> implementations
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>>>>>>> existing custom stores and non-transactional
> >>>>>>>>> stores
> >>>>>>>>>>>>> (e.g.
> >>>>>>>>>>>>>>>>>>>>>>> InMemoryKeyValueStore) do not force any early
> >>>>>>>>>> commits.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I've chosen to have the StateStore expose
> >>>>>>>>>>> approximations
> >>>>>>>>>>>>> of
> >>>>>>>>>>>>>>> its
> >>>>>>>>>>>>>>>>>> buffer
> >>>>>>>>>>>>>>>>>>>>>> size/count instead of opaquely requesting a
> >>>>>>>> commit
> >>>>>>>>> in
> >>>>>>>>>>>>> order to
> >>>>>>>>>>>>>>>>>>>> delegate
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> decision making to the Task itself. This enables
> >>>>>>>>>> Tasks
> >>>>>>>>>>>>> to look
> >>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>> *all*
> >>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>> their StateStores, and determine whether an
> >>>>>>>> early
> >>>>>>>>>>> commit
> >>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> necessary.
> >>>>>>>>>>>>>>>>>>>>>> Notably, it enables pre-Task thresholds,
> >>>>>>>> instead of
> >>>>>>>>>>>>> per-Store,
> >>>>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>>>> prevents Tasks with many StateStores from using
> >>>>>>>>> much
> >>>>>>>>>>> more
> >>>>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>> Tasks
> >>>>>>>>>>>>>>>>>>>>>> with one StateStore. This makes sense, since
> >>>>>>>>> commits
> >>>>>>>>>>> are
> >>>>>>>>>>>>> done
> >>>>>>>>>>>>>>>>>> by-Task,
> >>>>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>> by-Store.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Prizes* for anyone who can come up with a better
> >>>>>>>>> name
> >>>>>>>>>>>>> for the
> >>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>>>> config
> >>>>>>>>>>>>>>>>>>>>>> properties!
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks for pointing out the potential
> >>>>>>>> performance
> >>>>>>>>>>> issues
> >>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>> WBWI.
> >>>>>>>>>>>>>>>>>> From
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>> benchmarks that user posted[1], it looks like
> >>>>>>>> WBWI
> >>>>>>>>>>> still
> >>>>>>>>>>>>>>>> performs
> >>>>>>>>>>>>>>>>>>>>>> considerably better than individual puts, which
> >>>>>>>> is
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>>>> design,
> >>>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>> I'd actually expect a performance boost from
> >>>>>>>> WBWI,
> >>>>>>>>>> just
> >>>>>>>>>>>>> not as
> >>>>>>>>>>>>>>>>> great
> >>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>> we'd get from a plain WriteBatch. This does
> >>>>>>>> suggest
> >>>>>>>>>>> that
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>>>> optimization would be to use a regular
> >>>>>>>> WriteBatch
> >>>>>>>>> for
> >>>>>>>>>>>>>>>> restoration
> >>>>>>>>>>>>>>>>>> (in
> >>>>>>>>>>>>>>>>>>>>>> RocksDBStore#restoreBatch), since we know that
> >>>>>>>>> those
> >>>>>>>>>>>>> records
> >>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>> never
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>> queried before they're committed.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 1:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>
> >> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> * Just kidding, no prizes, sadly.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Wed, 23 Nov 2022 at 12:28, Alexander
> >>>>>>>> Sorokoumov
> >>>>>>>>>>>>>>>>>>>>>> <as...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 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 to keep the current
> >>>>>>>>>> checkpointing
> >>>>>>>>>>>>> as a
> >>>>>>>>>>>>>>>>>> fallback
> >>>>>>>>>>>>>>>>>>>>>>> mechanism. We want to keep existing users'
> >>>>>>>>> workflows
> >>>>>>>>>>>>> intact
> >>>>>>>>>>>>>>> if
> >>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> can. A
> >>>>>>>>>>>>>>>>>>>>>>> non-intrusive way would be to add a separate
> >>>>>>>>>>> StateStore
> >>>>>>>>>>>>>>> method,
> >>>>>>>>>>>>>>>>>> say,
> >>>>>>>>>>>>>>>>>>>>>>> StateStore#managesCheckpointing(), that
> >>>>>>>> controls
> >>>>>>>>>>>>> whether the
> >>>>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>> implementation owns checkpointing.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I think that a solution to the transactional
> >>>>>>>>> writes
> >>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>> address
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> OOMEs. One possible way to address that is to
> >>>>>>>> wire
> >>>>>>>>>>>>>>> StateStore's
> >>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>> request by adding, say, StateStore#commitNeeded
> >>>>>>>>> that
> >>>>>>>>>>> is
> >>>>>>>>>>>>>>> checked
> >>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>> StreamTask#commitNeeded via the corresponding
> >>>>>>>>>>>>>>>>>> ProcessorStateManager.
> >>>>>>>>>>>>>>>>>>>>> With
> >>>>>>>>>>>>>>>>>>>>>>> that change, RocksDBStore will have to track
> >>>>>>>> the
> >>>>>>>>>>> current
> >>>>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>>>> size
> >>>>>>>>>>>>>>>>>>>>>>> and request a commit when the size goes over a
> >>>>>>>>>>>>> (configurable)
> >>>>>>>>>>>>>>>>>>>> threshold.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> AFAIU WriteBatchWithIndex might perform
> >>>>>>>>>> significantly
> >>>>>>>>>>>>> slower
> >>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>>>> non-txn
> >>>>>>>>>>>>>>>>>>>>>>> puts as the batch size grows [1]. We should
> >>>>>>>> have a
> >>>>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> fall
> >>>>>>>>>>>>>>>>>>>>>>> back to the current behavior (and/or disable
> >>>>>>>> txn
> >>>>>>>>>>> stores
> >>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>> ALOS)
> >>>>>>>>>>>>>>>>>>>> unless
> >>>>>>>>>>>>>>>>>>>>>>> the benchmarks show negligible overhead for
> >>>>>>>> longer
> >>>>>>>>>>>>> commits /
> >>>>>>>>>>>>>>>>>>>>> large-enough
> >>>>>>>>>>>>>>>>>>>>>>> batch sizes.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> If you prefer to keep the KIP smaller, I would
> >>>>>>>>>> rather
> >>>>>>>>>>>>> cut out
> >>>>>>>>>>>>>>>>>>>>>>> state-store-managed checkpointing rather than
> >>>>>>>>> proper
> >>>>>>>>>>>>> OOMe
> >>>>>>>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>> being able to switch to non-txn behavior. The
> >>>>>>>>>>>>> checkpointing
> >>>>>>>>>>>>>>> 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 configuration parts as well as review
> >>>>>>>> and
> >>>>>>>>>>> test
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> patch.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>>>> Alex
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 1.
> >>>>>>>> https://github.com/facebook/rocksdb/issues/608
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
> >>>>>>>>>>>>>>>>>> nick.telford@gmail.com
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Hi John,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Thanks for the review and feedback!
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 1. Custom Stores: I've been mulling over this
> >>>>>>>>>>> problem
> >>>>>>>>>>>>>>> myself.
> >>>>>>>>>>>>>>>>> As
> >>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>> stands,
> >>>>>>>>>>>>>>>>>>>>>>>> custom stores would essentially lose
> >>>>>>>>> checkpointing
> >>>>>>>>>>>>> with no
> >>>>>>>>>>>>>>>>>>>> indication
> >>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>> they're expected to make changes, besides a
> >>>>>>>> line
> >>>>>>>>>> in
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> release
> >>>>>>>>>>>>>>>>>>>>> notes. I
> >>>>>>>>>>>>>>>>>>>>>>>> agree that the best solution would be to
> >>>>>>>>> provide a
> >>>>>>>>>>>>> default
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>> checkpoints
> >>>>>>>>>>>>>>>>>>>>>>>> to a file. The one thing I would change is
> >>>>>>>> that
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>>> checkpointing
> >>>>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>> store-local file, instead of a per-Task file.
> >>>>>>>>> This
> >>>>>>>>>>>>> way the
> >>>>>>>>>>>>>>>>>>>> StateStore
> >>>>>>>>>>>>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>>>>>>>> technically owns its own checkpointing (via a
> >>>>>>>>>>> default
> >>>>>>>>>>>>>>>>>>>> implementation),
> >>>>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>> the StateManager/Task execution engine
> >>>>>>>> doesn't
> >>>>>>>>>> need
> >>>>>>>>>>>>> to know
> >>>>>>>>>>>>>>>>>>>> anything
> >>>>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>>>> checkpointing, which greatly simplifies some
> >>>>>>>> of
> >>>>>>>>>> the
> >>>>>>>>>>>>> logic.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 2. OOME errors: The main reasons why I didn't
> >>>>>>>>>>> explore
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> this is
> >>>>>>>>>>>>>>>>>>>>>>>> a) to keep this KIP as simple as possible,
> >>>>>>>> and
> >>>>>>>>> b)
> >>>>>>>>>>>>> because
> >>>>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>>>>>>>>>>>>> how to signal that a Task should commit
> >>>>>>>>>> prematurely.
> >>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>>>> confident
> >>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>> possible, and I think it's worth adding a
> >>>>>>>>> section
> >>>>>>>>>> on
> >>>>>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>>>>>>>> Besides
> >>>>>>>>>>>>>>>>>>>>>>>> my proposal to force an early commit once
> >>>>>>>> memory
> >>>>>>>>>>> usage
> >>>>>>>>>>>>>>>> reaches
> >>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>> threshold,
> >>>>>>>>>>>>>>>>>>>>>>>> is there any other approach that you might
> >>>>>>>>> suggest
> >>>>>>>>>>> for
> >>>>>>>>>>>>>>>> tackling
> >>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>> problem?
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 3. ALOS: I can add in an explicit paragraph,
> >>>>>>>> but
> >>>>>>>>>> my
> >>>>>>>>>>>>>>>> assumption
> >>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>> since transactional behaviour comes at
> >>>>>>>> little/no
> >>>>>>>>>>>>> cost, that
> >>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>> available by default on all stores,
> >>>>>>>> irrespective
> >>>>>>>>>> of
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>>>> mode.
> >>>>>>>>>>>>>>>>>>>>>>>> While ALOS doesn't use transactions, the Task
> >>>>>>>>>> itself
> >>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>>>> "commits",
> >>>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>>>> the behaviour should be correct under ALOS
> >>>>>>>> too.
> >>>>>>>>>> I'm
> >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>> convinced
> >>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>>> worth having both
> >>>>>>>>> transactional/non-transactional
> >>>>>>>>>>>>> stores
> >>>>>>>>>>>>>>>>>>>> available, as
> >>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>> would considerably increase the complexity of
> >>>>>>>>> the
> >>>>>>>>>>>>> codebase,
> >>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>> very
> >>>>>>>>>>>>>>>>>>>>>>> little
> >>>>>>>>>>>>>>>>>>>>>>>> benefit.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> 4. Method deprecation: Are you referring to
> >>>>>>>>>>>>>>>>>>>> StateStore#getPosition()?
> >>>>>>>>>>>>>>>>>>>>>>> As I
> >>>>>>>>>>>>>>>>>>>>>>>> understand it, Position contains the
> >>>>>>>> position of
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> *source*
> >>>>>>>>>>>>>>>>>>>> topics,
> >>>>>>>>>>>>>>>>>>>>>>>> whereas the commit offsets would be the
> >>>>>>>>>> *changelog*
> >>>>>>>>>>>>>>> offsets.
> >>>>>>>>>>>>>>>> So
> >>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>>>>>>>> necessary to retain the Position data, as
> >>>>>>>> well
> >>>>>>>>> as
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> changelog
> >>>>>>>>>>>>>>>>>>>>> offsets.
> >>>>>>>>>>>>>>>>>>>>>>>> What I meant in the KIP is that Position
> >>>>>>>> offsets
> >>>>>>>>>> are
> >>>>>>>>>>>>>>>> currently
> >>>>>>>>>>>>>>>>>>>> stored
> >>>>>>>>>>>>>>>>>>>>>>> in a
> >>>>>>>>>>>>>>>>>>>>>>>> file, and since we can atomically store
> >>>>>>>> metadata
> >>>>>>>>>>>>> along with
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>>>>>>>> batch we commit to RocksDB, we can move our
> >>>>>>>>>> Position
> >>>>>>>>>>>>>>> offsets
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>> metadata too, and gain the same transactional
> >>>>>>>>>>>>> guarantees
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>>>> changelog offsets, ensuring that the Position
> >>>>>>>>>>> offsets
> >>>>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>>>> consistent
> >>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>> the records that are read from the database.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> On Tue, 22 Nov 2022 at 16:25, John Roesler <
> >>>>>>>>>>>>>>>>> vvcephei@apache.org>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks for publishing this alternative,
> >>>>>>>> Nick!
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> The benchmark you mentioned in the KIP-844
> >>>>>>>>>>>>> discussion
> >>>>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>>>> compelling reason to revisit the built-in
> >>>>>>>>>>>>>>> transactionality
> >>>>>>>>>>>>>>>>>>>>> mechanism.
> >>>>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>>>> also appreciate you analysis, showing that
> >>>>>>>> for
> >>>>>>>>>>> most
> >>>>>>>>>>>>> use
> >>>>>>>>>>>>>>>>> cases,
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>>>> batch approach should be just fine.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> There are a couple of points that would
> >>>>>>>> hold
> >>>>>>>>> me
> >>>>>>>>>>>>> back from
> >>>>>>>>>>>>>>>>>>>> approving
> >>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>> KIP right now:
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 1. Loss of coverage for custom stores.
> >>>>>>>>>>>>>>>>>>>>>>>>> The fact that you can plug in a
> >>>>>>>> (relatively)
> >>>>>>>>>>> simple
> >>>>>>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> XStateStore interfaces and automagically
> >>>>>>>> get a
> >>>>>>>>>>>>>>> distributed
> >>>>>>>>>>>>>>>>>>>> database
> >>>>>>>>>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>> it is a significant benefit of Kafka
> >>>>>>>> Streams.
> >>>>>>>>>> I'd
> >>>>>>>>>>>>> hate to
> >>>>>>>>>>>>>>>>> lose
> >>>>>>>>>>>>>>>>>>>> it,
> >>>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>>>> would be better to spend some time and
> >>>>>>>> come up
> >>>>>>>>>>> with
> >>>>>>>>>>>>> a way
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> preserve
> >>>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>> property. For example, can we provide a
> >>>>>>>>> default
> >>>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>>>> `commit(..)` that re-implements the
> >>>>>>>> existing
> >>>>>>>>>>>>>>>> checkpoint-file
> >>>>>>>>>>>>>>>>>>>>>>> approach? Or
> >>>>>>>>>>>>>>>>>>>>>>>>> perhaps add an `isTransactional()` flag to
> >>>>>>>> the
> >>>>>>>>>>> state
> >>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>>>>> that the runtime can decide whether to
> >>>>>>>>> continue
> >>>>>>>>>> to
> >>>>>>>>>>>>> manage
> >>>>>>>>>>>>>>>>>>>> checkpoint
> >>>>>>>>>>>>>>>>>>>>>>>> files
> >>>>>>>>>>>>>>>>>>>>>>>>> vs delegating transactionality to the
> >>>>>>>> stores?
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 2. Guarding against OOME
> >>>>>>>>>>>>>>>>>>>>>>>>> I appreciate your analysis, but I don't
> >>>>>>>> think
> >>>>>>>>>> it's
> >>>>>>>>>>>>>>>> sufficient
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>>>> we will solve the memory problem later if
> >>>>>>>> it
> >>>>>>>>>>> becomes
> >>>>>>>>>>>>>>>>> necessary.
> >>>>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>>>>> experience leading to that situation would
> >>>>>>>> be
> >>>>>>>>>>> quite
> >>>>>>>>>>>>> bad:
> >>>>>>>>>>>>>>>>>> Imagine,
> >>>>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>>>> upgrade to AK 3.next, your tests pass, so
> >>>>>>>> you
> >>>>>>>>>>>>> deploy to
> >>>>>>>>>>>>>>>>>>>> production.
> >>>>>>>>>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>>>> night, you get paged because your app is
> >>>>>>>> now
> >>>>>>>>>>>>> crashing
> >>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>> OOMEs. As
> >>>>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>>> all OOMEs, you'll have a really hard time
> >>>>>>>>>> finding
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> root
> >>>>>>>>>>>>>>>>>> cause,
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>>>> once
> >>>>>>>>>>>>>>>>>>>>>>>>> you do, you won't have a clear path to
> >>>>>>>> resolve
> >>>>>>>>>> the
> >>>>>>>>>>>>> issue.
> >>>>>>>>>>>>>>>> You
> >>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>>>> tune down the commit interval and cache
> >>>>>>>> buffer
> >>>>>>>>>>> size
> >>>>>>>>>>>>> until
> >>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>> stop
> >>>>>>>>>>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>>>>>>>>>> crashes.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> FYI, I know of multiple cases where people
> >>>>>>>> run
> >>>>>>>>>> EOS
> >>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>>>> larger
> >>>>>>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>>>> intervals to get better batching than the
> >>>>>>>>>> default,
> >>>>>>>>>>>>> so I
> >>>>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>> pathological case would be as rare as you
> >>>>>>>>>> suspect.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Given that we already have the rudiments
> >>>>>>>> of an
> >>>>>>>>>>> idea
> >>>>>>>>>>>>> of
> >>>>>>>>>>>>>>> what
> >>>>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>> do
> >>>>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>>>> prevent this downside, we should take the
> >>>>>>>> time
> >>>>>>>>>> to
> >>>>>>>>>>>>> design
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> solution.
> >>>>>>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>>>> owe
> >>>>>>>>>>>>>>>>>>>>>>>>> it to our users to ensure that awesome new
> >>>>>>>>>>> features
> >>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>>> bitter
> >>>>>>>>>>>>>>>>>>>>>>>>> pills unless we can't avoid it.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 3. ALOS mode.
> >>>>>>>>>>>>>>>>>>>>>>>>> On the other hand, I didn't see an
> >>>>>>>> indication
> >>>>>>>>> of
> >>>>>>>>>>> how
> >>>>>>>>>>>>>>> stores
> >>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>>>> handled under ALOS (aka non-EOS) mode.
> >>>>>>>>>>>>> Theoretically, the
> >>>>>>>>>>>>>>>>>>>>>>>> transactionality
> >>>>>>>>>>>>>>>>>>>>>>>>> of the store and the processing mode are
> >>>>>>>>>>>>> orthogonal. A
> >>>>>>>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>>>> would serve ALOS just as well as a
> >>>>>>>>>>>>> non-transactional one
> >>>>>>>>>>>>>>>> (if
> >>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>>>> better).
> >>>>>>>>>>>>>>>>>>>>>>>>> Under ALOS, though, the default commit
> >>>>>>>>> interval
> >>>>>>>>>> is
> >>>>>>>>>>>>> five
> >>>>>>>>>>>>>>>>>> minutes,
> >>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> memory issue is far more pressing.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> As I see it, we have several options to
> >>>>>>>>> resolve
> >>>>>>>>>>> this
> >>>>>>>>>>>>>>> point.
> >>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>>>> demonstrate that transactional stores work
> >>>>>>>>> just
> >>>>>>>>>>>>> fine for
> >>>>>>>>>>>>>>>> ALOS
> >>>>>>>>>>>>>>>>>>>> and we
> >>>>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>>>> therefore just swap over unconditionally.
> >>>>>>>> We
> >>>>>>>>>> could
> >>>>>>>>>>>>> also
> >>>>>>>>>>>>>>>>> disable
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> transactional mechanism under ALOS so that
> >>>>>>>>>> stores
> >>>>>>>>>>>>> operate
> >>>>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>>>> they do today when run in ALOS mode.
> >>>>>>>> Finally,
> >>>>>>>>> we
> >>>>>>>>>>>>> could do
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>> KIP-844 and make transactional stores
> >>>>>>>> opt-in
> >>>>>>>>>> (it'd
> >>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> extra opt-in mechanism, but it's a good
> >>>>>>>>>>>>>>>> get-out-of-jail-free
> >>>>>>>>>>>>>>>>>>>> card).
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> 4. (minor point) Deprecation of methods
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> You mentioned that the new `commit` 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?
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for the KIP! It's really nice
> >>>>>>>>> that
> >>>>>>>>>>> you
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>> Alex
> >>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>>>> chance to collaborate on both directions so
> >>>>>>>>> that
> >>>>>>>>>>> we
> >>>>>>>>>>>>> can
> >>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> best
> >>>>>>>>>>>>>>>>>>>>>>>>> outcome for Streams and its users.
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>> On 2022/11/21 15:02:15 Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>>>>>>>>>>>>>> ,
> >>>>>>>>>>>>>>>>>>>>>>>>>> so that it can be discussed/reviewed
> >>>>>>>>>> separately
> >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>>>> KIP-844.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Alex: I'm especially interested in what
> >>>>>>>> you
> >>>>>>>>>>> think!
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> I have a nearly complete implementation
> >>>>>>>> of
> >>>>>>>>> the
> >>>>>>>>>>>>> changes
> >>>>>>>>>>>>>>>>>>>> outlined in
> >>>>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>>>> KIP, please let me know if you'd like me
> >>>>>>>> to
> >>>>>>>>>> push
> >>>>>>>>>>>>> them
> >>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>> review
> >>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>>>> advance
> >>>>>>>>>>>>>>>>>>>>>>>>>> of a vote.
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>

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

Posted by John Roesler <vv...@apache.org>.
Hi Nick,

Thanks for picking this up again!

I did have one new thought over the intervening months, which I'd like 
your take on.

What if, instead of using the RocksDB atomic write primitive at all, we 
instead just:
1. disable memtables entirely
2. directly write the RecordCache into SST files when we flush
3. atomically ingest the SST file(s) into RocksDB when we get the ACK 
from the changelog (see 
https://github.com/EighteenZi/rocksdb_wiki/blob/master/Creating-and-Ingesting-SST-files.md 
and 
https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/IngestExternalFileOptions.java 
and 
https://github.com/facebook/rocksdb/blob/master/include/rocksdb/db.h#L1413-L1429)
4. track the changelog offsets either in another CF or the same CF with 
a reserved key, either of which will make the changelog offset update 
atomic with the file ingestions

I suspect this'll have a number of benefits:
* writes to RocksDB will always be atomic
* we don't fragment memory between the RecordCache and the memtables
* RecordCache gives far higher performance than memtable for reads and 
writes
* we don't need any new "transaction" concepts or memory bound configs

What do you think?

Thanks,
-John

On 6/20/23 10:51, Nick Telford wrote:
> Hi Bruno,
> 
> Thanks for reviewing the KIP. It's been a long road, I started working on
> this more than a year ago, and most of the time in the last 6 months has
> been spent on the "Atomic Checkpointing" stuff that's been benched, so some
> of the reasoning behind some of my decisions have been lost, but I'll do my
> best to reconstruct them.
> 
> 1.
> IIRC, this was the initial approach I tried. I don't remember the exact
> reasons I changed it to use a separate "view" of the StateStore that
> encapsulates the transaction, but I believe it had something to do with
> concurrent access to the StateStore from Interactive Query threads. Reads
> from interactive queries need to be isolated from the currently ongoing
> transaction, both for consistency (so interactive queries don't observe
> changes that are subsequently rolled-back), but also to prevent Iterators
> opened by an interactive query from being closed and invalidated by the
> StreamThread when it commits the transaction, which causes your interactive
> queries to crash.
> 
> Another reason I believe I implemented it this way was a separation of
> concerns. Recall that newTransaction() originally created an object of type
> Transaction, not StateStore. My intent was to improve the type-safety of
> the API, in an effort to ensure Transactions weren't used incorrectly.
> Unfortunately, this didn't pan out, but newTransaction() remained.
> 
> Finally, this had the added benefit that implementations could easily add
> support for transactions *without* re-writing their existing,
> non-transactional implementation. I think this can be a benefit both for
> implementers of custom StateStores, but also for anyone extending
> RocksDbStore, as they can rely on the existing access methods working how
> they expect them to.
> 
> I'm not too happy with the way the current design has panned out, so I'm
> open to ideas on how to improve it. Key to this is finding some way to
> ensure that reads from Interactive Query threads are properly isolated from
> the transaction, *without* the performance overhead of checking which
> thread the method is being called from on every access.
> 
> As for replacing flush() with commit() - I saw no reason to add this
> complexity to the KIP, unless there was a need to add arguments to the
> flush/commit method. This need arises with Atomic Checkpointing, but that
> will be implemented separately, in a future KIP. Do you see a need for some
> arguments to the flush/commit method that I've missed? Or were you simply
> suggesting a rename?
> 
> 2.
> This is simply due to the practical reason that isolationLevel() is really
> a proxy for checking if the app is under EOS. The application configuration
> is not provided to the constructor of StateStores, but it *is* provided to
> init(), via StateStoreContext. For this reason, it seemed somewhat natural
> to add it to StateStoreContext. I think this makes sense, since the
> IsolationLevel of all StateStores in an application *must* be the same, and
> since those stores are all initialized with the same StateStoreContext, it
> seems natural for that context to carry the desired IsolationLevel to use.
> 
> 3.
> Using IsolationLevel instead of just passing `boolean eosEnabled`, like
> much of the internals was an attempt to logically de-couple the StateStore
> API from the internals of Kafka Streams. Technically, StateStores don't
> need to know/care what processing mode the KS app is using, all they need
> to know is the isolation level expected of them.
> 
> Having formal definitions for the expectations of the two required
> IsolationLevels allow implementers to implement transactional stores
> without having to dig through the internals of Kafka Streams and understand
> exactly how they are used. The tight coupling between state stores and
> internal behaviour has actually significantly hindered my progress on this
> KIP, and encouraged me to avoid increasing this logical coupling as much as
> possible.
> 
> This also frees implementations to satisfy those requirements in any way
> they choose. Transactions might not be the only/available approach to an
> implementation, but they might have an alternative way to satisfy the
> isolation requirements. I admit that this point is more about semantics,
> but "transactional" would need to be formally defined in order for
> implementers to provide a valid implementation, and these IsolationLevels
> provide that formal definition.
> 
> 4.
> I can remove them. I added them only as I planned to include them in the
> org.apache.kafka.streams.state package, as a recommended base
> implementation for all StateStores, including those implemented by users. I
> had assumed that anything in "public" packages, such as
> org.apache.kafka.streams.state, should be included in a KIP. Is that wrong?
> 
> 5.
> RocksDB provides no way to measure the actual size of a
> WriteBatch(WithIndex), so we're limited to tracking the sum total of the
> size of keys + values that are written to the transaction. This obviously
> under-estimates the actual memory usage, because WriteBatch no-doubt
> includes some record overheads, and WriteBatchWithIndex has to maintain an
> index.
> 
> Ideally, we could trivially add a method upstream to WriteBatchInterface
> that provides the exact size of the batch, but that would require an
> upgrade of RocksDB, which won't happen soon. So for the time being, we're
> stuck with an approximation, so I felt that the new method should reflect
> that.
> 
> Would you prefer the new method name ignores this constraint and that we
> simply make the rocks measurement more accurate in the future?
> 
> 6.
> Done
> 
> 7.
> Very good point. The KIP already specifically calls out memory in the
> documentation of the config: "Maximum number of memory bytes to be used to
> buffer uncommitted state-store records." - did you have something else in
> mind?
> 
> Should we also make this clearer by renaming the config property itself?
> Perhaps to something like statestore.transaction.buffer.max.bytes?
> 
> 8.
> OK, I can remove this. The intent here was to describe how Streams itself
> will manage transaction roll-over etc. Presumably that means we also don't
> need a description of how Streams will manage the commit of changelog
> transactions, state store transactions and checkpointing?
> 
> 9.
> What do you mean by fail-over? Do you mean failing over an Active Task to
> an instance already hosting a Standby Task?
> 
> Thanks again and sorry for the essay of a response!
> 
> Regards,
> Nick
> 
> On Tue, 20 Jun 2023 at 10:49, Bruno Cadonna <ca...@apache.org> wrote:
> 
>> Hi Nick,
>>
>> Thanks for the updates!
>>
>> I really appreciate that you simplified the KIP by removing some
>> aspects. As I have already told you, I think the removed aspects are
>> also good ideas and we can discuss them on follow-up KIPs.
>>
>> Regarding the current KIP, I have the following feedback.
>>
>> 1.
>> Is there a good reason to add method newTransaction() to the StateStore
>> interface? As far as I understand, the idea is that users of a state
>> store (transactional or not) call this method at start-up and after each
>> commit. Since the call to newTransaction() is done in any case and I
>> think it would simplify the caller code if we just start a new
>> transaction after a commit in the implementation?
>> As far as I understand, you plan to commit the transaction in the
>> flush() method. I find the idea to replace flush() with commit()
>> presented in KIP-844 an elegant solution.
>>
>> 2.
>> Why is the method to query the isolation level added to the state store
>> context?
>>
>> 3.
>> Do we need all the isolation level definitions? I think it is good to
>> know the guarantees of the transactionality of the state store.
>> However, currently, Streams guarantees that there will only be one
>> transaction that writes to the state store. Only the stream thread that
>> executes the active task that owns the state store will write to the
>> state store. I think it should be enough to know if the state store is
>> transactional or not. So my proposal would be to just add a method on
>> the state store interface the returns if a state store is transactional
>> or not by returning a boolean or an enum.
>>
>> 4.
>> I am wondering why AbstractTransaction and AbstractTransactionalStore
>> are part of the KIP. They look like implementation details that should
>> not be exposed in the public API.
>>
>> 5.
>> Why does StateStore#approximateNumUncommittedBytes() return an
>> approximate number of bytes?
>>
>> 6.
>> RocksDB is just one implementation of the state stores in Streams.
>> However, the issues regarding OOM errors might also apply to other
>> custom implementations. So in the KIP I would extract that part from
>> section "RocksDB Transaction". I would also move section "RocksDB
>> Transaction" to the end of section "Proposed Changes" and handle it as
>> an example implementation for a state store.
>>
>> 7.
>> Should statestore.uncommitted.max.bytes only limit the uncommitted bytes
>> or the uncommitted bytes that reside in memory? In future, other
>> transactional state store implementations might implement a buffer for
>> uncommitted records that are able to spill records on disk. I think
>> statestore.uncommitted.max.bytes needs to limit the uncommitted bytes
>> irrespective if they reside in memory or disk. Since Streams will use
>> this config to decide if it needs to trigger a commit, state store
>> implementations that can spill to disk will never be able to spill to
>> disk. You would only need to change the doc of the config, if you agree
>> with me.
>>
>> 8.
>> Section "Transaction Management" about the wrappers is rather a
>> implementation detail that should not be in the KIP.
>>
>> 9.
>> Could you add a section that describes how failover will work with the
>> transactional state stores? I think section "Error handling" is already
>> a good start.
>>
>>
>> Best,
>> Bruno
>>
>>
>>
>>
>> On 15.05.23 11:04, Nick Telford wrote:
>>> Hi everyone,
>>>
>>> Quick update: I've added a new section to the KIP: "Offsets for Consumer
>>> Rebalances", that outlines my solution to the problem that
>>> StreamsPartitionAssignor needs to read StateStore offsets even if they're
>>> not currently open.
>>>
>>> Regards,
>>> Nick
>>>
>>> On Wed, 3 May 2023 at 11:34, Nick Telford <ni...@gmail.com>
>> wrote:
>>>
>>>> Hi Bruno,
>>>>
>>>> Thanks for reviewing my proposal.
>>>>
>>>> 1.
>>>> The main reason I added it was because it was easy to do. If we see no
>>>> value in it, I can remove it.
>>>>
>>>> 2.
>>>> Global StateStores can have multiple partitions in their input topics
>>>> (which function as their changelogs), so they would have more than one
>>>> partition.
>>>>
>>>> 3.
>>>> That's a good point. At present, the only method it adds is
>>>> isolationLevel(), which is likely not necessary outside of StateStores.
>>>> It *does* provide slightly different guarantees in the documentation to
>>>> several of the methods (hence the overrides). I'm not sure if this is
>>>> enough to warrant a new interface though.
>>>> I think the question that remains is whether this interface makes it
>>>> easier to implement custom transactional StateStores than if we were to
>>>> remove it? Probably not.
>>>>
>>>> 4.
>>>> The main motivation for the Atomic Checkpointing is actually
>> performance.
>>>> My team has been testing out an implementation of this KIP without it,
>> and
>>>> we had problems with RocksDB doing *much* more compaction, due to the
>>>> significantly increased flush rate. It was enough of a problem that (for
>>>> the time being), we had to revert back to Kafka Streams proper.
>>>> I think the best way to solve this, as you say, is to keep the
>> .checkpoint
>>>> files *in addition* to the offsets being stored within the store itself.
>>>> Essentially, when closing StateStores, we force a memtable flush, then
>>>> call getCommittedOffsets and write those out to the .checkpoint file.
>>>> That would ensure the metadata is available to the
>>>> StreamsPartitionAssignor for all closed stores.
>>>> If there's a crash (no clean close), then we won't be able to guarantee
>>>> which offsets were flushed to disk by RocksDB, so we'd need to open (
>>>> init()), read offsets, and then close() those stores. But since this is
>>>> the exception, and will only occur once (provided it doesn't crash every
>>>> time!), I think the performance impact here would be acceptable.
>>>>
>>>> Thanks for the feedback, please let me know if you have any more
>> comments
>>>> or questions!
>>>>
>>>> I'm currently working on rebasing against trunk. This involves adding
>>>> support for transactionality to VersionedStateStores. I will probably
>> need
>>>> to revise my implementation for transactional "segmented" stores, both
>> to
>>>> accommodate VersionedStateStore, and to clean up some other stuff.
>>>>
>>>> Regards,
>>>> Nick
>>>>
>>>>
>>>> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org> wrote:
>>>>
>>>>> Hi Nick,
>>>>>
>>>>> Thanks for the updates!
>>>>>
>>>>> I have a couple of questions/comments.
>>>>>
>>>>> 1.
>>>>> Why do you propose a configuration that involves max. bytes and max.
>>>>> reords? I think we are mainly concerned about memory consumption
>> because
>>>>> we want to limit the off-heap memory used. I cannot think of a case
>>>>> where one would want to set the max. number of records.
>>>>>
>>>>>
>>>>> 2.
>>>>> Why does
>>>>>
>>>>>     default void commit(final Map<TopicPartition, Long>
>> changelogOffsets) {
>>>>>         flush();
>>>>>     }
>>>>>
>>>>> take a map of partitions to changelog offsets?
>>>>> The mapping between state stores to partitions is a 1:1 relationship.
>>>>> Passing in a single changelog offset should suffice.
>>>>>
>>>>>
>>>>> 3.
>>>>> Why do we need the Transaction interface? It should be possible to hide
>>>>> beginning and committing a transactions withing the state store
>>>>> implementation, so that from outside the state store, it does not
>> matter
>>>>> whether the state store is transactional or not. What would be the
>>>>> advantage of using the Transaction interface?
>>>>>
>>>>>
>>>>> 4.
>>>>> Regarding checkpointing offsets, I think we should keep the checkpoint
>>>>> file in any case for the reason you mentioned about rebalancing. Even
>> if
>>>>> that would not be an issue, I would propose to move the change to
>> offset
>>>>> management to a new KIP and to not add more complexity than needed to
>>>>> this one. I would not be too concerned about the consistency violation
>>>>> you mention. As far as I understand, with transactional state stores
>>>>> Streams would write the checkpoint file during every commit even under
>>>>> EOS. In the failure case you describe, Streams would restore the state
>>>>> stores from the offsets found in the checkpoint file written during the
>>>>> penultimate commit instead of during the last commit. Basically,
>> Streams
>>>>> would overwrite the records written to the state store between the last
>>>>> two commits with the same records read from the changelogs. While I
>>>>> understand that this is wasteful, it is -- at the same time --
>>>>> acceptable and most importantly it does not break EOS.
>>>>>
>>>>> Best,
>>>>> Bruno
>>>>>
>>>>>
>>>>> On 27.04.23 12:34, Nick Telford wrote:
>>>>>> Hi everyone,
>>>>>>
>>>>>> I find myself (again) considering removing the offset management from
>>>>>> StateStores, and keeping the old checkpoint file system. The reason is
>>>>> that
>>>>>> the StreamPartitionAssignor directly reads checkpoint files in order
>> to
>>>>>> determine which instance has the most up-to-date copy of the local
>>>>> state.
>>>>>> If we move offsets into the StateStore itself, then we will need to
>>>>> open,
>>>>>> initialize, read offsets and then close each StateStore (that is not
>>>>>> already assigned and open) for which we have *any* local state, on
>> every
>>>>>> rebalance.
>>>>>>
>>>>>> Generally, I don't think there are many "orphan" stores like this
>>>>> sitting
>>>>>> around on most instances, but even a few would introduce additional
>>>>> latency
>>>>>> to an already somewhat lengthy rebalance procedure.
>>>>>>
>>>>>> I'm leaning towards Colt's (Slack) suggestion of just keeping things
>> in
>>>>> the
>>>>>> checkpoint file(s) for now, and not worrying about the race. The
>>>>> downside
>>>>>> is that we wouldn't be able to remove the explicit RocksDB flush
>>>>> on-commit,
>>>>>> which likely hurts performance.
>>>>>>
>>>>>> If anyone has any thoughts or ideas on this subject, I would
>> appreciate
>>>>> it!
>>>>>>
>>>>>> Regards,
>>>>>> Nick
>>>>>>
>>>>>> On Wed, 19 Apr 2023 at 15:05, Nick Telford <ni...@gmail.com>
>>>>> wrote:
>>>>>>
>>>>>>> Hi Colt,
>>>>>>>
>>>>>>> The issue is that if there's a crash between 2 and 3, then you still
>>>>> end
>>>>>>> up with inconsistent data in RocksDB. The only way to guarantee that
>>>>> your
>>>>>>> checkpoint offsets and locally stored data are consistent with each
>>>>> other
>>>>>>> are to atomically commit them, which can be achieved by having the
>>>>> offsets
>>>>>>> stored in RocksDB.
>>>>>>>
>>>>>>> The offsets column family is likely to be extremely small (one
>>>>>>> per-changelog partition + one per Topology input partition for
>> regular
>>>>>>> stores, one per input partition for global stores). So the overhead
>>>>> will be
>>>>>>> minimal.
>>>>>>>
>>>>>>> A major benefit of doing this is that we can remove the explicit
>> calls
>>>>> to
>>>>>>> db.flush(), which forcibly flushes memtables to disk on-commit. It
>>>>> turns
>>>>>>> out, RocksDB memtable flushes are largely dictated by Kafka Streams
>>>>>>> commits, *not* RocksDB configuration, which could be a major source
>> of
>>>>>>> confusion. Atomic checkpointing makes it safe to remove these
>> explicit
>>>>>>> flushes, because it no longer matters exactly when RocksDB flushes
>>>>> data to
>>>>>>> disk; since the data and corresponding checkpoint offsets will always
>>>>> be
>>>>>>> flushed together, the local store is always in a consistent state,
>> and
>>>>>>> on-restart, it can always safely resume restoration from the on-disk
>>>>>>> offsets, restoring the small amount of data that hadn't been flushed
>>>>> when
>>>>>>> the app exited/crashed.
>>>>>>>
>>>>>>> Regards,
>>>>>>> Nick
>>>>>>>
>>>>>>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io>
>>>>> wrote:
>>>>>>>
>>>>>>>> Nick,
>>>>>>>>
>>>>>>>> Thanks for your reply. Ack to A) and B).
>>>>>>>>
>>>>>>>> For item C), I see what you're referring to. Your proposed solution
>>>>> will
>>>>>>>> work, so no need to change it. What I was suggesting was that it
>>>>> might be
>>>>>>>> possible to achieve this with only one column family. So long as:
>>>>>>>>
>>>>>>>>       - No uncommitted records (i.e. not committed to the changelog)
>> are
>>>>>>>>       *committed* to the state store, AND
>>>>>>>>       - The Checkpoint offset (which refers to the changelog topic)
>> is
>>>>> less
>>>>>>>>       than or equal to the last written changelog offset in rocksdb
>>>>>>>>
>>>>>>>> I don't see the need to do the full restoration from scratch. My
>>>>>>>> understanding was that prior to 844/892, full restorations were
>>>>> required
>>>>>>>> because there could be uncommitted records written to RocksDB;
>>>>> however,
>>>>>>>> given your use of RocksDB transactions, that can be avoided with the
>>>>>>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
>>>>> transaction, 3)
>>>>>>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <
>> nick.telford@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Colt,
>>>>>>>>>
>>>>>>>>> A. I've done my best to de-couple the StateStore stuff from the
>> rest
>>>>> of
>>>>>>>> the
>>>>>>>>> Streams engine. The fact that there will be only one ongoing
>> (write)
>>>>>>>>> transaction at a time is not guaranteed by any API, and is just a
>>>>>>>>> consequence of the way Streams operates. To that end, I tried to
>>>>> ensure
>>>>>>>> the
>>>>>>>>> documentation and guarantees provided by the new APIs are
>>>>> independent of
>>>>>>>>> this incidental behaviour. In practice, you're right, this
>>>>> essentially
>>>>>>>>> refers to "interactive queries", which are technically "read
>>>>>>>> transactions",
>>>>>>>>> even if they don't actually use the transaction API to isolate
>>>>>>>> themselves.
>>>>>>>>>
>>>>>>>>> B. Yes, although not ideal. This is for backwards compatibility,
>>>>>>>> because:
>>>>>>>>>        1) Existing custom StateStore implementations will implement
>>>>>>>> flush(),
>>>>>>>>> and not commit(), but the Streams engine now calls commit(), so
>> those
>>>>>>>> calls
>>>>>>>>> need to be forwarded to flush() for these legacy stores.
>>>>>>>>>        2) Existing StateStore *users*, i.e. outside of the Streams
>>>>> engine
>>>>>>>>> itself, may depend on explicitly calling flush(), so for these
>> cases,
>>>>>>>>> flush() needs to be redirected to call commit().
>>>>>>>>> If anyone has a better way to guarantee compatibility without
>>>>>>>> introducing
>>>>>>>>> this potential recursion loop, I'm open to changes!
>>>>>>>>>
>>>>>>>>> C. This is described in the "Atomic Checkpointing" section. Offsets
>>>>> are
>>>>>>>>> stored in a separate RocksDB column family, which is guaranteed to
>> be
>>>>>>>>> atomically flushed to disk with all other column families. The
>> issue
>>>>> of
>>>>>>>>> checkpoints being written to disk after commit causing
>> inconsistency
>>>>> if
>>>>>>>> it
>>>>>>>>> crashes in between is the reason why, under EOS, checkpoint files
>> are
>>>>>>>> only
>>>>>>>>> written on clean shutdown. This is one of the major causes of "full
>>>>>>>>> restorations", so moving the offsets into a place where they can be
>>>>>>>>> guaranteed to be atomically written with the data they checkpoint
>>>>>>>> allows us
>>>>>>>>> to write the checkpoint offsets *on every commit*, not just on
>> clean
>>>>>>>>> shutdown.
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>> Nick
>>>>>>>>>
>>>>>>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io>
>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Nick,
>>>>>>>>>>
>>>>>>>>>> Thank you for continuing this work. I have a few minor clarifying
>>>>>>>>>> questions.
>>>>>>>>>>
>>>>>>>>>> A) "Records written to any transaction are visible to all other
>>>>>>>>>> transactions immediately." I am confused here—I thought there
>> could
>>>>>>>> only
>>>>>>>>> be
>>>>>>>>>> one transaction going on at a time for a given state store given
>> the
>>>>>>>>>> threading model for processing records on a Task. Do you mean
>>>>>>>> Interactive
>>>>>>>>>> Queries by "other transactions"? (If so, then everything makes
>>>>> sense—I
>>>>>>>>>> thought that since IQ were read-only then they didn't count as
>>>>>>>>>> transactions).
>>>>>>>>>>
>>>>>>>>>> B) Is it intentional that the default implementations of the
>> flush()
>>>>>>>> and
>>>>>>>>>> commit() methods in the StateStore class refer to each other in
>> some
>>>>>>>> sort
>>>>>>>>>> of unbounded recursion?
>>>>>>>>>>
>>>>>>>>>> C) How will the getCommittedOffset() method work? At first I
>> thought
>>>>>>>> the
>>>>>>>>>> way to do it would be using a special key in the RocksDB store to
>>>>>>>> store
>>>>>>>>> the
>>>>>>>>>> offset, and committing that with the transaction. But upon second
>>>>>>>>> thought,
>>>>>>>>>> since restoration from the changelog is an idempotent procedure, I
>>>>>>>> think
>>>>>>>>> it
>>>>>>>>>> would be fine to 1) commit the RocksDB transaction and then 2)
>> write
>>>>>>>> the
>>>>>>>>>> offset to disk in a checkpoint file. If there is a crash between
>> 1)
>>>>>>>> and
>>>>>>>>> 2),
>>>>>>>>>> I think the only downside is 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 <
>>>>> nick.telford@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>
>>>>>>>>>>> I've updated the KIP to reflect the latest version of the design:
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>
>>>>>>>>>>> There are several changes in there that reflect feedback from
>> this
>>>>>>>>>> thread,
>>>>>>>>>>> and there's a new section and a bunch of interface changes
>> relating
>>>>>>>> to
>>>>>>>>>>> Atomic Checkpointing, which is the final piece of the puzzle to
>>>>>>>> making
>>>>>>>>>>> everything robust.
>>>>>>>>>>>
>>>>>>>>>>> Let me know what you think!
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>> Nick
>>>>>>>>>>>
>>>>>>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <
>> nick.telford@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hi Lucas,
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for looking over my KIP.
>>>>>>>>>>>>
>>>>>>>>>>>> A) The bound is per-instance, not per-Task. This was a typo in
>> the
>>>>>>>>> KIP
>>>>>>>>>>>> that I've now corrected. It was originally per-Task, but I
>>>>>>>> changed it
>>>>>>>>>> to
>>>>>>>>>>>> per-instance for exactly the reason you highlighted.
>>>>>>>>>>>> B) It's worth noting that transactionality is only enabled under
>>>>>>>> EOS,
>>>>>>>>>> and
>>>>>>>>>>>> in the default mode of operation (ALOS), there should be no
>>>>>>>> change in
>>>>>>>>>>>> behavior at all. I think, under EOS, we can mitigate the impact
>> on
>>>>>>>>>> users
>>>>>>>>>>> by
>>>>>>>>>>>> sufficiently low default values for the memory bound
>>>>>>>> configuration. I
>>>>>>>>>>>> understand your hesitation to include a significant change of
>>>>>>>>>> behaviour,
>>>>>>>>>>>> especially in a minor release, but I suspect that most users
>> will
>>>>>>>>>> prefer
>>>>>>>>>>>> the memory impact (under EOS) to the existing behaviour of
>>>>>>>> frequent
>>>>>>>>>> state
>>>>>>>>>>>> restorations! If this is a problem, the changes can wait until
>> the
>>>>>>>>> next
>>>>>>>>>>>> major release. I'll be running a patched version of streams in
>>>>>>>>>> production
>>>>>>>>>>>> with these changes as soon as they're ready, so it won't disrupt
>>>>>>>> me
>>>>>>>>> :-D
>>>>>>>>>>>> C) The main purpose of this sentence was just to note that some
>>>>>>>>> changes
>>>>>>>>>>>> will need to be made to the way Segments are handled in order to
>>>>>>>>> ensure
>>>>>>>>>>>> they also benefit from transactions. At the time I wrote it, I
>>>>>>>> hadn't
>>>>>>>>>>>> figured out the specific changes necessary, so it was
>> deliberately
>>>>>>>>>> vague.
>>>>>>>>>>>> This is the one outstanding problem I'm currently working on,
>> and
>>>>>>>>> I'll
>>>>>>>>>>>> update this section with more detail once I have figured out the
>>>>>>>>> exact
>>>>>>>>>>>> changes required.
>>>>>>>>>>>> D) newTransaction() provides the necessary isolation guarantees.
>>>>>>>>> While
>>>>>>>>>>>> the RocksDB implementation of transactions doesn't technically
>>>>>>>> *need*
>>>>>>>>>>>> read-only users to call newTransaction(), other implementations
>>>>>>>>> (e.g. a
>>>>>>>>>>>> hypothetical PostgresStore) may require it. Calling
>>>>>>>> newTransaction()
>>>>>>>>>> when
>>>>>>>>>>>> no transaction is necessary is essentially free, as it will just
>>>>>>>>> return
>>>>>>>>>>>> this.
>>>>>>>>>>>>
>>>>>>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think it
>>>>>>>> should
>>>>>>>>> be
>>>>>>>>>>>> fairly obvious where the performance problems stem from: writes
>>>>>>>> under
>>>>>>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with the
>>>>>>>>>>>> tombstone/record flag, 1 to decode it from the tombstone/record
>>>>>>>> flag,
>>>>>>>>>>> and 1
>>>>>>>>>>>> to copy the record from the "temporary" store to the "main"
>> store,
>>>>>>>>> when
>>>>>>>>>>> the
>>>>>>>>>>>> transaction commits. The different approach taken by KIP-869
>>>>>>>> should
>>>>>>>>>>> perform
>>>>>>>>>>>> much better, as it avoids all these copies, and may actually
>>>>>>>> perform
>>>>>>>>>>>> slightly better than trunk, due to batched writes in RocksDB
>>>>>>>>> performing
>>>>>>>>>>>> better than non-batched writes.[1]
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>> Nick
>>>>>>>>>>>>
>>>>>>>>>>>> 1:
>>>>>>>>>>>
>>>>>>>>>
>>>>>
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>>
>>>>>>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
>>>>>>>> lbrutschy@confluent.io
>>>>>>>>>>> .invalid>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>>>
>>>>>>>>>>>>> I'm just starting to read up on the whole discussion about
>>>>>>>> KIP-892
>>>>>>>>> and
>>>>>>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
>>>>>>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do have some
>>>>>>>>>>>>> questions about the latest draft.
>>>>>>>>>>>>>
>>>>>>>>>>>>>     A) If I understand correctly, you propose to put a bound on
>> the
>>>>>>>>>>>>> (native) memory consumed by each task. However, I wonder if
>> this
>>>>>>>> is
>>>>>>>>>>>>> sufficient if we have temporary imbalances in the cluster. For
>>>>>>>>>>>>> example, depending on the timing of rebalances during a cluster
>>>>>>>>>>>>> restart, it could happen that a single streams node is
>> assigned a
>>>>>>>>> lot
>>>>>>>>>>>>> more tasks than expected. With your proposed change, this would
>>>>>>>> mean
>>>>>>>>>>>>> that the memory required by this one node could be a multiple
>> of
>>>>>>>>> what
>>>>>>>>>>>>> is required during normal operation. I wonder if it wouldn't be
>>>>>>>>> safer
>>>>>>>>>>>>> to put a global bound on the memory use, across all tasks.
>>>>>>>>>>>>>     B) Generally, the memory concerns still give me the feeling
>>>>> that
>>>>>>>>> this
>>>>>>>>>>>>> should not be enabled by default for all users in a minor
>>>>>>>> release.
>>>>>>>>>>>>>     C) In section "Transaction Management": the sentence "A
>> similar
>>>>>>>>>>>>> analogue will be created to automatically manage `Segment`
>>>>>>>>>>>>> transactions.". Maybe this is just me lacking some background,
>>>>>>>> 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 you to find out why the KIP-844 PoC is 20x slower
>>>>>>>> (e.g.
>>>>>>>>>>>>> by providing a flame graph), that would be quite interesting.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>> Lucas
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I've updated the KIP with a more detailed design, which
>>>>>>>> reflects
>>>>>>>>> the
>>>>>>>>>>>>>> implementation I've been working on:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> This new design should address the outstanding points already
>>>>>>>> made
>>>>>>>>>> in
>>>>>>>>>>>>> the
>>>>>>>>>>>>>> thread.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Please let me know if there are areas that are unclear or need
>>>>>>>>> more
>>>>>>>>>>>>>> clarification.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I have a (nearly) working implementation. I'm confident that
>>>>>>>> the
>>>>>>>>>>>>> remaining
>>>>>>>>>>>>>> work (making Segments behave) will not impact the documented
>>>>>>>>> design.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <
>> colt@littlehorse.io
>>>>>>>>>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thank you for the reply; that makes sense. I was hoping that,
>>>>>>>>>> since
>>>>>>>>>>>>> reading
>>>>>>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
>>>>>>>> documented
>>>>>>>>>> API,
>>>>>>>>>>>>> maybe
>>>>>>>>>>>>>>> you *wouldn't* have to wait for the next major release to
>>>>>>>> make
>>>>>>>>>> that
>>>>>>>>>>>>> change;
>>>>>>>>>>>>>>> but given 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 <
>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended to
>>>>>>>> try
>>>>>>>>> to
>>>>>>>>>>>>> keep the
>>>>>>>>>>>>>>>> behaviour unchanged as much as possible, otherwise we'd
>>>>>>>> have
>>>>>>>>> to
>>>>>>>>>>>>> wait for
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> major version release to land these changes.
>>>>>>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
>>>>>>>> guarantee,
>>>>>>>>>> and
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> typically longer commit intervals would be problematic when
>>>>>>>>>>> reading
>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>> "committed" records.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I've been away for 5 days recovering from minor surgery,
>>>>>>>> but I
>>>>>>>>>>>>> spent a
>>>>>>>>>>>>>>>> considerable amount of that time working through ideas for
>>>>>>>>>>> possible
>>>>>>>>>>>>>>>> solutions in my head. I think your suggestion of keeping
>>>>>>>> ALOS
>>>>>>>>>>>>> as-is, but
>>>>>>>>>>>>>>>> buffering writes for EOS is the right path forwards,
>>>>>>>> although
>>>>>>>>> I
>>>>>>>>>>>>> have a
>>>>>>>>>>>>>>>> solution that both expands on this, and provides for some
>>>>>>>> more
>>>>>>>>>>>>> formal
>>>>>>>>>>>>>>>> guarantees.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Essentially, adding support to KeyValueStores for
>>>>>>>>>> "Transactions",
>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> clearly defined IsolationLevels. Using "Read Committed"
>>>>>>>> when
>>>>>>>>>> under
>>>>>>>>>>>>> EOS,
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> "Read Uncommitted" under ALOS.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The nice thing about this approach is that it gives us much
>>>>>>>>> more
>>>>>>>>>>>>> clearly
>>>>>>>>>>>>>>>> defined isolation behaviour that can be properly
>>>>>>>> documented to
>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>> know what to expect.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I'm still working out the kinks in the design, and will
>>>>>>>> update
>>>>>>>>>> the
>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>> when
>>>>>>>>>>>>>>>> I have something. The main struggle is trying to implement
>>>>>>>>> this
>>>>>>>>>>>>> without
>>>>>>>>>>>>>>>> making any major changes to the existing interfaces or
>>>>>>>>> breaking
>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>> implementations, because currently everything expects to
>>>>>>>>> operate
>>>>>>>>>>>>> directly
>>>>>>>>>>>>>>>> on a StateStore, and not a Transaction of that store. I
>>>>>>>> think
>>>>>>>>>> I'm
>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>> close, although sadly I won't be able to progress much
>>>>>>>> until
>>>>>>>>>> next
>>>>>>>>>>>>> week
>>>>>>>>>>>>>>> due
>>>>>>>>>>>>>>>> to some work commitments.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
>>>>>>>>> colt@littlehorse.io>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thank you for the explanation, and also for the updated
>>>>>>>>> KIP. I
>>>>>>>>>>> am
>>>>>>>>>>>>> quite
>>>>>>>>>>>>>>>>> eager for this improvement to be released as it would
>>>>>>>>> greatly
>>>>>>>>>>>>> reduce
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> operational difficulties of EOS streams apps.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Two questions:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 10)
>>>>>>>>>>>>>>>>>> When reading records, we will use the
>>>>>>>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
>>>>>>>>>>>>>>>>>     and WriteBatchWithIndex#newIteratorWithBase utilities in
>>>>>>>>>> order
>>>>>>>>>>> to
>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>> that uncommitted writes are available to query.
>>>>>>>>>>>>>>>>> Why do extra work to enable the reading of uncommitted
>>>>>>>>> writes
>>>>>>>>>>>>> during
>>>>>>>>>>>>>>> IQ?
>>>>>>>>>>>>>>>>> Code complexity aside, reading uncommitted writes is, in
>>>>>>>> my
>>>>>>>>>>>>> opinion, a
>>>>>>>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have the
>>>>>>>>>>> guarantee
>>>>>>>>>>>>> that,
>>>>>>>>>>>>>>>>> with EOS, IQ only reads committed records. In order to
>>>>>>>> avoid
>>>>>>>>>>> dirty
>>>>>>>>>>>>>>> reads,
>>>>>>>>>>>>>>>>> one currently must query a standby replica (but this
>>>>>>>> still
>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>> fully
>>>>>>>>>>>>>>>>> guarantee monotonic reads).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 20) Is it also necessary to enable this optimization on
>>>>>>>> ALOS
>>>>>>>>>>>>> stores?
>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need to
>>>>>>>>> restore
>>>>>>>>>>>>> state
>>>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
>>>>>>>>> acceptable
>>>>>>>>>> to
>>>>>>>>>>>>> accept
>>>>>>>>>>>>>>>>> that there may have been uncommitted writes on disk. On a
>>>>>>>>> side
>>>>>>>>>>>>> note, if
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>> enable this type of store on ALOS processors, the
>>>>>>>> community
>>>>>>>>>>> would
>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
>>>>>>>>> longer
>>>>>>>>>>>>> include
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> StateStore management of checkpointing. This can be
>>>>>>>> added
>>>>>>>>>> as a
>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>>>> to further optimize the consistency and performance of
>>>>>>>>> state
>>>>>>>>>>>>> stores.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I've also added a section discussing some of the
>>>>>>>> concerns
>>>>>>>>>>> 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 2022 at 16:36, Nick Telford <
>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I didn't do any profiling, but the 844
>>>>>>>> implementation:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>       - Writes uncommitted records to a temporary
>>>>>>>> RocksDB
>>>>>>>>>>>>> instance
>>>>>>>>>>>>>>>>>>>          - Since tombstones need to be flagged, all
>>>>>>>> record
>>>>>>>>>>>>> values are
>>>>>>>>>>>>>>>>>>>          prefixed with a value/tombstone marker. This
>>>>>>>>>>>>> necessitates a
>>>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>> copy.
>>>>>>>>>>>>>>>>>>>       - On-commit, iterates all records in this
>>>>>>>> temporary
>>>>>>>>>>>>> instance and
>>>>>>>>>>>>>>>>>>>       writes them to the main RocksDB store.
>>>>>>>>>>>>>>>>>>>       - While iterating, the value/tombstone marker
>>>>>>>> needs
>>>>>>>>> to
>>>>>>>>>> be
>>>>>>>>>>>>> parsed
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>       the real value extracted. This necessitates
>>>>>>>> another
>>>>>>>>>>> memory
>>>>>>>>>>>>> copy.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> My guess is that the cost of iterating the temporary
>>>>>>>>>> RocksDB
>>>>>>>>>>>>> store
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> major factor, with the 2 extra memory copies
>>>>>>>> per-Record
>>>>>>>>>>>>>>> contributing
>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> significant amount too.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
>>>>>>>>>>>>> colt@littlehorse.io>
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Out of curiosity, why does the performance of the
>>>>>>>> store
>>>>>>>>>>>>> degrade so
>>>>>>>>>>>>>>>>>>>> significantly with the 844 implementation? I
>>>>>>>> wouldn't
>>>>>>>>> be
>>>>>>>>>>> too
>>>>>>>>>>>>>>>> surprised
>>>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> 50-60% drop (caused by each record being written
>>>>>>>>> twice),
>>>>>>>>>>> but
>>>>>>>>>>>>> 96%
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> extreme.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The only thing I can think of which could create
>>>>>>>> such a
>>>>>>>>>>>>> bottleneck
>>>>>>>>>>>>>>>>> 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 Nick Telford <
>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I've updated the KIP to resolve all the points
>>>>>>>> that
>>>>>>>>>> have
>>>>>>>>>>>>> been
>>>>>>>>>>>>>>>> raised
>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>> far, with one exception: the ALOS default commit
>>>>>>>>>> interval
>>>>>>>>>>>>> of 5
>>>>>>>>>>>>>>>>> minutes
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> likely to cause WriteBatchWithIndex memory to grow
>>>>>>>>> too
>>>>>>>>>>>>> large.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> There's a couple of different things I can think
>>>>>>>> of
>>>>>>>>> to
>>>>>>>>>>>>> solve
>>>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>       - We already have a memory/record limit in the
>>>>>>>> KIP
>>>>>>>>>> to
>>>>>>>>>>>>> prevent
>>>>>>>>>>>>>>>> OOM
>>>>>>>>>>>>>>>>>>>>>       errors. Should we choose a default value for
>>>>>>>>> these?
>>>>>>>>>> My
>>>>>>>>>>>>>>> concern
>>>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>       anything we choose might seem rather
>>>>>>>> arbitrary. We
>>>>>>>>>>> could
>>>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>>>>>       its behaviour such that under ALOS, it only
>>>>>>>>> triggers
>>>>>>>>>>> the
>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>       StateStore, but under EOS, it triggers a
>>>>>>>> commit of
>>>>>>>>>> the
>>>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>>>       - We could introduce a separate `
>>>>>>>>>>> checkpoint.interval.ms`
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>>>>>>       to commit the StateStores more frequently than
>>>>>>>> the
>>>>>>>>>>>>> general
>>>>>>>>>>>>>>>>>>>>>       commit.interval.ms? My concern here is that
>>>>>>>> the
>>>>>>>>>>>>> semantics of
>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>>>       would depend 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 <
>>>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi Alex,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks for the feedback.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I've updated the discussion of OOM issues by
>>>>>>>>>> describing
>>>>>>>>>>>>> how
>>>>>>>>>>>>>>>> we'll
>>>>>>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>>>>> it. Here's the new text:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> To mitigate this, we will automatically force a
>>>>>>>>> Task
>>>>>>>>>>>>> commit if
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> total
>>>>>>>>>>>>>>>>>>>>>>> uncommitted records returned by
>>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedEntries()
>>>>>>>>>>> exceeds a
>>>>>>>>>>>>>>>>> threshold,
>>>>>>>>>>>>>>>>>>>>>>> configured by
>>>>>>>>>> max.uncommitted.state.entries.per.task;
>>>>>>>>>>>>> or the
>>>>>>>>>>>>>>>>> total
>>>>>>>>>>>>>>>>>>>>>>> memory used for buffering uncommitted records
>>>>>>>>>> returned
>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
>>>>>>>>> exceeds
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> threshold
>>>>>>>>>>>>>>>>>>>>>>> configured by
>>>>>>>>> max.uncommitted.state.bytes.per.task.
>>>>>>>>>>>>> This will
>>>>>>>>>>>>>>>>>> roughly
>>>>>>>>>>>>>>>>>>>>>>> bound the memory required per-Task for
>>>>>>>> buffering
>>>>>>>>>>>>> uncommitted
>>>>>>>>>>>>>>>>>> records,
>>>>>>>>>>>>>>>>>>>>>>> irrespective of the commit.interval.ms, and
>>>>>>>> will
>>>>>>>>>>>>> effectively
>>>>>>>>>>>>>>>>> bound
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> number of records that will need to be
>>>>>>>> restored in
>>>>>>>>>> the
>>>>>>>>>>>>> event
>>>>>>>>>>>>>>>> of a
>>>>>>>>>>>>>>>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> These limits will be checked in
>>>>>>>> StreamTask#process
>>>>>>>>>> and
>>>>>>>>>>> a
>>>>>>>>>>>>>>>> premature
>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>> will be requested via Task#requestCommit().
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Note that these new methods provide default
>>>>>>>>>>>>> implementations
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>>>>> existing custom stores and non-transactional
>>>>>>>>> stores
>>>>>>>>>>>>> (e.g.
>>>>>>>>>>>>>>>>>>>>>>> InMemoryKeyValueStore) do not force any early
>>>>>>>>>> commits.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I've chosen to have the StateStore expose
>>>>>>>>>>> approximations
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>>>> buffer
>>>>>>>>>>>>>>>>>>>>>> size/count instead of opaquely requesting a
>>>>>>>> commit
>>>>>>>>> in
>>>>>>>>>>>>> order to
>>>>>>>>>>>>>>>>>>>> delegate
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> decision making to the Task itself. This enables
>>>>>>>>>> Tasks
>>>>>>>>>>>>> to look
>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>> *all*
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>> their StateStores, and determine whether an
>>>>>>>> early
>>>>>>>>>>> commit
>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> necessary.
>>>>>>>>>>>>>>>>>>>>>> Notably, it enables pre-Task thresholds,
>>>>>>>> instead of
>>>>>>>>>>>>> per-Store,
>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>> prevents Tasks with many StateStores from using
>>>>>>>>> much
>>>>>>>>>>> more
>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>> Tasks
>>>>>>>>>>>>>>>>>>>>>> with one StateStore. This makes sense, since
>>>>>>>>> commits
>>>>>>>>>>> are
>>>>>>>>>>>>> done
>>>>>>>>>>>>>>>>>> by-Task,
>>>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>> by-Store.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Prizes* for anyone who can come up with a better
>>>>>>>>> name
>>>>>>>>>>>>> for the
>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>>>> properties!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks for pointing out the potential
>>>>>>>> performance
>>>>>>>>>>> issues
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> WBWI.
>>>>>>>>>>>>>>>>>> From
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> benchmarks that user posted[1], it looks like
>>>>>>>> WBWI
>>>>>>>>>>> still
>>>>>>>>>>>>>>>> performs
>>>>>>>>>>>>>>>>>>>>>> considerably better than individual puts, which
>>>>>>>> is
>>>>>>>>>> the
>>>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>>>> design,
>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>> I'd actually expect a performance boost from
>>>>>>>> WBWI,
>>>>>>>>>> just
>>>>>>>>>>>>> not as
>>>>>>>>>>>>>>>>> great
>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>> we'd get from a plain WriteBatch. This does
>>>>>>>> suggest
>>>>>>>>>>> that
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>>>> optimization would be to use a regular
>>>>>>>> WriteBatch
>>>>>>>>> for
>>>>>>>>>>>>>>>> restoration
>>>>>>>>>>>>>>>>>> (in
>>>>>>>>>>>>>>>>>>>>>> RocksDBStore#restoreBatch), since we know that
>>>>>>>>> those
>>>>>>>>>>>>> records
>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>> never
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>> queried before they're committed.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 1:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> * Just kidding, no prizes, sadly.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Wed, 23 Nov 2022 at 12:28, Alexander
>>>>>>>> Sorokoumov
>>>>>>>>>>>>>>>>>>>>>> <as...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 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 to keep the current
>>>>>>>>>> checkpointing
>>>>>>>>>>>>> as a
>>>>>>>>>>>>>>>>>> fallback
>>>>>>>>>>>>>>>>>>>>>>> mechanism. We want to keep existing users'
>>>>>>>>> workflows
>>>>>>>>>>>>> intact
>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> can. A
>>>>>>>>>>>>>>>>>>>>>>> non-intrusive way would be to add a separate
>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>> method,
>>>>>>>>>>>>>>>>>> say,
>>>>>>>>>>>>>>>>>>>>>>> StateStore#managesCheckpointing(), that
>>>>>>>> controls
>>>>>>>>>>>>> whether the
>>>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>> implementation owns checkpointing.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I think that a solution to the transactional
>>>>>>>>> writes
>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>> address
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> OOMEs. One possible way to address that is to
>>>>>>>> wire
>>>>>>>>>>>>>>> StateStore's
>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>> request by adding, say, StateStore#commitNeeded
>>>>>>>>> that
>>>>>>>>>>> is
>>>>>>>>>>>>>>> checked
>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> StreamTask#commitNeeded via the corresponding
>>>>>>>>>>>>>>>>>> ProcessorStateManager.
>>>>>>>>>>>>>>>>>>>>> With
>>>>>>>>>>>>>>>>>>>>>>> that change, RocksDBStore will have to track
>>>>>>>> the
>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>>>> size
>>>>>>>>>>>>>>>>>>>>>>> and request a commit when the size goes over a
>>>>>>>>>>>>> (configurable)
>>>>>>>>>>>>>>>>>>>> threshold.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> AFAIU WriteBatchWithIndex might perform
>>>>>>>>>> significantly
>>>>>>>>>>>>> slower
>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>>>> non-txn
>>>>>>>>>>>>>>>>>>>>>>> puts as the batch size grows [1]. We should
>>>>>>>> have a
>>>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> fall
>>>>>>>>>>>>>>>>>>>>>>> back to the current behavior (and/or disable
>>>>>>>> txn
>>>>>>>>>>> stores
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>> ALOS)
>>>>>>>>>>>>>>>>>>>> unless
>>>>>>>>>>>>>>>>>>>>>>> the benchmarks show negligible overhead for
>>>>>>>> longer
>>>>>>>>>>>>> commits /
>>>>>>>>>>>>>>>>>>>>> large-enough
>>>>>>>>>>>>>>>>>>>>>>> batch sizes.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> If you prefer to keep the KIP smaller, I would
>>>>>>>>>> rather
>>>>>>>>>>>>> cut out
>>>>>>>>>>>>>>>>>>>>>>> state-store-managed checkpointing rather than
>>>>>>>>> proper
>>>>>>>>>>>>> OOMe
>>>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>> being able to switch to non-txn behavior. The
>>>>>>>>>>>>> checkpointing
>>>>>>>>>>>>>>> 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 configuration parts as well as review
>>>>>>>> and
>>>>>>>>>>> test
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> patch.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>> Alex
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>>>> https://github.com/facebook/rocksdb/issues/608
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
>>>>>>>>>>>>>>>>>> nick.telford@gmail.com
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the review and feedback!
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 1. Custom Stores: I've been mulling over this
>>>>>>>>>>> problem
>>>>>>>>>>>>>>> myself.
>>>>>>>>>>>>>>>>> As
>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>> stands,
>>>>>>>>>>>>>>>>>>>>>>>> custom stores would essentially lose
>>>>>>>>> checkpointing
>>>>>>>>>>>>> with no
>>>>>>>>>>>>>>>>>>>> indication
>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> they're expected to make changes, besides a
>>>>>>>> line
>>>>>>>>>> in
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> release
>>>>>>>>>>>>>>>>>>>>> notes. I
>>>>>>>>>>>>>>>>>>>>>>>> agree that the best solution would be to
>>>>>>>>> provide a
>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>> checkpoints
>>>>>>>>>>>>>>>>>>>>>>>> to a file. The one thing I would change is
>>>>>>>> that
>>>>>>>>>> the
>>>>>>>>>>>>>>>>> checkpointing
>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> store-local file, instead of a per-Task file.
>>>>>>>>> This
>>>>>>>>>>>>> way the
>>>>>>>>>>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>> technically owns its own checkpointing (via a
>>>>>>>>>>> default
>>>>>>>>>>>>>>>>>>>> implementation),
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>> the StateManager/Task execution engine
>>>>>>>> doesn't
>>>>>>>>>> need
>>>>>>>>>>>>> to know
>>>>>>>>>>>>>>>>>>>> anything
>>>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>>>> checkpointing, which greatly simplifies some
>>>>>>>> of
>>>>>>>>>> the
>>>>>>>>>>>>> logic.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 2. OOME errors: The main reasons why I didn't
>>>>>>>>>>> explore
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> this is
>>>>>>>>>>>>>>>>>>>>>>>> a) to keep this KIP as simple as possible,
>>>>>>>> and
>>>>>>>>> b)
>>>>>>>>>>>>> because
>>>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>>>> how to signal that a Task should commit
>>>>>>>>>> prematurely.
>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>>>> confident
>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>> possible, and I think it's worth adding a
>>>>>>>>> section
>>>>>>>>>> on
>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>>>> Besides
>>>>>>>>>>>>>>>>>>>>>>>> my proposal to force an early commit once
>>>>>>>> memory
>>>>>>>>>>> usage
>>>>>>>>>>>>>>>> reaches
>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>> threshold,
>>>>>>>>>>>>>>>>>>>>>>>> is there any other approach that you might
>>>>>>>>> suggest
>>>>>>>>>>> for
>>>>>>>>>>>>>>>> tackling
>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>> problem?
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 3. ALOS: I can add in an explicit paragraph,
>>>>>>>> but
>>>>>>>>>> my
>>>>>>>>>>>>>>>> assumption
>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>> since transactional behaviour comes at
>>>>>>>> little/no
>>>>>>>>>>>>> cost, that
>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>> available by default on all stores,
>>>>>>>> irrespective
>>>>>>>>>> of
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>>>> mode.
>>>>>>>>>>>>>>>>>>>>>>>> While ALOS doesn't use transactions, the Task
>>>>>>>>>> itself
>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>> "commits",
>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>> the behaviour should be correct under ALOS
>>>>>>>> too.
>>>>>>>>>> I'm
>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>> convinced
>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>>> worth having both
>>>>>>>>> transactional/non-transactional
>>>>>>>>>>>>> stores
>>>>>>>>>>>>>>>>>>>> available, as
>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>> would considerably increase the complexity of
>>>>>>>>> the
>>>>>>>>>>>>> codebase,
>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>> very
>>>>>>>>>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>>>>>>>>>> benefit.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> 4. Method deprecation: Are you referring to
>>>>>>>>>>>>>>>>>>>> StateStore#getPosition()?
>>>>>>>>>>>>>>>>>>>>>>> As I
>>>>>>>>>>>>>>>>>>>>>>>> understand it, Position contains the
>>>>>>>> position of
>>>>>>>>>> the
>>>>>>>>>>>>>>> *source*
>>>>>>>>>>>>>>>>>>>> topics,
>>>>>>>>>>>>>>>>>>>>>>>> whereas the commit offsets would be the
>>>>>>>>>> *changelog*
>>>>>>>>>>>>>>> offsets.
>>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>>>> necessary to retain the Position data, as
>>>>>>>> well
>>>>>>>>> as
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> changelog
>>>>>>>>>>>>>>>>>>>>> offsets.
>>>>>>>>>>>>>>>>>>>>>>>> What I meant in the KIP is that Position
>>>>>>>> offsets
>>>>>>>>>> are
>>>>>>>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>>>> stored
>>>>>>>>>>>>>>>>>>>>>>> in a
>>>>>>>>>>>>>>>>>>>>>>>> file, and since we can atomically store
>>>>>>>> metadata
>>>>>>>>>>>>> along with
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>>>> batch we commit to RocksDB, we can move our
>>>>>>>>>> Position
>>>>>>>>>>>>>>> offsets
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>> metadata too, and gain the same transactional
>>>>>>>>>>>>> guarantees
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>>>> changelog offsets, ensuring that the Position
>>>>>>>>>>> offsets
>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>> consistent
>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>> the records that are read from the database.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Tue, 22 Nov 2022 at 16:25, John Roesler <
>>>>>>>>>>>>>>>>> vvcephei@apache.org>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for publishing this alternative,
>>>>>>>> Nick!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> The benchmark you mentioned in the KIP-844
>>>>>>>>>>>>> discussion
>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>>> compelling reason to revisit the built-in
>>>>>>>>>>>>>>> transactionality
>>>>>>>>>>>>>>>>>>>>> mechanism.
>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>> also appreciate you analysis, showing that
>>>>>>>> for
>>>>>>>>>>> most
>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>> cases,
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>>>> batch approach should be just fine.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> There are a couple of points that would
>>>>>>>> hold
>>>>>>>>> me
>>>>>>>>>>>>> back from
>>>>>>>>>>>>>>>>>>>> approving
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>> KIP right now:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 1. Loss of coverage for custom stores.
>>>>>>>>>>>>>>>>>>>>>>>>> The fact that you can plug in a
>>>>>>>> (relatively)
>>>>>>>>>>> simple
>>>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> XStateStore interfaces and automagically
>>>>>>>> get a
>>>>>>>>>>>>>>> distributed
>>>>>>>>>>>>>>>>>>>> database
>>>>>>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>> it is a significant benefit of Kafka
>>>>>>>> Streams.
>>>>>>>>>> I'd
>>>>>>>>>>>>> hate to
>>>>>>>>>>>>>>>>> lose
>>>>>>>>>>>>>>>>>>>> it,
>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>>>> would be better to spend some time and
>>>>>>>> come up
>>>>>>>>>>> with
>>>>>>>>>>>>> a way
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> preserve
>>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>> property. For example, can we provide a
>>>>>>>>> default
>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>>> `commit(..)` that re-implements the
>>>>>>>> existing
>>>>>>>>>>>>>>>> checkpoint-file
>>>>>>>>>>>>>>>>>>>>>>> approach? Or
>>>>>>>>>>>>>>>>>>>>>>>>> perhaps add an `isTransactional()` flag to
>>>>>>>> the
>>>>>>>>>>> state
>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>>>> that the runtime can decide whether to
>>>>>>>>> continue
>>>>>>>>>> to
>>>>>>>>>>>>> manage
>>>>>>>>>>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>>>>>>>>>>> files
>>>>>>>>>>>>>>>>>>>>>>>>> vs delegating transactionality to the
>>>>>>>> stores?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 2. Guarding against OOME
>>>>>>>>>>>>>>>>>>>>>>>>> I appreciate your analysis, but I don't
>>>>>>>> think
>>>>>>>>>> it's
>>>>>>>>>>>>>>>> sufficient
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>>>> we will solve the memory problem later if
>>>>>>>> it
>>>>>>>>>>> becomes
>>>>>>>>>>>>>>>>> necessary.
>>>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>>>> experience leading to that situation would
>>>>>>>> be
>>>>>>>>>>> quite
>>>>>>>>>>>>> bad:
>>>>>>>>>>>>>>>>>> Imagine,
>>>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>>>> upgrade to AK 3.next, your tests pass, so
>>>>>>>> you
>>>>>>>>>>>>> deploy to
>>>>>>>>>>>>>>>>>>>> production.
>>>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>>>> night, you get paged because your app is
>>>>>>>> now
>>>>>>>>>>>>> crashing
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>> OOMEs. As
>>>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>>> all OOMEs, you'll have a really hard time
>>>>>>>>>> finding
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> root
>>>>>>>>>>>>>>>>>> cause,
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>> once
>>>>>>>>>>>>>>>>>>>>>>>>> you do, you won't have a clear path to
>>>>>>>> resolve
>>>>>>>>>> the
>>>>>>>>>>>>> issue.
>>>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>>>> tune down the commit interval and cache
>>>>>>>> buffer
>>>>>>>>>>> size
>>>>>>>>>>>>> until
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>> stop
>>>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>>>> crashes.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> FYI, I know of multiple cases where people
>>>>>>>> run
>>>>>>>>>> EOS
>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>>>> larger
>>>>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>>>> intervals to get better batching than the
>>>>>>>>>> default,
>>>>>>>>>>>>> so I
>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>> pathological case would be as rare as you
>>>>>>>>>> suspect.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Given that we already have the rudiments
>>>>>>>> of an
>>>>>>>>>>> idea
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> prevent this downside, we should take the
>>>>>>>> time
>>>>>>>>>> to
>>>>>>>>>>>>> design
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> solution.
>>>>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>>>> owe
>>>>>>>>>>>>>>>>>>>>>>>>> it to our users to ensure that awesome new
>>>>>>>>>>> features
>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>>> bitter
>>>>>>>>>>>>>>>>>>>>>>>>> pills unless we can't avoid it.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 3. ALOS mode.
>>>>>>>>>>>>>>>>>>>>>>>>> On the other hand, I didn't see an
>>>>>>>> indication
>>>>>>>>> of
>>>>>>>>>>> how
>>>>>>>>>>>>>>> stores
>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>>>> handled under ALOS (aka non-EOS) mode.
>>>>>>>>>>>>> Theoretically, the
>>>>>>>>>>>>>>>>>>>>>>>> transactionality
>>>>>>>>>>>>>>>>>>>>>>>>> of the store and the processing mode are
>>>>>>>>>>>>> orthogonal. A
>>>>>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>>>> would serve ALOS just as well as a
>>>>>>>>>>>>> non-transactional one
>>>>>>>>>>>>>>>> (if
>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>> better).
>>>>>>>>>>>>>>>>>>>>>>>>> Under ALOS, though, the default commit
>>>>>>>>> interval
>>>>>>>>>> is
>>>>>>>>>>>>> five
>>>>>>>>>>>>>>>>>> minutes,
>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> memory issue is far more pressing.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> As I see it, we have several options to
>>>>>>>>> resolve
>>>>>>>>>>> this
>>>>>>>>>>>>>>> point.
>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>>>> demonstrate that transactional stores work
>>>>>>>>> just
>>>>>>>>>>>>> fine for
>>>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>>>>> and we
>>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>> therefore just swap over unconditionally.
>>>>>>>> We
>>>>>>>>>> could
>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>>> disable
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> transactional mechanism under ALOS so that
>>>>>>>>>> stores
>>>>>>>>>>>>> operate
>>>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>> they do today when run in ALOS mode.
>>>>>>>> Finally,
>>>>>>>>> we
>>>>>>>>>>>>> could do
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>> KIP-844 and make transactional stores
>>>>>>>> opt-in
>>>>>>>>>> (it'd
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> extra opt-in mechanism, but it's a good
>>>>>>>>>>>>>>>> get-out-of-jail-free
>>>>>>>>>>>>>>>>>>>> card).
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> 4. (minor point) Deprecation of methods
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> You mentioned that the new `commit` 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?
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks again for the KIP! It's really nice
>>>>>>>>> that
>>>>>>>>>>> you
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> Alex
>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> chance to collaborate on both directions so
>>>>>>>>> that
>>>>>>>>>>> we
>>>>>>>>>>>>> can
>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> best
>>>>>>>>>>>>>>>>>>>>>>>>> outcome for Streams and its users.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On 2022/11/21 15:02:15 Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>>>>>>>>>>>> ,
>>>>>>>>>>>>>>>>>>>>>>>>>> so that it can be discussed/reviewed
>>>>>>>>>> separately
>>>>>>>>>>>>> from
>>>>>>>>>>>>>>>>> KIP-844.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Alex: I'm especially interested in what
>>>>>>>> you
>>>>>>>>>>> think!
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I have a nearly complete implementation
>>>>>>>> of
>>>>>>>>> the
>>>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>>>> outlined in
>>>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>>>> KIP, please let me know if you'd like me
>>>>>>>> to
>>>>>>>>>> push
>>>>>>>>>>>>> them
>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>> review
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>>> advance
>>>>>>>>>>>>>>>>>>>>>>>>>> of a vote.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Bruno,

Thanks for reviewing the KIP. It's been a long road, I started working on
this more than a year ago, and most of the time in the last 6 months has
been spent on the "Atomic Checkpointing" stuff that's been benched, so some
of the reasoning behind some of my decisions have been lost, but I'll do my
best to reconstruct them.

1.
IIRC, this was the initial approach I tried. I don't remember the exact
reasons I changed it to use a separate "view" of the StateStore that
encapsulates the transaction, but I believe it had something to do with
concurrent access to the StateStore from Interactive Query threads. Reads
from interactive queries need to be isolated from the currently ongoing
transaction, both for consistency (so interactive queries don't observe
changes that are subsequently rolled-back), but also to prevent Iterators
opened by an interactive query from being closed and invalidated by the
StreamThread when it commits the transaction, which causes your interactive
queries to crash.

Another reason I believe I implemented it this way was a separation of
concerns. Recall that newTransaction() originally created an object of type
Transaction, not StateStore. My intent was to improve the type-safety of
the API, in an effort to ensure Transactions weren't used incorrectly.
Unfortunately, this didn't pan out, but newTransaction() remained.

Finally, this had the added benefit that implementations could easily add
support for transactions *without* re-writing their existing,
non-transactional implementation. I think this can be a benefit both for
implementers of custom StateStores, but also for anyone extending
RocksDbStore, as they can rely on the existing access methods working how
they expect them to.

I'm not too happy with the way the current design has panned out, so I'm
open to ideas on how to improve it. Key to this is finding some way to
ensure that reads from Interactive Query threads are properly isolated from
the transaction, *without* the performance overhead of checking which
thread the method is being called from on every access.

As for replacing flush() with commit() - I saw no reason to add this
complexity to the KIP, unless there was a need to add arguments to the
flush/commit method. This need arises with Atomic Checkpointing, but that
will be implemented separately, in a future KIP. Do you see a need for some
arguments to the flush/commit method that I've missed? Or were you simply
suggesting a rename?

2.
This is simply due to the practical reason that isolationLevel() is really
a proxy for checking if the app is under EOS. The application configuration
is not provided to the constructor of StateStores, but it *is* provided to
init(), via StateStoreContext. For this reason, it seemed somewhat natural
to add it to StateStoreContext. I think this makes sense, since the
IsolationLevel of all StateStores in an application *must* be the same, and
since those stores are all initialized with the same StateStoreContext, it
seems natural for that context to carry the desired IsolationLevel to use.

3.
Using IsolationLevel instead of just passing `boolean eosEnabled`, like
much of the internals was an attempt to logically de-couple the StateStore
API from the internals of Kafka Streams. Technically, StateStores don't
need to know/care what processing mode the KS app is using, all they need
to know is the isolation level expected of them.

Having formal definitions for the expectations of the two required
IsolationLevels allow implementers to implement transactional stores
without having to dig through the internals of Kafka Streams and understand
exactly how they are used. The tight coupling between state stores and
internal behaviour has actually significantly hindered my progress on this
KIP, and encouraged me to avoid increasing this logical coupling as much as
possible.

This also frees implementations to satisfy those requirements in any way
they choose. Transactions might not be the only/available approach to an
implementation, but they might have an alternative way to satisfy the
isolation requirements. I admit that this point is more about semantics,
but "transactional" would need to be formally defined in order for
implementers to provide a valid implementation, and these IsolationLevels
provide that formal definition.

4.
I can remove them. I added them only as I planned to include them in the
org.apache.kafka.streams.state package, as a recommended base
implementation for all StateStores, including those implemented by users. I
had assumed that anything in "public" packages, such as
org.apache.kafka.streams.state, should be included in a KIP. Is that wrong?

5.
RocksDB provides no way to measure the actual size of a
WriteBatch(WithIndex), so we're limited to tracking the sum total of the
size of keys + values that are written to the transaction. This obviously
under-estimates the actual memory usage, because WriteBatch no-doubt
includes some record overheads, and WriteBatchWithIndex has to maintain an
index.

Ideally, we could trivially add a method upstream to WriteBatchInterface
that provides the exact size of the batch, but that would require an
upgrade of RocksDB, which won't happen soon. So for the time being, we're
stuck with an approximation, so I felt that the new method should reflect
that.

Would you prefer the new method name ignores this constraint and that we
simply make the rocks measurement more accurate in the future?

6.
Done

7.
Very good point. The KIP already specifically calls out memory in the
documentation of the config: "Maximum number of memory bytes to be used to
buffer uncommitted state-store records." - did you have something else in
mind?

Should we also make this clearer by renaming the config property itself?
Perhaps to something like statestore.transaction.buffer.max.bytes?

8.
OK, I can remove this. The intent here was to describe how Streams itself
will manage transaction roll-over etc. Presumably that means we also don't
need a description of how Streams will manage the commit of changelog
transactions, state store transactions and checkpointing?

9.
What do you mean by fail-over? Do you mean failing over an Active Task to
an instance already hosting a Standby Task?

Thanks again and sorry for the essay of a response!

Regards,
Nick

On Tue, 20 Jun 2023 at 10:49, Bruno Cadonna <ca...@apache.org> wrote:

> Hi Nick,
>
> Thanks for the updates!
>
> I really appreciate that you simplified the KIP by removing some
> aspects. As I have already told you, I think the removed aspects are
> also good ideas and we can discuss them on follow-up KIPs.
>
> Regarding the current KIP, I have the following feedback.
>
> 1.
> Is there a good reason to add method newTransaction() to the StateStore
> interface? As far as I understand, the idea is that users of a state
> store (transactional or not) call this method at start-up and after each
> commit. Since the call to newTransaction() is done in any case and I
> think it would simplify the caller code if we just start a new
> transaction after a commit in the implementation?
> As far as I understand, you plan to commit the transaction in the
> flush() method. I find the idea to replace flush() with commit()
> presented in KIP-844 an elegant solution.
>
> 2.
> Why is the method to query the isolation level added to the state store
> context?
>
> 3.
> Do we need all the isolation level definitions? I think it is good to
> know the guarantees of the transactionality of the state store.
> However, currently, Streams guarantees that there will only be one
> transaction that writes to the state store. Only the stream thread that
> executes the active task that owns the state store will write to the
> state store. I think it should be enough to know if the state store is
> transactional or not. So my proposal would be to just add a method on
> the state store interface the returns if a state store is transactional
> or not by returning a boolean or an enum.
>
> 4.
> I am wondering why AbstractTransaction and AbstractTransactionalStore
> are part of the KIP. They look like implementation details that should
> not be exposed in the public API.
>
> 5.
> Why does StateStore#approximateNumUncommittedBytes() return an
> approximate number of bytes?
>
> 6.
> RocksDB is just one implementation of the state stores in Streams.
> However, the issues regarding OOM errors might also apply to other
> custom implementations. So in the KIP I would extract that part from
> section "RocksDB Transaction". I would also move section "RocksDB
> Transaction" to the end of section "Proposed Changes" and handle it as
> an example implementation for a state store.
>
> 7.
> Should statestore.uncommitted.max.bytes only limit the uncommitted bytes
> or the uncommitted bytes that reside in memory? In future, other
> transactional state store implementations might implement a buffer for
> uncommitted records that are able to spill records on disk. I think
> statestore.uncommitted.max.bytes needs to limit the uncommitted bytes
> irrespective if they reside in memory or disk. Since Streams will use
> this config to decide if it needs to trigger a commit, state store
> implementations that can spill to disk will never be able to spill to
> disk. You would only need to change the doc of the config, if you agree
> with me.
>
> 8.
> Section "Transaction Management" about the wrappers is rather a
> implementation detail that should not be in the KIP.
>
> 9.
> Could you add a section that describes how failover will work with the
> transactional state stores? I think section "Error handling" is already
> a good start.
>
>
> Best,
> Bruno
>
>
>
>
> On 15.05.23 11:04, Nick Telford wrote:
> > Hi everyone,
> >
> > Quick update: I've added a new section to the KIP: "Offsets for Consumer
> > Rebalances", that outlines my solution to the problem that
> > StreamsPartitionAssignor needs to read StateStore offsets even if they're
> > not currently open.
> >
> > Regards,
> > Nick
> >
> > On Wed, 3 May 2023 at 11:34, Nick Telford <ni...@gmail.com>
> wrote:
> >
> >> Hi Bruno,
> >>
> >> Thanks for reviewing my proposal.
> >>
> >> 1.
> >> The main reason I added it was because it was easy to do. If we see no
> >> value in it, I can remove it.
> >>
> >> 2.
> >> Global StateStores can have multiple partitions in their input topics
> >> (which function as their changelogs), so they would have more than one
> >> partition.
> >>
> >> 3.
> >> That's a good point. At present, the only method it adds is
> >> isolationLevel(), which is likely not necessary outside of StateStores.
> >> It *does* provide slightly different guarantees in the documentation to
> >> several of the methods (hence the overrides). I'm not sure if this is
> >> enough to warrant a new interface though.
> >> I think the question that remains is whether this interface makes it
> >> easier to implement custom transactional StateStores than if we were to
> >> remove it? Probably not.
> >>
> >> 4.
> >> The main motivation for the Atomic Checkpointing is actually
> performance.
> >> My team has been testing out an implementation of this KIP without it,
> and
> >> we had problems with RocksDB doing *much* more compaction, due to the
> >> significantly increased flush rate. It was enough of a problem that (for
> >> the time being), we had to revert back to Kafka Streams proper.
> >> I think the best way to solve this, as you say, is to keep the
> .checkpoint
> >> files *in addition* to the offsets being stored within the store itself.
> >> Essentially, when closing StateStores, we force a memtable flush, then
> >> call getCommittedOffsets and write those out to the .checkpoint file.
> >> That would ensure the metadata is available to the
> >> StreamsPartitionAssignor for all closed stores.
> >> If there's a crash (no clean close), then we won't be able to guarantee
> >> which offsets were flushed to disk by RocksDB, so we'd need to open (
> >> init()), read offsets, and then close() those stores. But since this is
> >> the exception, and will only occur once (provided it doesn't crash every
> >> time!), I think the performance impact here would be acceptable.
> >>
> >> Thanks for the feedback, please let me know if you have any more
> comments
> >> or questions!
> >>
> >> I'm currently working on rebasing against trunk. This involves adding
> >> support for transactionality to VersionedStateStores. I will probably
> need
> >> to revise my implementation for transactional "segmented" stores, both
> to
> >> accommodate VersionedStateStore, and to clean up some other stuff.
> >>
> >> Regards,
> >> Nick
> >>
> >>
> >> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org> wrote:
> >>
> >>> Hi Nick,
> >>>
> >>> Thanks for the updates!
> >>>
> >>> I have a couple of questions/comments.
> >>>
> >>> 1.
> >>> Why do you propose a configuration that involves max. bytes and max.
> >>> reords? I think we are mainly concerned about memory consumption
> because
> >>> we want to limit the off-heap memory used. I cannot think of a case
> >>> where one would want to set the max. number of records.
> >>>
> >>>
> >>> 2.
> >>> Why does
> >>>
> >>>    default void commit(final Map<TopicPartition, Long>
> changelogOffsets) {
> >>>        flush();
> >>>    }
> >>>
> >>> take a map of partitions to changelog offsets?
> >>> The mapping between state stores to partitions is a 1:1 relationship.
> >>> Passing in a single changelog offset should suffice.
> >>>
> >>>
> >>> 3.
> >>> Why do we need the Transaction interface? It should be possible to hide
> >>> beginning and committing a transactions withing the state store
> >>> implementation, so that from outside the state store, it does not
> matter
> >>> whether the state store is transactional or not. What would be the
> >>> advantage of using the Transaction interface?
> >>>
> >>>
> >>> 4.
> >>> Regarding checkpointing offsets, I think we should keep the checkpoint
> >>> file in any case for the reason you mentioned about rebalancing. Even
> if
> >>> that would not be an issue, I would propose to move the change to
> offset
> >>> management to a new KIP and to not add more complexity than needed to
> >>> this one. I would not be too concerned about the consistency violation
> >>> you mention. As far as I understand, with transactional state stores
> >>> Streams would write the checkpoint file during every commit even under
> >>> EOS. In the failure case you describe, Streams would restore the state
> >>> stores from the offsets found in the checkpoint file written during the
> >>> penultimate commit instead of during the last commit. Basically,
> Streams
> >>> would overwrite the records written to the state store between the last
> >>> two commits with the same records read from the changelogs. While I
> >>> understand that this is wasteful, it is -- at the same time --
> >>> acceptable and most importantly it does not break EOS.
> >>>
> >>> Best,
> >>> Bruno
> >>>
> >>>
> >>> On 27.04.23 12:34, Nick Telford wrote:
> >>>> Hi everyone,
> >>>>
> >>>> I find myself (again) considering removing the offset management from
> >>>> StateStores, and keeping the old checkpoint file system. The reason is
> >>> that
> >>>> the StreamPartitionAssignor directly reads checkpoint files in order
> to
> >>>> determine which instance has the most up-to-date copy of the local
> >>> state.
> >>>> If we move offsets into the StateStore itself, then we will need to
> >>> open,
> >>>> initialize, read offsets and then close each StateStore (that is not
> >>>> already assigned and open) for which we have *any* local state, on
> every
> >>>> rebalance.
> >>>>
> >>>> Generally, I don't think there are many "orphan" stores like this
> >>> sitting
> >>>> around on most instances, but even a few would introduce additional
> >>> latency
> >>>> to an already somewhat lengthy rebalance procedure.
> >>>>
> >>>> I'm leaning towards Colt's (Slack) suggestion of just keeping things
> in
> >>> the
> >>>> checkpoint file(s) for now, and not worrying about the race. The
> >>> downside
> >>>> is that we wouldn't be able to remove the explicit RocksDB flush
> >>> on-commit,
> >>>> which likely hurts performance.
> >>>>
> >>>> If anyone has any thoughts or ideas on this subject, I would
> appreciate
> >>> it!
> >>>>
> >>>> Regards,
> >>>> Nick
> >>>>
> >>>> On Wed, 19 Apr 2023 at 15:05, Nick Telford <ni...@gmail.com>
> >>> wrote:
> >>>>
> >>>>> Hi Colt,
> >>>>>
> >>>>> The issue is that if there's a crash between 2 and 3, then you still
> >>> end
> >>>>> up with inconsistent data in RocksDB. The only way to guarantee that
> >>> your
> >>>>> checkpoint offsets and locally stored data are consistent with each
> >>> other
> >>>>> are to atomically commit them, which can be achieved by having the
> >>> offsets
> >>>>> stored in RocksDB.
> >>>>>
> >>>>> The offsets column family is likely to be extremely small (one
> >>>>> per-changelog partition + one per Topology input partition for
> regular
> >>>>> stores, one per input partition for global stores). So the overhead
> >>> will be
> >>>>> minimal.
> >>>>>
> >>>>> A major benefit of doing this is that we can remove the explicit
> calls
> >>> to
> >>>>> db.flush(), which forcibly flushes memtables to disk on-commit. It
> >>> turns
> >>>>> out, RocksDB memtable flushes are largely dictated by Kafka Streams
> >>>>> commits, *not* RocksDB configuration, which could be a major source
> of
> >>>>> confusion. Atomic checkpointing makes it safe to remove these
> explicit
> >>>>> flushes, because it no longer matters exactly when RocksDB flushes
> >>> data to
> >>>>> disk; since the data and corresponding checkpoint offsets will always
> >>> be
> >>>>> flushed together, the local store is always in a consistent state,
> and
> >>>>> on-restart, it can always safely resume restoration from the on-disk
> >>>>> offsets, restoring the small amount of data that hadn't been flushed
> >>> when
> >>>>> the app exited/crashed.
> >>>>>
> >>>>> Regards,
> >>>>> Nick
> >>>>>
> >>>>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io>
> >>> wrote:
> >>>>>
> >>>>>> Nick,
> >>>>>>
> >>>>>> Thanks for your reply. Ack to A) and B).
> >>>>>>
> >>>>>> For item C), I see what you're referring to. Your proposed solution
> >>> will
> >>>>>> work, so no need to change it. What I was suggesting was that it
> >>> might be
> >>>>>> possible to achieve this with only one column family. So long as:
> >>>>>>
> >>>>>>      - No uncommitted records (i.e. not committed to the changelog)
> are
> >>>>>>      *committed* to the state store, AND
> >>>>>>      - The Checkpoint offset (which refers to the changelog topic)
> is
> >>> less
> >>>>>>      than or equal to the last written changelog offset in rocksdb
> >>>>>>
> >>>>>> I don't see the need to do the full restoration from scratch. My
> >>>>>> understanding was that prior to 844/892, full restorations were
> >>> required
> >>>>>> because there could be uncommitted records written to RocksDB;
> >>> however,
> >>>>>> given your use of RocksDB transactions, that can be avoided with the
> >>>>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
> >>> transaction, 3)
> >>>>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <
> nick.telford@gmail.com>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Hi Colt,
> >>>>>>>
> >>>>>>> A. I've done my best to de-couple the StateStore stuff from the
> rest
> >>> of
> >>>>>> the
> >>>>>>> Streams engine. The fact that there will be only one ongoing
> (write)
> >>>>>>> transaction at a time is not guaranteed by any API, and is just a
> >>>>>>> consequence of the way Streams operates. To that end, I tried to
> >>> ensure
> >>>>>> the
> >>>>>>> documentation and guarantees provided by the new APIs are
> >>> independent of
> >>>>>>> this incidental behaviour. In practice, you're right, this
> >>> essentially
> >>>>>>> refers to "interactive queries", which are technically "read
> >>>>>> transactions",
> >>>>>>> even if they don't actually use the transaction API to isolate
> >>>>>> themselves.
> >>>>>>>
> >>>>>>> B. Yes, although not ideal. This is for backwards compatibility,
> >>>>>> because:
> >>>>>>>       1) Existing custom StateStore implementations will implement
> >>>>>> flush(),
> >>>>>>> and not commit(), but the Streams engine now calls commit(), so
> those
> >>>>>> calls
> >>>>>>> need to be forwarded to flush() for these legacy stores.
> >>>>>>>       2) Existing StateStore *users*, i.e. outside of the Streams
> >>> engine
> >>>>>>> itself, may depend on explicitly calling flush(), so for these
> cases,
> >>>>>>> flush() needs to be redirected to call commit().
> >>>>>>> If anyone has a better way to guarantee compatibility without
> >>>>>> introducing
> >>>>>>> this potential recursion loop, I'm open to changes!
> >>>>>>>
> >>>>>>> C. This is described in the "Atomic Checkpointing" section. Offsets
> >>> are
> >>>>>>> stored in a separate RocksDB column family, which is guaranteed to
> be
> >>>>>>> atomically flushed to disk with all other column families. The
> issue
> >>> of
> >>>>>>> checkpoints being written to disk after commit causing
> inconsistency
> >>> if
> >>>>>> it
> >>>>>>> crashes in between is the reason why, under EOS, checkpoint files
> are
> >>>>>> only
> >>>>>>> written on clean shutdown. This is one of the major causes of "full
> >>>>>>> restorations", so moving the offsets into a place where they can be
> >>>>>>> guaranteed to be atomically written with the data they checkpoint
> >>>>>> allows us
> >>>>>>> to write the checkpoint offsets *on every commit*, not just on
> clean
> >>>>>>> shutdown.
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>> Nick
> >>>>>>>
> >>>>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io>
> >>> wrote:
> >>>>>>>
> >>>>>>>> Nick,
> >>>>>>>>
> >>>>>>>> Thank you for continuing this work. I have a few minor clarifying
> >>>>>>>> questions.
> >>>>>>>>
> >>>>>>>> A) "Records written to any transaction are visible to all other
> >>>>>>>> transactions immediately." I am confused here—I thought there
> could
> >>>>>> only
> >>>>>>> be
> >>>>>>>> one transaction going on at a time for a given state store given
> the
> >>>>>>>> threading model for processing records on a Task. Do you mean
> >>>>>> Interactive
> >>>>>>>> Queries by "other transactions"? (If so, then everything makes
> >>> sense—I
> >>>>>>>> thought that since IQ were read-only then they didn't count as
> >>>>>>>> transactions).
> >>>>>>>>
> >>>>>>>> B) Is it intentional that the default implementations of the
> flush()
> >>>>>> and
> >>>>>>>> commit() methods in the StateStore class refer to each other in
> some
> >>>>>> sort
> >>>>>>>> of unbounded recursion?
> >>>>>>>>
> >>>>>>>> C) How will the getCommittedOffset() method work? At first I
> thought
> >>>>>> the
> >>>>>>>> way to do it would be using a special key in the RocksDB store to
> >>>>>> store
> >>>>>>> the
> >>>>>>>> offset, and committing that with the transaction. But upon second
> >>>>>>> thought,
> >>>>>>>> since restoration from the changelog is an idempotent procedure, I
> >>>>>> think
> >>>>>>> it
> >>>>>>>> would be fine to 1) commit the RocksDB transaction and then 2)
> write
> >>>>>> the
> >>>>>>>> offset to disk in a checkpoint file. If there is a crash between
> 1)
> >>>>>> and
> >>>>>>> 2),
> >>>>>>>> I think the only downside is 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 <
> >>> nick.telford@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi everyone,
> >>>>>>>>>
> >>>>>>>>> I've updated the KIP to reflect the latest version of the design:
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>
> >>>>>>>>> There are several changes in there that reflect feedback from
> this
> >>>>>>>> thread,
> >>>>>>>>> and there's a new section and a bunch of interface changes
> relating
> >>>>>> to
> >>>>>>>>> Atomic Checkpointing, which is the final piece of the puzzle to
> >>>>>> making
> >>>>>>>>> everything robust.
> >>>>>>>>>
> >>>>>>>>> Let me know what you think!
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>> Nick
> >>>>>>>>>
> >>>>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <
> nick.telford@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Hi Lucas,
> >>>>>>>>>>
> >>>>>>>>>> Thanks for looking over my KIP.
> >>>>>>>>>>
> >>>>>>>>>> A) The bound is per-instance, not per-Task. This was a typo in
> the
> >>>>>>> KIP
> >>>>>>>>>> that I've now corrected. It was originally per-Task, but I
> >>>>>> changed it
> >>>>>>>> to
> >>>>>>>>>> per-instance for exactly the reason you highlighted.
> >>>>>>>>>> B) It's worth noting that transactionality is only enabled under
> >>>>>> EOS,
> >>>>>>>> and
> >>>>>>>>>> in the default mode of operation (ALOS), there should be no
> >>>>>> change in
> >>>>>>>>>> behavior at all. I think, under EOS, we can mitigate the impact
> on
> >>>>>>>> users
> >>>>>>>>> by
> >>>>>>>>>> sufficiently low default values for the memory bound
> >>>>>> configuration. I
> >>>>>>>>>> understand your hesitation to include a significant change of
> >>>>>>>> behaviour,
> >>>>>>>>>> especially in a minor release, but I suspect that most users
> will
> >>>>>>>> prefer
> >>>>>>>>>> the memory impact (under EOS) to the existing behaviour of
> >>>>>> frequent
> >>>>>>>> state
> >>>>>>>>>> restorations! If this is a problem, the changes can wait until
> the
> >>>>>>> next
> >>>>>>>>>> major release. I'll be running a patched version of streams in
> >>>>>>>> production
> >>>>>>>>>> with these changes as soon as they're ready, so it won't disrupt
> >>>>>> me
> >>>>>>> :-D
> >>>>>>>>>> C) The main purpose of this sentence was just to note that some
> >>>>>>> changes
> >>>>>>>>>> will need to be made to the way Segments are handled in order to
> >>>>>>> ensure
> >>>>>>>>>> they also benefit from transactions. At the time I wrote it, I
> >>>>>> hadn't
> >>>>>>>>>> figured out the specific changes necessary, so it was
> deliberately
> >>>>>>>> vague.
> >>>>>>>>>> This is the one outstanding problem I'm currently working on,
> and
> >>>>>>> I'll
> >>>>>>>>>> update this section with more detail once I have figured out the
> >>>>>>> exact
> >>>>>>>>>> changes required.
> >>>>>>>>>> D) newTransaction() provides the necessary isolation guarantees.
> >>>>>>> While
> >>>>>>>>>> the RocksDB implementation of transactions doesn't technically
> >>>>>> *need*
> >>>>>>>>>> read-only users to call newTransaction(), other implementations
> >>>>>>> (e.g. a
> >>>>>>>>>> hypothetical PostgresStore) may require it. Calling
> >>>>>> newTransaction()
> >>>>>>>> when
> >>>>>>>>>> no transaction is necessary is essentially free, as it will just
> >>>>>>> return
> >>>>>>>>>> this.
> >>>>>>>>>>
> >>>>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think it
> >>>>>> should
> >>>>>>> be
> >>>>>>>>>> fairly obvious where the performance problems stem from: writes
> >>>>>> under
> >>>>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with the
> >>>>>>>>>> tombstone/record flag, 1 to decode it from the tombstone/record
> >>>>>> flag,
> >>>>>>>>> and 1
> >>>>>>>>>> to copy the record from the "temporary" store to the "main"
> store,
> >>>>>>> when
> >>>>>>>>> the
> >>>>>>>>>> transaction commits. The different approach taken by KIP-869
> >>>>>> should
> >>>>>>>>> perform
> >>>>>>>>>> much better, as it avoids all these copies, and may actually
> >>>>>> perform
> >>>>>>>>>> slightly better than trunk, due to batched writes in RocksDB
> >>>>>>> performing
> >>>>>>>>>> better than non-batched writes.[1]
> >>>>>>>>>>
> >>>>>>>>>> Regards,
> >>>>>>>>>> Nick
> >>>>>>>>>>
> >>>>>>>>>> 1:
> >>>>>>>>>
> >>>>>>>
> >>>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>>>
> >>>>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
> >>>>>> lbrutschy@confluent.io
> >>>>>>>>> .invalid>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Nick,
> >>>>>>>>>>>
> >>>>>>>>>>> I'm just starting to read up on the whole discussion about
> >>>>>> KIP-892
> >>>>>>> and
> >>>>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
> >>>>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do have some
> >>>>>>>>>>> questions about the latest draft.
> >>>>>>>>>>>
> >>>>>>>>>>>    A) If I understand correctly, you propose to put a bound on
> the
> >>>>>>>>>>> (native) memory consumed by each task. However, I wonder if
> this
> >>>>>> is
> >>>>>>>>>>> sufficient if we have temporary imbalances in the cluster. For
> >>>>>>>>>>> example, depending on the timing of rebalances during a cluster
> >>>>>>>>>>> restart, it could happen that a single streams node is
> assigned a
> >>>>>>> lot
> >>>>>>>>>>> more tasks than expected. With your proposed change, this would
> >>>>>> mean
> >>>>>>>>>>> that the memory required by this one node could be a multiple
> of
> >>>>>>> what
> >>>>>>>>>>> is required during normal operation. I wonder if it wouldn't be
> >>>>>>> safer
> >>>>>>>>>>> to put a global bound on the memory use, across all tasks.
> >>>>>>>>>>>    B) Generally, the memory concerns still give me the feeling
> >>> that
> >>>>>>> this
> >>>>>>>>>>> should not be enabled by default for all users in a minor
> >>>>>> release.
> >>>>>>>>>>>    C) In section "Transaction Management": the sentence "A
> similar
> >>>>>>>>>>> analogue will be created to automatically manage `Segment`
> >>>>>>>>>>> transactions.". Maybe this is just me lacking some background,
> >>>>>> 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 you to find out why the KIP-844 PoC is 20x slower
> >>>>>> (e.g.
> >>>>>>>>>>> by providing a flame graph), that would be quite interesting.
> >>>>>>>>>>>
> >>>>>>>>>>> Cheers,
> >>>>>>>>>>> Lucas
> >>>>>>>>>>>
> >>>>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
> >>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>
> >>>>>>>>>>>> I've updated the KIP with a more detailed design, which
> >>>>>> reflects
> >>>>>>> the
> >>>>>>>>>>>> implementation I've been working on:
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>
> >>>>>>>>>>>> This new design should address the outstanding points already
> >>>>>> made
> >>>>>>>> in
> >>>>>>>>>>> the
> >>>>>>>>>>>> thread.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Please let me know if there are areas that are unclear or need
> >>>>>>> more
> >>>>>>>>>>>> clarification.
> >>>>>>>>>>>>
> >>>>>>>>>>>> I have a (nearly) working implementation. I'm confident that
> >>>>>> the
> >>>>>>>>>>> remaining
> >>>>>>>>>>>> work (making Segments behave) will not impact the documented
> >>>>>>> design.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Regards,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Nick
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <
> colt@littlehorse.io
> >>>>>>>
> >>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thank you for the reply; that makes sense. I was hoping that,
> >>>>>>>> since
> >>>>>>>>>>> reading
> >>>>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
> >>>>>> documented
> >>>>>>>> API,
> >>>>>>>>>>> maybe
> >>>>>>>>>>>>> you *wouldn't* have to wait for the next major release to
> >>>>>> make
> >>>>>>>> that
> >>>>>>>>>>> change;
> >>>>>>>>>>>>> but given 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 <
> >>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended to
> >>>>>> try
> >>>>>>> to
> >>>>>>>>>>> keep the
> >>>>>>>>>>>>>> behaviour unchanged as much as possible, otherwise we'd
> >>>>>> have
> >>>>>>> to
> >>>>>>>>>>> wait for
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>> major version release to land these changes.
> >>>>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
> >>>>>> guarantee,
> >>>>>>>> and
> >>>>>>>>>>> the
> >>>>>>>>>>>>>> typically longer commit intervals would be problematic when
> >>>>>>>>> reading
> >>>>>>>>>>> only
> >>>>>>>>>>>>>> "committed" records.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I've been away for 5 days recovering from minor surgery,
> >>>>>> but I
> >>>>>>>>>>> spent a
> >>>>>>>>>>>>>> considerable amount of that time working through ideas for
> >>>>>>>>> possible
> >>>>>>>>>>>>>> solutions in my head. I think your suggestion of keeping
> >>>>>> ALOS
> >>>>>>>>>>> as-is, but
> >>>>>>>>>>>>>> buffering writes for EOS is the right path forwards,
> >>>>>> although
> >>>>>>> I
> >>>>>>>>>>> have a
> >>>>>>>>>>>>>> solution that both expands on this, and provides for some
> >>>>>> more
> >>>>>>>>>>> formal
> >>>>>>>>>>>>>> guarantees.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Essentially, adding support to KeyValueStores for
> >>>>>>>> "Transactions",
> >>>>>>>>>>> with
> >>>>>>>>>>>>>> clearly defined IsolationLevels. Using "Read Committed"
> >>>>>> when
> >>>>>>>> under
> >>>>>>>>>>> EOS,
> >>>>>>>>>>>>> and
> >>>>>>>>>>>>>> "Read Uncommitted" under ALOS.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> The nice thing about this approach is that it gives us much
> >>>>>>> more
> >>>>>>>>>>> clearly
> >>>>>>>>>>>>>> defined isolation behaviour that can be properly
> >>>>>> documented to
> >>>>>>>>>>> ensure
> >>>>>>>>>>>>> users
> >>>>>>>>>>>>>> know what to expect.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I'm still working out the kinks in the design, and will
> >>>>>> update
> >>>>>>>> the
> >>>>>>>>>>> KIP
> >>>>>>>>>>>>> when
> >>>>>>>>>>>>>> I have something. The main struggle is trying to implement
> >>>>>>> this
> >>>>>>>>>>> without
> >>>>>>>>>>>>>> making any major changes to the existing interfaces or
> >>>>>>> breaking
> >>>>>>>>>>> existing
> >>>>>>>>>>>>>> implementations, because currently everything expects to
> >>>>>>> operate
> >>>>>>>>>>> directly
> >>>>>>>>>>>>>> on a StateStore, and not a Transaction of that store. I
> >>>>>> think
> >>>>>>>> I'm
> >>>>>>>>>>> getting
> >>>>>>>>>>>>>> close, although sadly I won't be able to progress much
> >>>>>> until
> >>>>>>>> next
> >>>>>>>>>>> week
> >>>>>>>>>>>>> due
> >>>>>>>>>>>>>> to some work commitments.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
> >>>>>>> colt@littlehorse.io>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Nick,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Thank you for the explanation, and also for the updated
> >>>>>>> KIP. I
> >>>>>>>>> am
> >>>>>>>>>>> quite
> >>>>>>>>>>>>>>> eager for this improvement to be released as it would
> >>>>>>> greatly
> >>>>>>>>>>> reduce
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>> operational difficulties of EOS streams apps.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Two questions:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 10)
> >>>>>>>>>>>>>>>> When reading records, we will use the
> >>>>>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
> >>>>>>>>>>>>>>>    and WriteBatchWithIndex#newIteratorWithBase utilities in
> >>>>>>>> order
> >>>>>>>>> to
> >>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>> that uncommitted writes are available to query.
> >>>>>>>>>>>>>>> Why do extra work to enable the reading of uncommitted
> >>>>>>> writes
> >>>>>>>>>>> during
> >>>>>>>>>>>>> IQ?
> >>>>>>>>>>>>>>> Code complexity aside, reading uncommitted writes is, in
> >>>>>> my
> >>>>>>>>>>> opinion, a
> >>>>>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have the
> >>>>>>>>> guarantee
> >>>>>>>>>>> that,
> >>>>>>>>>>>>>>> with EOS, IQ only reads committed records. In order to
> >>>>>> avoid
> >>>>>>>>> dirty
> >>>>>>>>>>>>> reads,
> >>>>>>>>>>>>>>> one currently must query a standby replica (but this
> >>>>>> still
> >>>>>>>>> doesn't
> >>>>>>>>>>>>> fully
> >>>>>>>>>>>>>>> guarantee monotonic reads).
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> 20) Is it also necessary to enable this optimization on
> >>>>>> ALOS
> >>>>>>>>>>> stores?
> >>>>>>>>>>>>> The
> >>>>>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need to
> >>>>>>> restore
> >>>>>>>>>>> state
> >>>>>>>>>>>>> from
> >>>>>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
> >>>>>>> acceptable
> >>>>>>>> to
> >>>>>>>>>>> accept
> >>>>>>>>>>>>>>> that there may have been uncommitted writes on disk. On a
> >>>>>>> side
> >>>>>>>>>>> note, if
> >>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>> enable this type of store on ALOS processors, the
> >>>>>> community
> >>>>>>>>> would
> >>>>>>>>>>>>>>> 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 <
> >>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
> >>>>>>> longer
> >>>>>>>>>>> include
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> StateStore management of checkpointing. This can be
> >>>>>> added
> >>>>>>>> as a
> >>>>>>>>>>> KIP
> >>>>>>>>>>>>>> later
> >>>>>>>>>>>>>>> on
> >>>>>>>>>>>>>>>> to further optimize the consistency and performance of
> >>>>>>> state
> >>>>>>>>>>> stores.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I've also added a section discussing some of the
> >>>>>> concerns
> >>>>>>>>> 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 2022 at 16:36, Nick Telford <
> >>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I didn't do any profiling, but the 844
> >>>>>> implementation:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>      - Writes uncommitted records to a temporary
> >>>>>> RocksDB
> >>>>>>>>>>> instance
> >>>>>>>>>>>>>>>>>         - Since tombstones need to be flagged, all
> >>>>>> record
> >>>>>>>>>>> values are
> >>>>>>>>>>>>>>>>>         prefixed with a value/tombstone marker. This
> >>>>>>>>>>> necessitates a
> >>>>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>> copy.
> >>>>>>>>>>>>>>>>>      - On-commit, iterates all records in this
> >>>>>> temporary
> >>>>>>>>>>> instance and
> >>>>>>>>>>>>>>>>>      writes them to the main RocksDB store.
> >>>>>>>>>>>>>>>>>      - While iterating, the value/tombstone marker
> >>>>>> needs
> >>>>>>> to
> >>>>>>>> be
> >>>>>>>>>>> parsed
> >>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>      the real value extracted. This necessitates
> >>>>>> another
> >>>>>>>>> memory
> >>>>>>>>>>> copy.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> My guess is that the cost of iterating the temporary
> >>>>>>>> RocksDB
> >>>>>>>>>>> store
> >>>>>>>>>>>>> is
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> major factor, with the 2 extra memory copies
> >>>>>> per-Record
> >>>>>>>>>>>>> contributing
> >>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>> significant amount too.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
> >>>>>>>>>>> colt@littlehorse.io>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Out of curiosity, why does the performance of the
> >>>>>> store
> >>>>>>>>>>> degrade so
> >>>>>>>>>>>>>>>>>> significantly with the 844 implementation? I
> >>>>>> wouldn't
> >>>>>>> be
> >>>>>>>>> too
> >>>>>>>>>>>>>> surprised
> >>>>>>>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>> 50-60% drop (caused by each record being written
> >>>>>>> twice),
> >>>>>>>>> but
> >>>>>>>>>>> 96%
> >>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> extreme.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> The only thing I can think of which could create
> >>>>>> such a
> >>>>>>>>>>> bottleneck
> >>>>>>>>>>>>>>> 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 Nick Telford <
> >>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I've updated the KIP to resolve all the points
> >>>>>> that
> >>>>>>>> have
> >>>>>>>>>>> been
> >>>>>>>>>>>>>> raised
> >>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>> far, with one exception: the ALOS default commit
> >>>>>>>> interval
> >>>>>>>>>>> of 5
> >>>>>>>>>>>>>>> minutes
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> likely to cause WriteBatchWithIndex memory to grow
> >>>>>>> too
> >>>>>>>>>>> large.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> There's a couple of different things I can think
> >>>>>> of
> >>>>>>> to
> >>>>>>>>>>> solve
> >>>>>>>>>>>>> this:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>      - We already have a memory/record limit in the
> >>>>>> KIP
> >>>>>>>> to
> >>>>>>>>>>> prevent
> >>>>>>>>>>>>>> OOM
> >>>>>>>>>>>>>>>>>>>      errors. Should we choose a default value for
> >>>>>>> these?
> >>>>>>>> My
> >>>>>>>>>>>>> concern
> >>>>>>>>>>>>>>> here
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>      anything we choose might seem rather
> >>>>>> arbitrary. We
> >>>>>>>>> could
> >>>>>>>>>>>>> change
> >>>>>>>>>>>>>>>>>>>      its behaviour such that under ALOS, it only
> >>>>>>> triggers
> >>>>>>>>> the
> >>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>      StateStore, but under EOS, it triggers a
> >>>>>> commit of
> >>>>>>>> the
> >>>>>>>>>>> Kafka
> >>>>>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>>>>      - We could introduce a separate `
> >>>>>>>>> checkpoint.interval.ms`
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>> allow
> >>>>>>>>>>>>>>>>>> ALOS
> >>>>>>>>>>>>>>>>>>>      to commit the StateStores more frequently than
> >>>>>> the
> >>>>>>>>>>> general
> >>>>>>>>>>>>>>>>>>>      commit.interval.ms? My concern here is that
> >>>>>> the
> >>>>>>>>>>> semantics of
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> config
> >>>>>>>>>>>>>>>>>>>      would depend 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 <
> >>>>>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Hi Alex,
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks for the feedback.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I've updated the discussion of OOM issues by
> >>>>>>>> describing
> >>>>>>>>>>> how
> >>>>>>>>>>>>>> we'll
> >>>>>>>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>>>>> it. Here's the new text:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> To mitigate this, we will automatically force a
> >>>>>>> Task
> >>>>>>>>>>> commit if
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> total
> >>>>>>>>>>>>>>>>>>>>> uncommitted records returned by
> >>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedEntries()
> >>>>>>>>> exceeds a
> >>>>>>>>>>>>>>> threshold,
> >>>>>>>>>>>>>>>>>>>>> configured by
> >>>>>>>> max.uncommitted.state.entries.per.task;
> >>>>>>>>>>> or the
> >>>>>>>>>>>>>>> total
> >>>>>>>>>>>>>>>>>>>>> memory used for buffering uncommitted records
> >>>>>>>> returned
> >>>>>>>>>>> by
> >>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
> >>>>>>> exceeds
> >>>>>>>>> the
> >>>>>>>>>>>>>> threshold
> >>>>>>>>>>>>>>>>>>>>> configured by
> >>>>>>> max.uncommitted.state.bytes.per.task.
> >>>>>>>>>>> This will
> >>>>>>>>>>>>>>>> roughly
> >>>>>>>>>>>>>>>>>>>>> bound the memory required per-Task for
> >>>>>> buffering
> >>>>>>>>>>> uncommitted
> >>>>>>>>>>>>>>>> records,
> >>>>>>>>>>>>>>>>>>>>> irrespective of the commit.interval.ms, and
> >>>>>> will
> >>>>>>>>>>> effectively
> >>>>>>>>>>>>>>> bound
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> number of records that will need to be
> >>>>>> restored in
> >>>>>>>> the
> >>>>>>>>>>> event
> >>>>>>>>>>>>>> of a
> >>>>>>>>>>>>>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> These limits will be checked in
> >>>>>> StreamTask#process
> >>>>>>>> and
> >>>>>>>>> a
> >>>>>>>>>>>>>> premature
> >>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>> will be requested via Task#requestCommit().
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Note that these new methods provide default
> >>>>>>>>>>> implementations
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>>>>> existing custom stores and non-transactional
> >>>>>>> stores
> >>>>>>>>>>> (e.g.
> >>>>>>>>>>>>>>>>>>>>> InMemoryKeyValueStore) do not force any early
> >>>>>>>> commits.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I've chosen to have the StateStore expose
> >>>>>>>>> approximations
> >>>>>>>>>>> of
> >>>>>>>>>>>>> its
> >>>>>>>>>>>>>>>> buffer
> >>>>>>>>>>>>>>>>>>>> size/count instead of opaquely requesting a
> >>>>>> commit
> >>>>>>> in
> >>>>>>>>>>> order to
> >>>>>>>>>>>>>>>>>> delegate
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> decision making to the Task itself. This enables
> >>>>>>>> Tasks
> >>>>>>>>>>> to look
> >>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>> *all*
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> their StateStores, and determine whether an
> >>>>>> early
> >>>>>>>>> commit
> >>>>>>>>>>> is
> >>>>>>>>>>>>>>>> necessary.
> >>>>>>>>>>>>>>>>>>>> Notably, it enables pre-Task thresholds,
> >>>>>> instead of
> >>>>>>>>>>> per-Store,
> >>>>>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>>>>> prevents Tasks with many StateStores from using
> >>>>>>> much
> >>>>>>>>> more
> >>>>>>>>>>>>> memory
> >>>>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>> Tasks
> >>>>>>>>>>>>>>>>>>>> with one StateStore. This makes sense, since
> >>>>>>> commits
> >>>>>>>>> are
> >>>>>>>>>>> done
> >>>>>>>>>>>>>>>> by-Task,
> >>>>>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>> by-Store.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Prizes* for anyone who can come up with a better
> >>>>>>> name
> >>>>>>>>>>> for the
> >>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>>> config
> >>>>>>>>>>>>>>>>>>>> properties!
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks for pointing out the potential
> >>>>>> performance
> >>>>>>>>> issues
> >>>>>>>>>>> of
> >>>>>>>>>>>>>> WBWI.
> >>>>>>>>>>>>>>>> From
> >>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> benchmarks that user posted[1], it looks like
> >>>>>> WBWI
> >>>>>>>>> still
> >>>>>>>>>>>>>> performs
> >>>>>>>>>>>>>>>>>>>> considerably better than individual puts, which
> >>>>>> is
> >>>>>>>> the
> >>>>>>>>>>>>> existing
> >>>>>>>>>>>>>>>>>> design,
> >>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>> I'd actually expect a performance boost from
> >>>>>> WBWI,
> >>>>>>>> just
> >>>>>>>>>>> not as
> >>>>>>>>>>>>>>> great
> >>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>> we'd get from a plain WriteBatch. This does
> >>>>>> suggest
> >>>>>>>>> that
> >>>>>>>>>>> a
> >>>>>>>>>>>>> good
> >>>>>>>>>>>>>>>>>>>> optimization would be to use a regular
> >>>>>> WriteBatch
> >>>>>>> for
> >>>>>>>>>>>>>> restoration
> >>>>>>>>>>>>>>>> (in
> >>>>>>>>>>>>>>>>>>>> RocksDBStore#restoreBatch), since we know that
> >>>>>>> those
> >>>>>>>>>>> records
> >>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>> never
> >>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>> queried before they're committed.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 1:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> * Just kidding, no prizes, sadly.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On Wed, 23 Nov 2022 at 12:28, Alexander
> >>>>>> Sorokoumov
> >>>>>>>>>>>>>>>>>>>> <as...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 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 to keep the current
> >>>>>>>> checkpointing
> >>>>>>>>>>> as a
> >>>>>>>>>>>>>>>> fallback
> >>>>>>>>>>>>>>>>>>>>> mechanism. We want to keep existing users'
> >>>>>>> workflows
> >>>>>>>>>>> intact
> >>>>>>>>>>>>> if
> >>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> can. A
> >>>>>>>>>>>>>>>>>>>>> non-intrusive way would be to add a separate
> >>>>>>>>> StateStore
> >>>>>>>>>>>>> method,
> >>>>>>>>>>>>>>>> say,
> >>>>>>>>>>>>>>>>>>>>> StateStore#managesCheckpointing(), that
> >>>>>> controls
> >>>>>>>>>>> whether the
> >>>>>>>>>>>>>>> state
> >>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>> implementation owns checkpointing.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I think that a solution to the transactional
> >>>>>>> writes
> >>>>>>>>>>> should
> >>>>>>>>>>>>>>> address
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> OOMEs. One possible way to address that is to
> >>>>>> wire
> >>>>>>>>>>>>> StateStore's
> >>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>> request by adding, say, StateStore#commitNeeded
> >>>>>>> that
> >>>>>>>>> is
> >>>>>>>>>>>>> checked
> >>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>> StreamTask#commitNeeded via the corresponding
> >>>>>>>>>>>>>>>> ProcessorStateManager.
> >>>>>>>>>>>>>>>>>>> With
> >>>>>>>>>>>>>>>>>>>>> that change, RocksDBStore will have to track
> >>>>>> the
> >>>>>>>>> current
> >>>>>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>>>>> size
> >>>>>>>>>>>>>>>>>>>>> and request a commit when the size goes over a
> >>>>>>>>>>> (configurable)
> >>>>>>>>>>>>>>>>>> threshold.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> AFAIU WriteBatchWithIndex might perform
> >>>>>>>> significantly
> >>>>>>>>>>> slower
> >>>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>>>>> non-txn
> >>>>>>>>>>>>>>>>>>>>> puts as the batch size grows [1]. We should
> >>>>>> have a
> >>>>>>>>>>>>>> configuration
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> fall
> >>>>>>>>>>>>>>>>>>>>> back to the current behavior (and/or disable
> >>>>>> txn
> >>>>>>>>> stores
> >>>>>>>>>>> for
> >>>>>>>>>>>>>> ALOS)
> >>>>>>>>>>>>>>>>>> unless
> >>>>>>>>>>>>>>>>>>>>> the benchmarks show negligible overhead for
> >>>>>> longer
> >>>>>>>>>>> commits /
> >>>>>>>>>>>>>>>>>>> large-enough
> >>>>>>>>>>>>>>>>>>>>> batch sizes.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> If you prefer to keep the KIP smaller, I would
> >>>>>>>> rather
> >>>>>>>>>>> cut out
> >>>>>>>>>>>>>>>>>>>>> state-store-managed checkpointing rather than
> >>>>>>> proper
> >>>>>>>>>>> OOMe
> >>>>>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>> being able to switch to non-txn behavior. The
> >>>>>>>>>>> checkpointing
> >>>>>>>>>>>>> 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 configuration parts as well as review
> >>>>>> and
> >>>>>>>>> test
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>> patch.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>>>>> Alex
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> 1.
> >>>>>> https://github.com/facebook/rocksdb/issues/608
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
> >>>>>>>>>>>>>>>> nick.telford@gmail.com
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Hi John,
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Thanks for the review and feedback!
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 1. Custom Stores: I've been mulling over this
> >>>>>>>>> problem
> >>>>>>>>>>>>> myself.
> >>>>>>>>>>>>>>> As
> >>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>> stands,
> >>>>>>>>>>>>>>>>>>>>>> custom stores would essentially lose
> >>>>>>> checkpointing
> >>>>>>>>>>> with no
> >>>>>>>>>>>>>>>>>> indication
> >>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> they're expected to make changes, besides a
> >>>>>> line
> >>>>>>>> in
> >>>>>>>>>>> the
> >>>>>>>>>>>>>> release
> >>>>>>>>>>>>>>>>>>> notes. I
> >>>>>>>>>>>>>>>>>>>>>> agree that the best solution would be to
> >>>>>>> provide a
> >>>>>>>>>>> default
> >>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>> checkpoints
> >>>>>>>>>>>>>>>>>>>>>> to a file. The one thing I would change is
> >>>>>> that
> >>>>>>>> the
> >>>>>>>>>>>>>>> checkpointing
> >>>>>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>> store-local file, instead of a per-Task file.
> >>>>>>> This
> >>>>>>>>>>> way the
> >>>>>>>>>>>>>>>>>> StateStore
> >>>>>>>>>>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>>>>>> technically owns its own checkpointing (via a
> >>>>>>>>> default
> >>>>>>>>>>>>>>>>>> implementation),
> >>>>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>> the StateManager/Task execution engine
> >>>>>> doesn't
> >>>>>>>> need
> >>>>>>>>>>> to know
> >>>>>>>>>>>>>>>>>> anything
> >>>>>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>>>>> checkpointing, which greatly simplifies some
> >>>>>> of
> >>>>>>>> the
> >>>>>>>>>>> logic.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 2. OOME errors: The main reasons why I didn't
> >>>>>>>>> explore
> >>>>>>>>>>> a
> >>>>>>>>>>>>>>> solution
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>> this is
> >>>>>>>>>>>>>>>>>>>>>> a) to keep this KIP as simple as possible,
> >>>>>> and
> >>>>>>> b)
> >>>>>>>>>>> because
> >>>>>>>>>>>>> I'm
> >>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>>>>>>>>>>> how to signal that a Task should commit
> >>>>>>>> prematurely.
> >>>>>>>>>>> I'm
> >>>>>>>>>>>>>>>> confident
> >>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>> possible, and I think it's worth adding a
> >>>>>>> section
> >>>>>>>> on
> >>>>>>>>>>>>> handling
> >>>>>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>>>>>> Besides
> >>>>>>>>>>>>>>>>>>>>>> my proposal to force an early commit once
> >>>>>> memory
> >>>>>>>>> usage
> >>>>>>>>>>>>>> reaches
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>> threshold,
> >>>>>>>>>>>>>>>>>>>>>> is there any other approach that you might
> >>>>>>> suggest
> >>>>>>>>> for
> >>>>>>>>>>>>>> tackling
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>> problem?
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 3. ALOS: I can add in an explicit paragraph,
> >>>>>> but
> >>>>>>>> my
> >>>>>>>>>>>>>> assumption
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>> since transactional behaviour comes at
> >>>>>> little/no
> >>>>>>>>>>> cost, that
> >>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>> available by default on all stores,
> >>>>>> irrespective
> >>>>>>>> of
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>>>>> mode.
> >>>>>>>>>>>>>>>>>>>>>> While ALOS doesn't use transactions, the Task
> >>>>>>>> itself
> >>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>> "commits",
> >>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>> the behaviour should be correct under ALOS
> >>>>>> too.
> >>>>>>>> I'm
> >>>>>>>>>>> not
> >>>>>>>>>>>>>>> convinced
> >>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>>> worth having both
> >>>>>>> transactional/non-transactional
> >>>>>>>>>>> stores
> >>>>>>>>>>>>>>>>>> available, as
> >>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>> would considerably increase the complexity of
> >>>>>>> the
> >>>>>>>>>>> codebase,
> >>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>> very
> >>>>>>>>>>>>>>>>>>>>> little
> >>>>>>>>>>>>>>>>>>>>>> benefit.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> 4. Method deprecation: Are you referring to
> >>>>>>>>>>>>>>>>>> StateStore#getPosition()?
> >>>>>>>>>>>>>>>>>>>>> As I
> >>>>>>>>>>>>>>>>>>>>>> understand it, Position contains the
> >>>>>> position of
> >>>>>>>> the
> >>>>>>>>>>>>> *source*
> >>>>>>>>>>>>>>>>>> topics,
> >>>>>>>>>>>>>>>>>>>>>> whereas the commit offsets would be the
> >>>>>>>> *changelog*
> >>>>>>>>>>>>> offsets.
> >>>>>>>>>>>>>> So
> >>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>>>>>> necessary to retain the Position data, as
> >>>>>> well
> >>>>>>> as
> >>>>>>>>> the
> >>>>>>>>>>>>>> changelog
> >>>>>>>>>>>>>>>>>>> offsets.
> >>>>>>>>>>>>>>>>>>>>>> What I meant in the KIP is that Position
> >>>>>> offsets
> >>>>>>>> are
> >>>>>>>>>>>>>> currently
> >>>>>>>>>>>>>>>>>> stored
> >>>>>>>>>>>>>>>>>>>>> in a
> >>>>>>>>>>>>>>>>>>>>>> file, and since we can atomically store
> >>>>>> metadata
> >>>>>>>>>>> along with
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>>>>>> batch we commit to RocksDB, we can move our
> >>>>>>>> Position
> >>>>>>>>>>>>> offsets
> >>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>> metadata too, and gain the same transactional
> >>>>>>>>>>> guarantees
> >>>>>>>>>>>>> that
> >>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>>>>> changelog offsets, ensuring that the Position
> >>>>>>>>> offsets
> >>>>>>>>>>> are
> >>>>>>>>>>>>>>>>>> consistent
> >>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>> the records that are read from the database.
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On Tue, 22 Nov 2022 at 16:25, John Roesler <
> >>>>>>>>>>>>>>> vvcephei@apache.org>
> >>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thanks for publishing this alternative,
> >>>>>> Nick!
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> The benchmark you mentioned in the KIP-844
> >>>>>>>>>>> discussion
> >>>>>>>>>>>>> seems
> >>>>>>>>>>>>>>>> like
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>>>>> compelling reason to revisit the built-in
> >>>>>>>>>>>>> transactionality
> >>>>>>>>>>>>>>>>>>> mechanism.
> >>>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>>> also appreciate you analysis, showing that
> >>>>>> for
> >>>>>>>>> most
> >>>>>>>>>>> use
> >>>>>>>>>>>>>>> cases,
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>>>>> batch approach should be just fine.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> There are a couple of points that would
> >>>>>> hold
> >>>>>>> me
> >>>>>>>>>>> back from
> >>>>>>>>>>>>>>>>>> approving
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> KIP right now:
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 1. Loss of coverage for custom stores.
> >>>>>>>>>>>>>>>>>>>>>>> The fact that you can plug in a
> >>>>>> (relatively)
> >>>>>>>>> simple
> >>>>>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> XStateStore interfaces and automagically
> >>>>>> get a
> >>>>>>>>>>>>> distributed
> >>>>>>>>>>>>>>>>>> database
> >>>>>>>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>> it is a significant benefit of Kafka
> >>>>>> Streams.
> >>>>>>>> I'd
> >>>>>>>>>>> hate to
> >>>>>>>>>>>>>>> lose
> >>>>>>>>>>>>>>>>>> it,
> >>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>>>>> would be better to spend some time and
> >>>>>> come up
> >>>>>>>>> with
> >>>>>>>>>>> a way
> >>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> preserve
> >>>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>> property. For example, can we provide a
> >>>>>>> default
> >>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>>>>> `commit(..)` that re-implements the
> >>>>>> existing
> >>>>>>>>>>>>>> checkpoint-file
> >>>>>>>>>>>>>>>>>>>>> approach? Or
> >>>>>>>>>>>>>>>>>>>>>>> perhaps add an `isTransactional()` flag to
> >>>>>> the
> >>>>>>>>> state
> >>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>>>> that the runtime can decide whether to
> >>>>>>> continue
> >>>>>>>> to
> >>>>>>>>>>> manage
> >>>>>>>>>>>>>>>>>> checkpoint
> >>>>>>>>>>>>>>>>>>>>>> files
> >>>>>>>>>>>>>>>>>>>>>>> vs delegating transactionality to the
> >>>>>> stores?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 2. Guarding against OOME
> >>>>>>>>>>>>>>>>>>>>>>> I appreciate your analysis, but I don't
> >>>>>> think
> >>>>>>>> it's
> >>>>>>>>>>>>>> sufficient
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>>>>> we will solve the memory problem later if
> >>>>>> it
> >>>>>>>>> becomes
> >>>>>>>>>>>>>>> necessary.
> >>>>>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>>>>> experience leading to that situation would
> >>>>>> be
> >>>>>>>>> quite
> >>>>>>>>>>> bad:
> >>>>>>>>>>>>>>>> Imagine,
> >>>>>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>>>>> upgrade to AK 3.next, your tests pass, so
> >>>>>> you
> >>>>>>>>>>> deploy to
> >>>>>>>>>>>>>>>>>> production.
> >>>>>>>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>>>>>> night, you get paged because your app is
> >>>>>> now
> >>>>>>>>>>> crashing
> >>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>> OOMEs. As
> >>>>>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>>> all OOMEs, you'll have a really hard time
> >>>>>>>> finding
> >>>>>>>>>>> the
> >>>>>>>>>>>>> root
> >>>>>>>>>>>>>>>> cause,
> >>>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>>>>> once
> >>>>>>>>>>>>>>>>>>>>>>> you do, you won't have a clear path to
> >>>>>> resolve
> >>>>>>>> the
> >>>>>>>>>>> issue.
> >>>>>>>>>>>>>> You
> >>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>>>>> tune down the commit interval and cache
> >>>>>> buffer
> >>>>>>>>> size
> >>>>>>>>>>> until
> >>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>> stop
> >>>>>>>>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>>>>>>>> crashes.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> FYI, I know of multiple cases where people
> >>>>>> run
> >>>>>>>> EOS
> >>>>>>>>>>> with
> >>>>>>>>>>>>>> much
> >>>>>>>>>>>>>>>>>> larger
> >>>>>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>>>>> intervals to get better batching than the
> >>>>>>>> default,
> >>>>>>>>>>> so I
> >>>>>>>>>>>>>> don't
> >>>>>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>> pathological case would be as rare as you
> >>>>>>>> suspect.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Given that we already have the rudiments
> >>>>>> of an
> >>>>>>>>> idea
> >>>>>>>>>>> of
> >>>>>>>>>>>>> what
> >>>>>>>>>>>>>>> we
> >>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>> do
> >>>>>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> prevent this downside, we should take the
> >>>>>> time
> >>>>>>>> to
> >>>>>>>>>>> design
> >>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>> solution.
> >>>>>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>>>>> owe
> >>>>>>>>>>>>>>>>>>>>>>> it to our users to ensure that awesome new
> >>>>>>>>> features
> >>>>>>>>>>> don't
> >>>>>>>>>>>>>>> come
> >>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>>>> bitter
> >>>>>>>>>>>>>>>>>>>>>>> pills unless we can't avoid it.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 3. ALOS mode.
> >>>>>>>>>>>>>>>>>>>>>>> On the other hand, I didn't see an
> >>>>>> indication
> >>>>>>> of
> >>>>>>>>> how
> >>>>>>>>>>>>> stores
> >>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>>>>> handled under ALOS (aka non-EOS) mode.
> >>>>>>>>>>> Theoretically, the
> >>>>>>>>>>>>>>>>>>>>>> transactionality
> >>>>>>>>>>>>>>>>>>>>>>> of the store and the processing mode are
> >>>>>>>>>>> orthogonal. A
> >>>>>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>>>>> would serve ALOS just as well as a
> >>>>>>>>>>> non-transactional one
> >>>>>>>>>>>>>> (if
> >>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>>>>> better).
> >>>>>>>>>>>>>>>>>>>>>>> Under ALOS, though, the default commit
> >>>>>>> interval
> >>>>>>>> is
> >>>>>>>>>>> five
> >>>>>>>>>>>>>>>> minutes,
> >>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> memory issue is far more pressing.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> As I see it, we have several options to
> >>>>>>> resolve
> >>>>>>>>> this
> >>>>>>>>>>>>> point.
> >>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>>>>> demonstrate that transactional stores work
> >>>>>>> just
> >>>>>>>>>>> fine for
> >>>>>>>>>>>>>> ALOS
> >>>>>>>>>>>>>>>>>> and we
> >>>>>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>>>>> therefore just swap over unconditionally.
> >>>>>> We
> >>>>>>>> could
> >>>>>>>>>>> also
> >>>>>>>>>>>>>>> disable
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> transactional mechanism under ALOS so that
> >>>>>>>> stores
> >>>>>>>>>>> operate
> >>>>>>>>>>>>>>> just
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>>>> they do today when run in ALOS mode.
> >>>>>> Finally,
> >>>>>>> we
> >>>>>>>>>>> could do
> >>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>> KIP-844 and make transactional stores
> >>>>>> opt-in
> >>>>>>>> (it'd
> >>>>>>>>>>> be
> >>>>>>>>>>>>>> better
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> extra opt-in mechanism, but it's a good
> >>>>>>>>>>>>>> get-out-of-jail-free
> >>>>>>>>>>>>>>>>>> card).
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> 4. (minor point) Deprecation of methods
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> You mentioned that the new `commit` 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?
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thanks again for the KIP! It's really nice
> >>>>>>> that
> >>>>>>>>> you
> >>>>>>>>>>> and
> >>>>>>>>>>>>>> Alex
> >>>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>>>> chance to collaborate on both directions so
> >>>>>>> that
> >>>>>>>>> we
> >>>>>>>>>>> can
> >>>>>>>>>>>>> get
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> best
> >>>>>>>>>>>>>>>>>>>>>>> outcome for Streams and its users.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> On 2022/11/21 15:02:15 Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>>>>>>>>>>>> ,
> >>>>>>>>>>>>>>>>>>>>>>>> so that it can be discussed/reviewed
> >>>>>>>> separately
> >>>>>>>>>>> from
> >>>>>>>>>>>>>>> KIP-844.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Alex: I'm especially interested in what
> >>>>>> you
> >>>>>>>>> think!
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> I have a nearly complete implementation
> >>>>>> of
> >>>>>>> the
> >>>>>>>>>>> changes
> >>>>>>>>>>>>>>>>>> outlined in
> >>>>>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>>>>> KIP, please let me know if you'd like me
> >>>>>> to
> >>>>>>>> push
> >>>>>>>>>>> them
> >>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>> review
> >>>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>>>>> advance
> >>>>>>>>>>>>>>>>>>>>>>>> of a vote.
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>

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

Posted by Bruno Cadonna <ca...@apache.org>.
Hi Nick,

Thanks for the updates!

I really appreciate that you simplified the KIP by removing some 
aspects. As I have already told you, I think the removed aspects are 
also good ideas and we can discuss them on follow-up KIPs.

Regarding the current KIP, I have the following feedback.

1.
Is there a good reason to add method newTransaction() to the StateStore 
interface? As far as I understand, the idea is that users of a state 
store (transactional or not) call this method at start-up and after each 
commit. Since the call to newTransaction() is done in any case and I 
think it would simplify the caller code if we just start a new 
transaction after a commit in the implementation?
As far as I understand, you plan to commit the transaction in the 
flush() method. I find the idea to replace flush() with commit() 
presented in KIP-844 an elegant solution.

2.
Why is the method to query the isolation level added to the state store 
context?

3.
Do we need all the isolation level definitions? I think it is good to 
know the guarantees of the transactionality of the state store. 
However, currently, Streams guarantees that there will only be one 
transaction that writes to the state store. Only the stream thread that 
executes the active task that owns the state store will write to the 
state store. I think it should be enough to know if the state store is 
transactional or not. So my proposal would be to just add a method on 
the state store interface the returns if a state store is transactional 
or not by returning a boolean or an enum.

4.
I am wondering why AbstractTransaction and AbstractTransactionalStore 
are part of the KIP. They look like implementation details that should 
not be exposed in the public API.

5.
Why does StateStore#approximateNumUncommittedBytes() return an 
approximate number of bytes?

6.
RocksDB is just one implementation of the state stores in Streams. 
However, the issues regarding OOM errors might also apply to other 
custom implementations. So in the KIP I would extract that part from 
section "RocksDB Transaction". I would also move section "RocksDB 
Transaction" to the end of section "Proposed Changes" and handle it as 
an example implementation for a state store.

7.
Should statestore.uncommitted.max.bytes only limit the uncommitted bytes 
or the uncommitted bytes that reside in memory? In future, other 
transactional state store implementations might implement a buffer for 
uncommitted records that are able to spill records on disk. I think 
statestore.uncommitted.max.bytes needs to limit the uncommitted bytes 
irrespective if they reside in memory or disk. Since Streams will use 
this config to decide if it needs to trigger a commit, state store 
implementations that can spill to disk will never be able to spill to 
disk. You would only need to change the doc of the config, if you agree 
with me.

8.
Section "Transaction Management" about the wrappers is rather a 
implementation detail that should not be in the KIP.

9.
Could you add a section that describes how failover will work with the 
transactional state stores? I think section "Error handling" is already 
a good start.


Best,
Bruno




On 15.05.23 11:04, Nick Telford wrote:
> Hi everyone,
> 
> Quick update: I've added a new section to the KIP: "Offsets for Consumer
> Rebalances", that outlines my solution to the problem that
> StreamsPartitionAssignor needs to read StateStore offsets even if they're
> not currently open.
> 
> Regards,
> Nick
> 
> On Wed, 3 May 2023 at 11:34, Nick Telford <ni...@gmail.com> wrote:
> 
>> Hi Bruno,
>>
>> Thanks for reviewing my proposal.
>>
>> 1.
>> The main reason I added it was because it was easy to do. If we see no
>> value in it, I can remove it.
>>
>> 2.
>> Global StateStores can have multiple partitions in their input topics
>> (which function as their changelogs), so they would have more than one
>> partition.
>>
>> 3.
>> That's a good point. At present, the only method it adds is
>> isolationLevel(), which is likely not necessary outside of StateStores.
>> It *does* provide slightly different guarantees in the documentation to
>> several of the methods (hence the overrides). I'm not sure if this is
>> enough to warrant a new interface though.
>> I think the question that remains is whether this interface makes it
>> easier to implement custom transactional StateStores than if we were to
>> remove it? Probably not.
>>
>> 4.
>> The main motivation for the Atomic Checkpointing is actually performance.
>> My team has been testing out an implementation of this KIP without it, and
>> we had problems with RocksDB doing *much* more compaction, due to the
>> significantly increased flush rate. It was enough of a problem that (for
>> the time being), we had to revert back to Kafka Streams proper.
>> I think the best way to solve this, as you say, is to keep the .checkpoint
>> files *in addition* to the offsets being stored within the store itself.
>> Essentially, when closing StateStores, we force a memtable flush, then
>> call getCommittedOffsets and write those out to the .checkpoint file.
>> That would ensure the metadata is available to the
>> StreamsPartitionAssignor for all closed stores.
>> If there's a crash (no clean close), then we won't be able to guarantee
>> which offsets were flushed to disk by RocksDB, so we'd need to open (
>> init()), read offsets, and then close() those stores. But since this is
>> the exception, and will only occur once (provided it doesn't crash every
>> time!), I think the performance impact here would be acceptable.
>>
>> Thanks for the feedback, please let me know if you have any more comments
>> or questions!
>>
>> I'm currently working on rebasing against trunk. This involves adding
>> support for transactionality to VersionedStateStores. I will probably need
>> to revise my implementation for transactional "segmented" stores, both to
>> accommodate VersionedStateStore, and to clean up some other stuff.
>>
>> Regards,
>> Nick
>>
>>
>> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org> wrote:
>>
>>> Hi Nick,
>>>
>>> Thanks for the updates!
>>>
>>> I have a couple of questions/comments.
>>>
>>> 1.
>>> Why do you propose a configuration that involves max. bytes and max.
>>> reords? I think we are mainly concerned about memory consumption because
>>> we want to limit the off-heap memory used. I cannot think of a case
>>> where one would want to set the max. number of records.
>>>
>>>
>>> 2.
>>> Why does
>>>
>>>    default void commit(final Map<TopicPartition, Long> changelogOffsets) {
>>>        flush();
>>>    }
>>>
>>> take a map of partitions to changelog offsets?
>>> The mapping between state stores to partitions is a 1:1 relationship.
>>> Passing in a single changelog offset should suffice.
>>>
>>>
>>> 3.
>>> Why do we need the Transaction interface? It should be possible to hide
>>> beginning and committing a transactions withing the state store
>>> implementation, so that from outside the state store, it does not matter
>>> whether the state store is transactional or not. What would be the
>>> advantage of using the Transaction interface?
>>>
>>>
>>> 4.
>>> Regarding checkpointing offsets, I think we should keep the checkpoint
>>> file in any case for the reason you mentioned about rebalancing. Even if
>>> that would not be an issue, I would propose to move the change to offset
>>> management to a new KIP and to not add more complexity than needed to
>>> this one. I would not be too concerned about the consistency violation
>>> you mention. As far as I understand, with transactional state stores
>>> Streams would write the checkpoint file during every commit even under
>>> EOS. In the failure case you describe, Streams would restore the state
>>> stores from the offsets found in the checkpoint file written during the
>>> penultimate commit instead of during the last commit. Basically, Streams
>>> would overwrite the records written to the state store between the last
>>> two commits with the same records read from the changelogs. While I
>>> understand that this is wasteful, it is -- at the same time --
>>> acceptable and most importantly it does not break EOS.
>>>
>>> Best,
>>> Bruno
>>>
>>>
>>> On 27.04.23 12:34, Nick Telford wrote:
>>>> Hi everyone,
>>>>
>>>> I find myself (again) considering removing the offset management from
>>>> StateStores, and keeping the old checkpoint file system. The reason is
>>> that
>>>> the StreamPartitionAssignor directly reads checkpoint files in order to
>>>> determine which instance has the most up-to-date copy of the local
>>> state.
>>>> If we move offsets into the StateStore itself, then we will need to
>>> open,
>>>> initialize, read offsets and then close each StateStore (that is not
>>>> already assigned and open) for which we have *any* local state, on every
>>>> rebalance.
>>>>
>>>> Generally, I don't think there are many "orphan" stores like this
>>> sitting
>>>> around on most instances, but even a few would introduce additional
>>> latency
>>>> to an already somewhat lengthy rebalance procedure.
>>>>
>>>> I'm leaning towards Colt's (Slack) suggestion of just keeping things in
>>> the
>>>> checkpoint file(s) for now, and not worrying about the race. The
>>> downside
>>>> is that we wouldn't be able to remove the explicit RocksDB flush
>>> on-commit,
>>>> which likely hurts performance.
>>>>
>>>> If anyone has any thoughts or ideas on this subject, I would appreciate
>>> it!
>>>>
>>>> Regards,
>>>> Nick
>>>>
>>>> On Wed, 19 Apr 2023 at 15:05, Nick Telford <ni...@gmail.com>
>>> wrote:
>>>>
>>>>> Hi Colt,
>>>>>
>>>>> The issue is that if there's a crash between 2 and 3, then you still
>>> end
>>>>> up with inconsistent data in RocksDB. The only way to guarantee that
>>> your
>>>>> checkpoint offsets and locally stored data are consistent with each
>>> other
>>>>> are to atomically commit them, which can be achieved by having the
>>> offsets
>>>>> stored in RocksDB.
>>>>>
>>>>> The offsets column family is likely to be extremely small (one
>>>>> per-changelog partition + one per Topology input partition for regular
>>>>> stores, one per input partition for global stores). So the overhead
>>> will be
>>>>> minimal.
>>>>>
>>>>> A major benefit of doing this is that we can remove the explicit calls
>>> to
>>>>> db.flush(), which forcibly flushes memtables to disk on-commit. It
>>> turns
>>>>> out, RocksDB memtable flushes are largely dictated by Kafka Streams
>>>>> commits, *not* RocksDB configuration, which could be a major source of
>>>>> confusion. Atomic checkpointing makes it safe to remove these explicit
>>>>> flushes, because it no longer matters exactly when RocksDB flushes
>>> data to
>>>>> disk; since the data and corresponding checkpoint offsets will always
>>> be
>>>>> flushed together, the local store is always in a consistent state, and
>>>>> on-restart, it can always safely resume restoration from the on-disk
>>>>> offsets, restoring the small amount of data that hadn't been flushed
>>> when
>>>>> the app exited/crashed.
>>>>>
>>>>> Regards,
>>>>> Nick
>>>>>
>>>>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io>
>>> wrote:
>>>>>
>>>>>> Nick,
>>>>>>
>>>>>> Thanks for your reply. Ack to A) and B).
>>>>>>
>>>>>> For item C), I see what you're referring to. Your proposed solution
>>> will
>>>>>> work, so no need to change it. What I was suggesting was that it
>>> might be
>>>>>> possible to achieve this with only one column family. So long as:
>>>>>>
>>>>>>      - No uncommitted records (i.e. not committed to the changelog) are
>>>>>>      *committed* to the state store, AND
>>>>>>      - The Checkpoint offset (which refers to the changelog topic) is
>>> less
>>>>>>      than or equal to the last written changelog offset in rocksdb
>>>>>>
>>>>>> I don't see the need to do the full restoration from scratch. My
>>>>>> understanding was that prior to 844/892, full restorations were
>>> required
>>>>>> because there could be uncommitted records written to RocksDB;
>>> however,
>>>>>> given your use of RocksDB transactions, that can be avoided with the
>>>>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
>>> transaction, 3)
>>>>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <ni...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi Colt,
>>>>>>>
>>>>>>> A. I've done my best to de-couple the StateStore stuff from the rest
>>> of
>>>>>> the
>>>>>>> Streams engine. The fact that there will be only one ongoing (write)
>>>>>>> transaction at a time is not guaranteed by any API, and is just a
>>>>>>> consequence of the way Streams operates. To that end, I tried to
>>> ensure
>>>>>> the
>>>>>>> documentation and guarantees provided by the new APIs are
>>> independent of
>>>>>>> this incidental behaviour. In practice, you're right, this
>>> essentially
>>>>>>> refers to "interactive queries", which are technically "read
>>>>>> transactions",
>>>>>>> even if they don't actually use the transaction API to isolate
>>>>>> themselves.
>>>>>>>
>>>>>>> B. Yes, although not ideal. This is for backwards compatibility,
>>>>>> because:
>>>>>>>       1) Existing custom StateStore implementations will implement
>>>>>> flush(),
>>>>>>> and not commit(), but the Streams engine now calls commit(), so those
>>>>>> calls
>>>>>>> need to be forwarded to flush() for these legacy stores.
>>>>>>>       2) Existing StateStore *users*, i.e. outside of the Streams
>>> engine
>>>>>>> itself, may depend on explicitly calling flush(), so for these cases,
>>>>>>> flush() needs to be redirected to call commit().
>>>>>>> If anyone has a better way to guarantee compatibility without
>>>>>> introducing
>>>>>>> this potential recursion loop, I'm open to changes!
>>>>>>>
>>>>>>> C. This is described in the "Atomic Checkpointing" section. Offsets
>>> are
>>>>>>> stored in a separate RocksDB column family, which is guaranteed to be
>>>>>>> atomically flushed to disk with all other column families. The issue
>>> of
>>>>>>> checkpoints being written to disk after commit causing inconsistency
>>> if
>>>>>> it
>>>>>>> crashes in between is the reason why, under EOS, checkpoint files are
>>>>>> only
>>>>>>> written on clean shutdown. This is one of the major causes of "full
>>>>>>> restorations", so moving the offsets into a place where they can be
>>>>>>> guaranteed to be atomically written with the data they checkpoint
>>>>>> allows us
>>>>>>> to write the checkpoint offsets *on every commit*, not just on clean
>>>>>>> shutdown.
>>>>>>>
>>>>>>> Regards,
>>>>>>> Nick
>>>>>>>
>>>>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io>
>>> wrote:
>>>>>>>
>>>>>>>> Nick,
>>>>>>>>
>>>>>>>> Thank you for continuing this work. I have a few minor clarifying
>>>>>>>> questions.
>>>>>>>>
>>>>>>>> A) "Records written to any transaction are visible to all other
>>>>>>>> transactions immediately." I am confused here—I thought there could
>>>>>> only
>>>>>>> be
>>>>>>>> one transaction going on at a time for a given state store given the
>>>>>>>> threading model for processing records on a Task. Do you mean
>>>>>> Interactive
>>>>>>>> Queries by "other transactions"? (If so, then everything makes
>>> sense—I
>>>>>>>> thought that since IQ were read-only then they didn't count as
>>>>>>>> transactions).
>>>>>>>>
>>>>>>>> B) Is it intentional that the default implementations of the flush()
>>>>>> and
>>>>>>>> commit() methods in the StateStore class refer to each other in some
>>>>>> sort
>>>>>>>> of unbounded recursion?
>>>>>>>>
>>>>>>>> C) How will the getCommittedOffset() method work? At first I thought
>>>>>> the
>>>>>>>> way to do it would be using a special key in the RocksDB store to
>>>>>> store
>>>>>>> the
>>>>>>>> offset, and committing that with the transaction. But upon second
>>>>>>> thought,
>>>>>>>> since restoration from the changelog is an idempotent procedure, I
>>>>>> think
>>>>>>> it
>>>>>>>> would be fine to 1) commit the RocksDB transaction and then 2) write
>>>>>> the
>>>>>>>> offset to disk in a checkpoint file. If there is a crash between 1)
>>>>>> and
>>>>>>> 2),
>>>>>>>> I think the only downside is 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 <
>>> nick.telford@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi everyone,
>>>>>>>>>
>>>>>>>>> I've updated the KIP to reflect the latest version of the design:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>
>>>>>>>>> There are several changes in there that reflect feedback from this
>>>>>>>> thread,
>>>>>>>>> and there's a new section and a bunch of interface changes relating
>>>>>> to
>>>>>>>>> Atomic Checkpointing, which is the final piece of the puzzle to
>>>>>> making
>>>>>>>>> everything robust.
>>>>>>>>>
>>>>>>>>> Let me know what you think!
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>> Nick
>>>>>>>>>
>>>>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <ni...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Lucas,
>>>>>>>>>>
>>>>>>>>>> Thanks for looking over my KIP.
>>>>>>>>>>
>>>>>>>>>> A) The bound is per-instance, not per-Task. This was a typo in the
>>>>>>> KIP
>>>>>>>>>> that I've now corrected. It was originally per-Task, but I
>>>>>> changed it
>>>>>>>> to
>>>>>>>>>> per-instance for exactly the reason you highlighted.
>>>>>>>>>> B) It's worth noting that transactionality is only enabled under
>>>>>> EOS,
>>>>>>>> and
>>>>>>>>>> in the default mode of operation (ALOS), there should be no
>>>>>> change in
>>>>>>>>>> behavior at all. I think, under EOS, we can mitigate the impact on
>>>>>>>> users
>>>>>>>>> by
>>>>>>>>>> sufficiently low default values for the memory bound
>>>>>> configuration. I
>>>>>>>>>> understand your hesitation to include a significant change of
>>>>>>>> behaviour,
>>>>>>>>>> especially in a minor release, but I suspect that most users will
>>>>>>>> prefer
>>>>>>>>>> the memory impact (under EOS) to the existing behaviour of
>>>>>> frequent
>>>>>>>> state
>>>>>>>>>> restorations! If this is a problem, the changes can wait until the
>>>>>>> next
>>>>>>>>>> major release. I'll be running a patched version of streams in
>>>>>>>> production
>>>>>>>>>> with these changes as soon as they're ready, so it won't disrupt
>>>>>> me
>>>>>>> :-D
>>>>>>>>>> C) The main purpose of this sentence was just to note that some
>>>>>>> changes
>>>>>>>>>> will need to be made to the way Segments are handled in order to
>>>>>>> ensure
>>>>>>>>>> they also benefit from transactions. At the time I wrote it, I
>>>>>> hadn't
>>>>>>>>>> figured out the specific changes necessary, so it was deliberately
>>>>>>>> vague.
>>>>>>>>>> This is the one outstanding problem I'm currently working on, and
>>>>>>> I'll
>>>>>>>>>> update this section with more detail once I have figured out the
>>>>>>> exact
>>>>>>>>>> changes required.
>>>>>>>>>> D) newTransaction() provides the necessary isolation guarantees.
>>>>>>> While
>>>>>>>>>> the RocksDB implementation of transactions doesn't technically
>>>>>> *need*
>>>>>>>>>> read-only users to call newTransaction(), other implementations
>>>>>>> (e.g. a
>>>>>>>>>> hypothetical PostgresStore) may require it. Calling
>>>>>> newTransaction()
>>>>>>>> when
>>>>>>>>>> no transaction is necessary is essentially free, as it will just
>>>>>>> return
>>>>>>>>>> this.
>>>>>>>>>>
>>>>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think it
>>>>>> should
>>>>>>> be
>>>>>>>>>> fairly obvious where the performance problems stem from: writes
>>>>>> under
>>>>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with the
>>>>>>>>>> tombstone/record flag, 1 to decode it from the tombstone/record
>>>>>> flag,
>>>>>>>>> and 1
>>>>>>>>>> to copy the record from the "temporary" store to the "main" store,
>>>>>>> when
>>>>>>>>> the
>>>>>>>>>> transaction commits. The different approach taken by KIP-869
>>>>>> should
>>>>>>>>> perform
>>>>>>>>>> much better, as it avoids all these copies, and may actually
>>>>>> perform
>>>>>>>>>> slightly better than trunk, due to batched writes in RocksDB
>>>>>>> performing
>>>>>>>>>> better than non-batched writes.[1]
>>>>>>>>>>
>>>>>>>>>> Regards,
>>>>>>>>>> Nick
>>>>>>>>>>
>>>>>>>>>> 1:
>>>>>>>>>
>>>>>>>
>>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>
>>>>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
>>>>>> lbrutschy@confluent.io
>>>>>>>>> .invalid>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Nick,
>>>>>>>>>>>
>>>>>>>>>>> I'm just starting to read up on the whole discussion about
>>>>>> KIP-892
>>>>>>> and
>>>>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
>>>>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do have some
>>>>>>>>>>> questions about the latest draft.
>>>>>>>>>>>
>>>>>>>>>>>    A) If I understand correctly, you propose to put a bound on the
>>>>>>>>>>> (native) memory consumed by each task. However, I wonder if this
>>>>>> is
>>>>>>>>>>> sufficient if we have temporary imbalances in the cluster. For
>>>>>>>>>>> example, depending on the timing of rebalances during a cluster
>>>>>>>>>>> restart, it could happen that a single streams node is assigned a
>>>>>>> lot
>>>>>>>>>>> more tasks than expected. With your proposed change, this would
>>>>>> mean
>>>>>>>>>>> that the memory required by this one node could be a multiple of
>>>>>>> what
>>>>>>>>>>> is required during normal operation. I wonder if it wouldn't be
>>>>>>> safer
>>>>>>>>>>> to put a global bound on the memory use, across all tasks.
>>>>>>>>>>>    B) Generally, the memory concerns still give me the feeling
>>> that
>>>>>>> this
>>>>>>>>>>> should not be enabled by default for all users in a minor
>>>>>> release.
>>>>>>>>>>>    C) In section "Transaction Management": the sentence "A similar
>>>>>>>>>>> analogue will be created to automatically manage `Segment`
>>>>>>>>>>> transactions.". Maybe this is just me lacking some background,
>>>>>> 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 you to find out why the KIP-844 PoC is 20x slower
>>>>>> (e.g.
>>>>>>>>>>> by providing a flame graph), that would be quite interesting.
>>>>>>>>>>>
>>>>>>>>>>> Cheers,
>>>>>>>>>>> Lucas
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>
>>>>>>>>>>>> I've updated the KIP with a more detailed design, which
>>>>>> reflects
>>>>>>> the
>>>>>>>>>>>> implementation I've been working on:
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>
>>>>>>>>>>>> This new design should address the outstanding points already
>>>>>> made
>>>>>>>> in
>>>>>>>>>>> the
>>>>>>>>>>>> thread.
>>>>>>>>>>>>
>>>>>>>>>>>> Please let me know if there are areas that are unclear or need
>>>>>>> more
>>>>>>>>>>>> clarification.
>>>>>>>>>>>>
>>>>>>>>>>>> I have a (nearly) working implementation. I'm confident that
>>>>>> the
>>>>>>>>>>> remaining
>>>>>>>>>>>> work (making Segments behave) will not impact the documented
>>>>>>> design.
>>>>>>>>>>>>
>>>>>>>>>>>> Regards,
>>>>>>>>>>>>
>>>>>>>>>>>> Nick
>>>>>>>>>>>>
>>>>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <colt@littlehorse.io
>>>>>>>
>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thank you for the reply; that makes sense. I was hoping that,
>>>>>>>> since
>>>>>>>>>>> reading
>>>>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
>>>>>> documented
>>>>>>>> API,
>>>>>>>>>>> maybe
>>>>>>>>>>>>> you *wouldn't* have to wait for the next major release to
>>>>>> make
>>>>>>>> that
>>>>>>>>>>> change;
>>>>>>>>>>>>> but given 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 <
>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended to
>>>>>> try
>>>>>>> to
>>>>>>>>>>> keep the
>>>>>>>>>>>>>> behaviour unchanged as much as possible, otherwise we'd
>>>>>> have
>>>>>>> to
>>>>>>>>>>> wait for
>>>>>>>>>>>>> a
>>>>>>>>>>>>>> major version release to land these changes.
>>>>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
>>>>>> guarantee,
>>>>>>>> and
>>>>>>>>>>> the
>>>>>>>>>>>>>> typically longer commit intervals would be problematic when
>>>>>>>>> reading
>>>>>>>>>>> only
>>>>>>>>>>>>>> "committed" records.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I've been away for 5 days recovering from minor surgery,
>>>>>> but I
>>>>>>>>>>> spent a
>>>>>>>>>>>>>> considerable amount of that time working through ideas for
>>>>>>>>> possible
>>>>>>>>>>>>>> solutions in my head. I think your suggestion of keeping
>>>>>> ALOS
>>>>>>>>>>> as-is, but
>>>>>>>>>>>>>> buffering writes for EOS is the right path forwards,
>>>>>> although
>>>>>>> I
>>>>>>>>>>> have a
>>>>>>>>>>>>>> solution that both expands on this, and provides for some
>>>>>> more
>>>>>>>>>>> formal
>>>>>>>>>>>>>> guarantees.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Essentially, adding support to KeyValueStores for
>>>>>>>> "Transactions",
>>>>>>>>>>> with
>>>>>>>>>>>>>> clearly defined IsolationLevels. Using "Read Committed"
>>>>>> when
>>>>>>>> under
>>>>>>>>>>> EOS,
>>>>>>>>>>>>> and
>>>>>>>>>>>>>> "Read Uncommitted" under ALOS.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The nice thing about this approach is that it gives us much
>>>>>>> more
>>>>>>>>>>> clearly
>>>>>>>>>>>>>> defined isolation behaviour that can be properly
>>>>>> documented to
>>>>>>>>>>> ensure
>>>>>>>>>>>>> users
>>>>>>>>>>>>>> know what to expect.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I'm still working out the kinks in the design, and will
>>>>>> update
>>>>>>>> the
>>>>>>>>>>> KIP
>>>>>>>>>>>>> when
>>>>>>>>>>>>>> I have something. The main struggle is trying to implement
>>>>>>> this
>>>>>>>>>>> without
>>>>>>>>>>>>>> making any major changes to the existing interfaces or
>>>>>>> breaking
>>>>>>>>>>> existing
>>>>>>>>>>>>>> implementations, because currently everything expects to
>>>>>>> operate
>>>>>>>>>>> directly
>>>>>>>>>>>>>> on a StateStore, and not a Transaction of that store. I
>>>>>> think
>>>>>>>> I'm
>>>>>>>>>>> getting
>>>>>>>>>>>>>> close, although sadly I won't be able to progress much
>>>>>> until
>>>>>>>> next
>>>>>>>>>>> week
>>>>>>>>>>>>> due
>>>>>>>>>>>>>> to some work commitments.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
>>>>>>> colt@littlehorse.io>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Nick,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thank you for the explanation, and also for the updated
>>>>>>> KIP. I
>>>>>>>>> am
>>>>>>>>>>> quite
>>>>>>>>>>>>>>> eager for this improvement to be released as it would
>>>>>>> greatly
>>>>>>>>>>> reduce
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> operational difficulties of EOS streams apps.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Two questions:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 10)
>>>>>>>>>>>>>>>> When reading records, we will use the
>>>>>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
>>>>>>>>>>>>>>>    and WriteBatchWithIndex#newIteratorWithBase utilities in
>>>>>>>> order
>>>>>>>>> to
>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>> that uncommitted writes are available to query.
>>>>>>>>>>>>>>> Why do extra work to enable the reading of uncommitted
>>>>>>> writes
>>>>>>>>>>> during
>>>>>>>>>>>>> IQ?
>>>>>>>>>>>>>>> Code complexity aside, reading uncommitted writes is, in
>>>>>> my
>>>>>>>>>>> opinion, a
>>>>>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have the
>>>>>>>>> guarantee
>>>>>>>>>>> that,
>>>>>>>>>>>>>>> with EOS, IQ only reads committed records. In order to
>>>>>> avoid
>>>>>>>>> dirty
>>>>>>>>>>>>> reads,
>>>>>>>>>>>>>>> one currently must query a standby replica (but this
>>>>>> still
>>>>>>>>> doesn't
>>>>>>>>>>>>> fully
>>>>>>>>>>>>>>> guarantee monotonic reads).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> 20) Is it also necessary to enable this optimization on
>>>>>> ALOS
>>>>>>>>>>> stores?
>>>>>>>>>>>>> The
>>>>>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need to
>>>>>>> restore
>>>>>>>>>>> state
>>>>>>>>>>>>> from
>>>>>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
>>>>>>> acceptable
>>>>>>>> to
>>>>>>>>>>> accept
>>>>>>>>>>>>>>> that there may have been uncommitted writes on disk. On a
>>>>>>> side
>>>>>>>>>>> note, if
>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>> enable this type of store on ALOS processors, the
>>>>>> community
>>>>>>>>> would
>>>>>>>>>>>>>>> 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 <
>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
>>>>>>> longer
>>>>>>>>>>> include
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> StateStore management of checkpointing. This can be
>>>>>> added
>>>>>>>> as a
>>>>>>>>>>> KIP
>>>>>>>>>>>>>> later
>>>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>> to further optimize the consistency and performance of
>>>>>>> state
>>>>>>>>>>> stores.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I've also added a section discussing some of the
>>>>>> concerns
>>>>>>>>> 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 2022 at 16:36, Nick Telford <
>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I didn't do any profiling, but the 844
>>>>>> implementation:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>      - Writes uncommitted records to a temporary
>>>>>> RocksDB
>>>>>>>>>>> instance
>>>>>>>>>>>>>>>>>         - Since tombstones need to be flagged, all
>>>>>> record
>>>>>>>>>>> values are
>>>>>>>>>>>>>>>>>         prefixed with a value/tombstone marker. This
>>>>>>>>>>> necessitates a
>>>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>> copy.
>>>>>>>>>>>>>>>>>      - On-commit, iterates all records in this
>>>>>> temporary
>>>>>>>>>>> instance and
>>>>>>>>>>>>>>>>>      writes them to the main RocksDB store.
>>>>>>>>>>>>>>>>>      - While iterating, the value/tombstone marker
>>>>>> needs
>>>>>>> to
>>>>>>>> be
>>>>>>>>>>> parsed
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>      the real value extracted. This necessitates
>>>>>> another
>>>>>>>>> memory
>>>>>>>>>>> copy.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> My guess is that the cost of iterating the temporary
>>>>>>>> RocksDB
>>>>>>>>>>> store
>>>>>>>>>>>>> is
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> major factor, with the 2 extra memory copies
>>>>>> per-Record
>>>>>>>>>>>>> contributing
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> significant amount too.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
>>>>>>>>>>> colt@littlehorse.io>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Out of curiosity, why does the performance of the
>>>>>> store
>>>>>>>>>>> degrade so
>>>>>>>>>>>>>>>>>> significantly with the 844 implementation? I
>>>>>> wouldn't
>>>>>>> be
>>>>>>>>> too
>>>>>>>>>>>>>> surprised
>>>>>>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> 50-60% drop (caused by each record being written
>>>>>>> twice),
>>>>>>>>> but
>>>>>>>>>>> 96%
>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> extreme.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The only thing I can think of which could create
>>>>>> such a
>>>>>>>>>>> bottleneck
>>>>>>>>>>>>>>> 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 Nick Telford <
>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I've updated the KIP to resolve all the points
>>>>>> that
>>>>>>>> have
>>>>>>>>>>> been
>>>>>>>>>>>>>> raised
>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>> far, with one exception: the ALOS default commit
>>>>>>>> interval
>>>>>>>>>>> of 5
>>>>>>>>>>>>>>> minutes
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> likely to cause WriteBatchWithIndex memory to grow
>>>>>>> too
>>>>>>>>>>> large.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> There's a couple of different things I can think
>>>>>> of
>>>>>>> to
>>>>>>>>>>> solve
>>>>>>>>>>>>> this:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>      - We already have a memory/record limit in the
>>>>>> KIP
>>>>>>>> to
>>>>>>>>>>> prevent
>>>>>>>>>>>>>> OOM
>>>>>>>>>>>>>>>>>>>      errors. Should we choose a default value for
>>>>>>> these?
>>>>>>>> My
>>>>>>>>>>>>> concern
>>>>>>>>>>>>>>> here
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>      anything we choose might seem rather
>>>>>> arbitrary. We
>>>>>>>>> could
>>>>>>>>>>>>> change
>>>>>>>>>>>>>>>>>>>      its behaviour such that under ALOS, it only
>>>>>>> triggers
>>>>>>>>> the
>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>      StateStore, but under EOS, it triggers a
>>>>>> commit of
>>>>>>>> the
>>>>>>>>>>> Kafka
>>>>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>>>>      - We could introduce a separate `
>>>>>>>>> checkpoint.interval.ms`
>>>>>>>>>>> to
>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>>>>      to commit the StateStores more frequently than
>>>>>> the
>>>>>>>>>>> general
>>>>>>>>>>>>>>>>>>>      commit.interval.ms? My concern here is that
>>>>>> the
>>>>>>>>>>> semantics of
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>      would depend 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 <
>>>>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hi Alex,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks for the feedback.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I've updated the discussion of OOM issues by
>>>>>>>> describing
>>>>>>>>>>> how
>>>>>>>>>>>>>> we'll
>>>>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>>>>> it. Here's the new text:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> To mitigate this, we will automatically force a
>>>>>>> Task
>>>>>>>>>>> commit if
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> total
>>>>>>>>>>>>>>>>>>>>> uncommitted records returned by
>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedEntries()
>>>>>>>>> exceeds a
>>>>>>>>>>>>>>> threshold,
>>>>>>>>>>>>>>>>>>>>> configured by
>>>>>>>> max.uncommitted.state.entries.per.task;
>>>>>>>>>>> or the
>>>>>>>>>>>>>>> total
>>>>>>>>>>>>>>>>>>>>> memory used for buffering uncommitted records
>>>>>>>> returned
>>>>>>>>>>> by
>>>>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
>>>>>>> exceeds
>>>>>>>>> the
>>>>>>>>>>>>>> threshold
>>>>>>>>>>>>>>>>>>>>> configured by
>>>>>>> max.uncommitted.state.bytes.per.task.
>>>>>>>>>>> This will
>>>>>>>>>>>>>>>> roughly
>>>>>>>>>>>>>>>>>>>>> bound the memory required per-Task for
>>>>>> buffering
>>>>>>>>>>> uncommitted
>>>>>>>>>>>>>>>> records,
>>>>>>>>>>>>>>>>>>>>> irrespective of the commit.interval.ms, and
>>>>>> will
>>>>>>>>>>> effectively
>>>>>>>>>>>>>>> bound
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> number of records that will need to be
>>>>>> restored in
>>>>>>>> the
>>>>>>>>>>> event
>>>>>>>>>>>>>> of a
>>>>>>>>>>>>>>>>>>> failure.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> These limits will be checked in
>>>>>> StreamTask#process
>>>>>>>> and
>>>>>>>>> a
>>>>>>>>>>>>>> premature
>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>> will be requested via Task#requestCommit().
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Note that these new methods provide default
>>>>>>>>>>> implementations
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>>>>> existing custom stores and non-transactional
>>>>>>> stores
>>>>>>>>>>> (e.g.
>>>>>>>>>>>>>>>>>>>>> InMemoryKeyValueStore) do not force any early
>>>>>>>> commits.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I've chosen to have the StateStore expose
>>>>>>>>> approximations
>>>>>>>>>>> of
>>>>>>>>>>>>> its
>>>>>>>>>>>>>>>> buffer
>>>>>>>>>>>>>>>>>>>> size/count instead of opaquely requesting a
>>>>>> commit
>>>>>>> in
>>>>>>>>>>> order to
>>>>>>>>>>>>>>>>>> delegate
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> decision making to the Task itself. This enables
>>>>>>>> Tasks
>>>>>>>>>>> to look
>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>> *all*
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> their StateStores, and determine whether an
>>>>>> early
>>>>>>>>> commit
>>>>>>>>>>> is
>>>>>>>>>>>>>>>> necessary.
>>>>>>>>>>>>>>>>>>>> Notably, it enables pre-Task thresholds,
>>>>>> instead of
>>>>>>>>>>> per-Store,
>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>> prevents Tasks with many StateStores from using
>>>>>>> much
>>>>>>>>> more
>>>>>>>>>>>>> memory
>>>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>> Tasks
>>>>>>>>>>>>>>>>>>>> with one StateStore. This makes sense, since
>>>>>>> commits
>>>>>>>>> are
>>>>>>>>>>> done
>>>>>>>>>>>>>>>> by-Task,
>>>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>> by-Store.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Prizes* for anyone who can come up with a better
>>>>>>> name
>>>>>>>>>>> for the
>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>>>>> properties!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks for pointing out the potential
>>>>>> performance
>>>>>>>>> issues
>>>>>>>>>>> of
>>>>>>>>>>>>>> WBWI.
>>>>>>>>>>>>>>>> From
>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> benchmarks that user posted[1], it looks like
>>>>>> WBWI
>>>>>>>>> still
>>>>>>>>>>>>>> performs
>>>>>>>>>>>>>>>>>>>> considerably better than individual puts, which
>>>>>> is
>>>>>>>> the
>>>>>>>>>>>>> existing
>>>>>>>>>>>>>>>>>> design,
>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>> I'd actually expect a performance boost from
>>>>>> WBWI,
>>>>>>>> just
>>>>>>>>>>> not as
>>>>>>>>>>>>>>> great
>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>> we'd get from a plain WriteBatch. This does
>>>>>> suggest
>>>>>>>>> that
>>>>>>>>>>> a
>>>>>>>>>>>>> good
>>>>>>>>>>>>>>>>>>>> optimization would be to use a regular
>>>>>> WriteBatch
>>>>>>> for
>>>>>>>>>>>>>> restoration
>>>>>>>>>>>>>>>> (in
>>>>>>>>>>>>>>>>>>>> RocksDBStore#restoreBatch), since we know that
>>>>>>> those
>>>>>>>>>>> records
>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>> never
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> queried before they're committed.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 1:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>
>>>>>>
>>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> * Just kidding, no prizes, sadly.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Wed, 23 Nov 2022 at 12:28, Alexander
>>>>>> Sorokoumov
>>>>>>>>>>>>>>>>>>>> <as...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 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 to keep the current
>>>>>>>> checkpointing
>>>>>>>>>>> as a
>>>>>>>>>>>>>>>> fallback
>>>>>>>>>>>>>>>>>>>>> mechanism. We want to keep existing users'
>>>>>>> workflows
>>>>>>>>>>> intact
>>>>>>>>>>>>> if
>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> can. A
>>>>>>>>>>>>>>>>>>>>> non-intrusive way would be to add a separate
>>>>>>>>> StateStore
>>>>>>>>>>>>> method,
>>>>>>>>>>>>>>>> say,
>>>>>>>>>>>>>>>>>>>>> StateStore#managesCheckpointing(), that
>>>>>> controls
>>>>>>>>>>> whether the
>>>>>>>>>>>>>>> state
>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>> implementation owns checkpointing.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I think that a solution to the transactional
>>>>>>> writes
>>>>>>>>>>> should
>>>>>>>>>>>>>>> address
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> OOMEs. One possible way to address that is to
>>>>>> wire
>>>>>>>>>>>>> StateStore's
>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>> request by adding, say, StateStore#commitNeeded
>>>>>>> that
>>>>>>>>> is
>>>>>>>>>>>>> checked
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>> StreamTask#commitNeeded via the corresponding
>>>>>>>>>>>>>>>> ProcessorStateManager.
>>>>>>>>>>>>>>>>>>> With
>>>>>>>>>>>>>>>>>>>>> that change, RocksDBStore will have to track
>>>>>> the
>>>>>>>>> current
>>>>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>>>>> size
>>>>>>>>>>>>>>>>>>>>> and request a commit when the size goes over a
>>>>>>>>>>> (configurable)
>>>>>>>>>>>>>>>>>> threshold.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> AFAIU WriteBatchWithIndex might perform
>>>>>>>> significantly
>>>>>>>>>>> slower
>>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>>>>> non-txn
>>>>>>>>>>>>>>>>>>>>> puts as the batch size grows [1]. We should
>>>>>> have a
>>>>>>>>>>>>>> configuration
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> fall
>>>>>>>>>>>>>>>>>>>>> back to the current behavior (and/or disable
>>>>>> txn
>>>>>>>>> stores
>>>>>>>>>>> for
>>>>>>>>>>>>>> ALOS)
>>>>>>>>>>>>>>>>>> unless
>>>>>>>>>>>>>>>>>>>>> the benchmarks show negligible overhead for
>>>>>> longer
>>>>>>>>>>> commits /
>>>>>>>>>>>>>>>>>>> large-enough
>>>>>>>>>>>>>>>>>>>>> batch sizes.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> If you prefer to keep the KIP smaller, I would
>>>>>>>> rather
>>>>>>>>>>> cut out
>>>>>>>>>>>>>>>>>>>>> state-store-managed checkpointing rather than
>>>>>>> proper
>>>>>>>>>>> OOMe
>>>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>> being able to switch to non-txn behavior. The
>>>>>>>>>>> checkpointing
>>>>>>>>>>>>> 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 configuration parts as well as review
>>>>>> and
>>>>>>>>> test
>>>>>>>>>>> the
>>>>>>>>>>>>>>> patch.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>> Alex
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> 1.
>>>>>> https://github.com/facebook/rocksdb/issues/608
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
>>>>>>>>>>>>>>>> nick.telford@gmail.com
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks for the review and feedback!
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 1. Custom Stores: I've been mulling over this
>>>>>>>>> problem
>>>>>>>>>>>>> myself.
>>>>>>>>>>>>>>> As
>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>> stands,
>>>>>>>>>>>>>>>>>>>>>> custom stores would essentially lose
>>>>>>> checkpointing
>>>>>>>>>>> with no
>>>>>>>>>>>>>>>>>> indication
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> they're expected to make changes, besides a
>>>>>> line
>>>>>>>> in
>>>>>>>>>>> the
>>>>>>>>>>>>>> release
>>>>>>>>>>>>>>>>>>> notes. I
>>>>>>>>>>>>>>>>>>>>>> agree that the best solution would be to
>>>>>>> provide a
>>>>>>>>>>> default
>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>> checkpoints
>>>>>>>>>>>>>>>>>>>>>> to a file. The one thing I would change is
>>>>>> that
>>>>>>>> the
>>>>>>>>>>>>>>> checkpointing
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>> store-local file, instead of a per-Task file.
>>>>>>> This
>>>>>>>>>>> way the
>>>>>>>>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>> technically owns its own checkpointing (via a
>>>>>>>>> default
>>>>>>>>>>>>>>>>>> implementation),
>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> the StateManager/Task execution engine
>>>>>> doesn't
>>>>>>>> need
>>>>>>>>>>> to know
>>>>>>>>>>>>>>>>>> anything
>>>>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>>>>> checkpointing, which greatly simplifies some
>>>>>> of
>>>>>>>> the
>>>>>>>>>>> logic.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 2. OOME errors: The main reasons why I didn't
>>>>>>>>> explore
>>>>>>>>>>> a
>>>>>>>>>>>>>>> solution
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> this is
>>>>>>>>>>>>>>>>>>>>>> a) to keep this KIP as simple as possible,
>>>>>> and
>>>>>>> b)
>>>>>>>>>>> because
>>>>>>>>>>>>> I'm
>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>>>>> how to signal that a Task should commit
>>>>>>>> prematurely.
>>>>>>>>>>> I'm
>>>>>>>>>>>>>>>> confident
>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>> possible, and I think it's worth adding a
>>>>>>> section
>>>>>>>> on
>>>>>>>>>>>>> handling
>>>>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>>>>> Besides
>>>>>>>>>>>>>>>>>>>>>> my proposal to force an early commit once
>>>>>> memory
>>>>>>>>> usage
>>>>>>>>>>>>>> reaches
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>> threshold,
>>>>>>>>>>>>>>>>>>>>>> is there any other approach that you might
>>>>>>> suggest
>>>>>>>>> for
>>>>>>>>>>>>>> tackling
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>> problem?
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 3. ALOS: I can add in an explicit paragraph,
>>>>>> but
>>>>>>>> my
>>>>>>>>>>>>>> assumption
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> since transactional behaviour comes at
>>>>>> little/no
>>>>>>>>>>> cost, that
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>> available by default on all stores,
>>>>>> irrespective
>>>>>>>> of
>>>>>>>>>>> the
>>>>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>>>>> mode.
>>>>>>>>>>>>>>>>>>>>>> While ALOS doesn't use transactions, the Task
>>>>>>>> itself
>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>> "commits",
>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>> the behaviour should be correct under ALOS
>>>>>> too.
>>>>>>>> I'm
>>>>>>>>>>> not
>>>>>>>>>>>>>>> convinced
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>>> worth having both
>>>>>>> transactional/non-transactional
>>>>>>>>>>> stores
>>>>>>>>>>>>>>>>>> available, as
>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>> would considerably increase the complexity of
>>>>>>> the
>>>>>>>>>>> codebase,
>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> very
>>>>>>>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>>>>>>>> benefit.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> 4. Method deprecation: Are you referring to
>>>>>>>>>>>>>>>>>> StateStore#getPosition()?
>>>>>>>>>>>>>>>>>>>>> As I
>>>>>>>>>>>>>>>>>>>>>> understand it, Position contains the
>>>>>> position of
>>>>>>>> the
>>>>>>>>>>>>> *source*
>>>>>>>>>>>>>>>>>> topics,
>>>>>>>>>>>>>>>>>>>>>> whereas the commit offsets would be the
>>>>>>>> *changelog*
>>>>>>>>>>>>> offsets.
>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>>>>> necessary to retain the Position data, as
>>>>>> well
>>>>>>> as
>>>>>>>>> the
>>>>>>>>>>>>>> changelog
>>>>>>>>>>>>>>>>>>> offsets.
>>>>>>>>>>>>>>>>>>>>>> What I meant in the KIP is that Position
>>>>>> offsets
>>>>>>>> are
>>>>>>>>>>>>>> currently
>>>>>>>>>>>>>>>>>> stored
>>>>>>>>>>>>>>>>>>>>> in a
>>>>>>>>>>>>>>>>>>>>>> file, and since we can atomically store
>>>>>> metadata
>>>>>>>>>>> along with
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>>>>> batch we commit to RocksDB, we can move our
>>>>>>>> Position
>>>>>>>>>>>>> offsets
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>> metadata too, and gain the same transactional
>>>>>>>>>>> guarantees
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>>>>> changelog offsets, ensuring that the Position
>>>>>>>>> offsets
>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>> consistent
>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>> the records that are read from the database.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Tue, 22 Nov 2022 at 16:25, John Roesler <
>>>>>>>>>>>>>>> vvcephei@apache.org>
>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks for publishing this alternative,
>>>>>> Nick!
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> The benchmark you mentioned in the KIP-844
>>>>>>>>>>> discussion
>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>> compelling reason to revisit the built-in
>>>>>>>>>>>>> transactionality
>>>>>>>>>>>>>>>>>>> mechanism.
>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>> also appreciate you analysis, showing that
>>>>>> for
>>>>>>>>> most
>>>>>>>>>>> use
>>>>>>>>>>>>>>> cases,
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>>>>> batch approach should be just fine.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> There are a couple of points that would
>>>>>> hold
>>>>>>> me
>>>>>>>>>>> back from
>>>>>>>>>>>>>>>>>> approving
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> KIP right now:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 1. Loss of coverage for custom stores.
>>>>>>>>>>>>>>>>>>>>>>> The fact that you can plug in a
>>>>>> (relatively)
>>>>>>>>> simple
>>>>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> XStateStore interfaces and automagically
>>>>>> get a
>>>>>>>>>>>>> distributed
>>>>>>>>>>>>>>>>>> database
>>>>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>> it is a significant benefit of Kafka
>>>>>> Streams.
>>>>>>>> I'd
>>>>>>>>>>> hate to
>>>>>>>>>>>>>>> lose
>>>>>>>>>>>>>>>>>> it,
>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>> would be better to spend some time and
>>>>>> come up
>>>>>>>>> with
>>>>>>>>>>> a way
>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> preserve
>>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>> property. For example, can we provide a
>>>>>>> default
>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>> `commit(..)` that re-implements the
>>>>>> existing
>>>>>>>>>>>>>> checkpoint-file
>>>>>>>>>>>>>>>>>>>>> approach? Or
>>>>>>>>>>>>>>>>>>>>>>> perhaps add an `isTransactional()` flag to
>>>>>> the
>>>>>>>>> state
>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>>>> that the runtime can decide whether to
>>>>>>> continue
>>>>>>>> to
>>>>>>>>>>> manage
>>>>>>>>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>>>>>>>>> files
>>>>>>>>>>>>>>>>>>>>>>> vs delegating transactionality to the
>>>>>> stores?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 2. Guarding against OOME
>>>>>>>>>>>>>>>>>>>>>>> I appreciate your analysis, but I don't
>>>>>> think
>>>>>>>> it's
>>>>>>>>>>>>>> sufficient
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>>> we will solve the memory problem later if
>>>>>> it
>>>>>>>>> becomes
>>>>>>>>>>>>>>> necessary.
>>>>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>>>>> experience leading to that situation would
>>>>>> be
>>>>>>>>> quite
>>>>>>>>>>> bad:
>>>>>>>>>>>>>>>> Imagine,
>>>>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>>>>> upgrade to AK 3.next, your tests pass, so
>>>>>> you
>>>>>>>>>>> deploy to
>>>>>>>>>>>>>>>>>> production.
>>>>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>>>>> night, you get paged because your app is
>>>>>> now
>>>>>>>>>>> crashing
>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>> OOMEs. As
>>>>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>>> all OOMEs, you'll have a really hard time
>>>>>>>> finding
>>>>>>>>>>> the
>>>>>>>>>>>>> root
>>>>>>>>>>>>>>>> cause,
>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>> once
>>>>>>>>>>>>>>>>>>>>>>> you do, you won't have a clear path to
>>>>>> resolve
>>>>>>>> the
>>>>>>>>>>> issue.
>>>>>>>>>>>>>> You
>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>>>>> tune down the commit interval and cache
>>>>>> buffer
>>>>>>>>> size
>>>>>>>>>>> until
>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>> stop
>>>>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>>>>> crashes.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> FYI, I know of multiple cases where people
>>>>>> run
>>>>>>>> EOS
>>>>>>>>>>> with
>>>>>>>>>>>>>> much
>>>>>>>>>>>>>>>>>> larger
>>>>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>>>>> intervals to get better batching than the
>>>>>>>> default,
>>>>>>>>>>> so I
>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>> pathological case would be as rare as you
>>>>>>>> suspect.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Given that we already have the rudiments
>>>>>> of an
>>>>>>>>> idea
>>>>>>>>>>> of
>>>>>>>>>>>>> what
>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> prevent this downside, we should take the
>>>>>> time
>>>>>>>> to
>>>>>>>>>>> design
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> solution.
>>>>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>>>>> owe
>>>>>>>>>>>>>>>>>>>>>>> it to our users to ensure that awesome new
>>>>>>>>> features
>>>>>>>>>>> don't
>>>>>>>>>>>>>>> come
>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>>>> bitter
>>>>>>>>>>>>>>>>>>>>>>> pills unless we can't avoid it.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 3. ALOS mode.
>>>>>>>>>>>>>>>>>>>>>>> On the other hand, I didn't see an
>>>>>> indication
>>>>>>> of
>>>>>>>>> how
>>>>>>>>>>>>> stores
>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> handled under ALOS (aka non-EOS) mode.
>>>>>>>>>>> Theoretically, the
>>>>>>>>>>>>>>>>>>>>>> transactionality
>>>>>>>>>>>>>>>>>>>>>>> of the store and the processing mode are
>>>>>>>>>>> orthogonal. A
>>>>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>>>>> would serve ALOS just as well as a
>>>>>>>>>>> non-transactional one
>>>>>>>>>>>>>> (if
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>> better).
>>>>>>>>>>>>>>>>>>>>>>> Under ALOS, though, the default commit
>>>>>>> interval
>>>>>>>> is
>>>>>>>>>>> five
>>>>>>>>>>>>>>>> minutes,
>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> memory issue is far more pressing.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> As I see it, we have several options to
>>>>>>> resolve
>>>>>>>>> this
>>>>>>>>>>>>> point.
>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>>>>> demonstrate that transactional stores work
>>>>>>> just
>>>>>>>>>>> fine for
>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>>> and we
>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>> therefore just swap over unconditionally.
>>>>>> We
>>>>>>>> could
>>>>>>>>>>> also
>>>>>>>>>>>>>>> disable
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> transactional mechanism under ALOS so that
>>>>>>>> stores
>>>>>>>>>>> operate
>>>>>>>>>>>>>>> just
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>> they do today when run in ALOS mode.
>>>>>> Finally,
>>>>>>> we
>>>>>>>>>>> could do
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> KIP-844 and make transactional stores
>>>>>> opt-in
>>>>>>>> (it'd
>>>>>>>>>>> be
>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> extra opt-in mechanism, but it's a good
>>>>>>>>>>>>>> get-out-of-jail-free
>>>>>>>>>>>>>>>>>> card).
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> 4. (minor point) Deprecation of methods
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> You mentioned that the new `commit` 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?
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks again for the KIP! It's really nice
>>>>>>> that
>>>>>>>>> you
>>>>>>>>>>> and
>>>>>>>>>>>>>> Alex
>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> chance to collaborate on both directions so
>>>>>>> that
>>>>>>>>> we
>>>>>>>>>>> can
>>>>>>>>>>>>> get
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> best
>>>>>>>>>>>>>>>>>>>>>>> outcome for Streams and its users.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On 2022/11/21 15:02:15 Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>>>>>>>>>> ,
>>>>>>>>>>>>>>>>>>>>>>>> so that it can be discussed/reviewed
>>>>>>>> separately
>>>>>>>>>>> from
>>>>>>>>>>>>>>> KIP-844.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Alex: I'm especially interested in what
>>>>>> you
>>>>>>>>> think!
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I have a nearly complete implementation
>>>>>> of
>>>>>>> the
>>>>>>>>>>> changes
>>>>>>>>>>>>>>>>>> outlined in
>>>>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>> KIP, please let me know if you'd like me
>>>>>> to
>>>>>>>> push
>>>>>>>>>>> them
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>> review
>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>> advance
>>>>>>>>>>>>>>>>>>>>>>>> of a vote.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 

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

Posted by Nick Telford <ni...@gmail.com>.
Hi everyone,

Quick update: I've added a new section to the KIP: "Offsets for Consumer
Rebalances", that outlines my solution to the problem that
StreamsPartitionAssignor needs to read StateStore offsets even if they're
not currently open.

Regards,
Nick

On Wed, 3 May 2023 at 11:34, Nick Telford <ni...@gmail.com> wrote:

> Hi Bruno,
>
> Thanks for reviewing my proposal.
>
> 1.
> The main reason I added it was because it was easy to do. If we see no
> value in it, I can remove it.
>
> 2.
> Global StateStores can have multiple partitions in their input topics
> (which function as their changelogs), so they would have more than one
> partition.
>
> 3.
> That's a good point. At present, the only method it adds is
> isolationLevel(), which is likely not necessary outside of StateStores.
> It *does* provide slightly different guarantees in the documentation to
> several of the methods (hence the overrides). I'm not sure if this is
> enough to warrant a new interface though.
> I think the question that remains is whether this interface makes it
> easier to implement custom transactional StateStores than if we were to
> remove it? Probably not.
>
> 4.
> The main motivation for the Atomic Checkpointing is actually performance.
> My team has been testing out an implementation of this KIP without it, and
> we had problems with RocksDB doing *much* more compaction, due to the
> significantly increased flush rate. It was enough of a problem that (for
> the time being), we had to revert back to Kafka Streams proper.
> I think the best way to solve this, as you say, is to keep the .checkpoint
> files *in addition* to the offsets being stored within the store itself.
> Essentially, when closing StateStores, we force a memtable flush, then
> call getCommittedOffsets and write those out to the .checkpoint file.
> That would ensure the metadata is available to the
> StreamsPartitionAssignor for all closed stores.
> If there's a crash (no clean close), then we won't be able to guarantee
> which offsets were flushed to disk by RocksDB, so we'd need to open (
> init()), read offsets, and then close() those stores. But since this is
> the exception, and will only occur once (provided it doesn't crash every
> time!), I think the performance impact here would be acceptable.
>
> Thanks for the feedback, please let me know if you have any more comments
> or questions!
>
> I'm currently working on rebasing against trunk. This involves adding
> support for transactionality to VersionedStateStores. I will probably need
> to revise my implementation for transactional "segmented" stores, both to
> accommodate VersionedStateStore, and to clean up some other stuff.
>
> Regards,
> Nick
>
>
> On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org> wrote:
>
>> Hi Nick,
>>
>> Thanks for the updates!
>>
>> I have a couple of questions/comments.
>>
>> 1.
>> Why do you propose a configuration that involves max. bytes and max.
>> reords? I think we are mainly concerned about memory consumption because
>> we want to limit the off-heap memory used. I cannot think of a case
>> where one would want to set the max. number of records.
>>
>>
>> 2.
>> Why does
>>
>>   default void commit(final Map<TopicPartition, Long> changelogOffsets) {
>>       flush();
>>   }
>>
>> take a map of partitions to changelog offsets?
>> The mapping between state stores to partitions is a 1:1 relationship.
>> Passing in a single changelog offset should suffice.
>>
>>
>> 3.
>> Why do we need the Transaction interface? It should be possible to hide
>> beginning and committing a transactions withing the state store
>> implementation, so that from outside the state store, it does not matter
>> whether the state store is transactional or not. What would be the
>> advantage of using the Transaction interface?
>>
>>
>> 4.
>> Regarding checkpointing offsets, I think we should keep the checkpoint
>> file in any case for the reason you mentioned about rebalancing. Even if
>> that would not be an issue, I would propose to move the change to offset
>> management to a new KIP and to not add more complexity than needed to
>> this one. I would not be too concerned about the consistency violation
>> you mention. As far as I understand, with transactional state stores
>> Streams would write the checkpoint file during every commit even under
>> EOS. In the failure case you describe, Streams would restore the state
>> stores from the offsets found in the checkpoint file written during the
>> penultimate commit instead of during the last commit. Basically, Streams
>> would overwrite the records written to the state store between the last
>> two commits with the same records read from the changelogs. While I
>> understand that this is wasteful, it is -- at the same time --
>> acceptable and most importantly it does not break EOS.
>>
>> Best,
>> Bruno
>>
>>
>> On 27.04.23 12:34, Nick Telford wrote:
>> > Hi everyone,
>> >
>> > I find myself (again) considering removing the offset management from
>> > StateStores, and keeping the old checkpoint file system. The reason is
>> that
>> > the StreamPartitionAssignor directly reads checkpoint files in order to
>> > determine which instance has the most up-to-date copy of the local
>> state.
>> > If we move offsets into the StateStore itself, then we will need to
>> open,
>> > initialize, read offsets and then close each StateStore (that is not
>> > already assigned and open) for which we have *any* local state, on every
>> > rebalance.
>> >
>> > Generally, I don't think there are many "orphan" stores like this
>> sitting
>> > around on most instances, but even a few would introduce additional
>> latency
>> > to an already somewhat lengthy rebalance procedure.
>> >
>> > I'm leaning towards Colt's (Slack) suggestion of just keeping things in
>> the
>> > checkpoint file(s) for now, and not worrying about the race. The
>> downside
>> > is that we wouldn't be able to remove the explicit RocksDB flush
>> on-commit,
>> > which likely hurts performance.
>> >
>> > If anyone has any thoughts or ideas on this subject, I would appreciate
>> it!
>> >
>> > Regards,
>> > Nick
>> >
>> > On Wed, 19 Apr 2023 at 15:05, Nick Telford <ni...@gmail.com>
>> wrote:
>> >
>> >> Hi Colt,
>> >>
>> >> The issue is that if there's a crash between 2 and 3, then you still
>> end
>> >> up with inconsistent data in RocksDB. The only way to guarantee that
>> your
>> >> checkpoint offsets and locally stored data are consistent with each
>> other
>> >> are to atomically commit them, which can be achieved by having the
>> offsets
>> >> stored in RocksDB.
>> >>
>> >> The offsets column family is likely to be extremely small (one
>> >> per-changelog partition + one per Topology input partition for regular
>> >> stores, one per input partition for global stores). So the overhead
>> will be
>> >> minimal.
>> >>
>> >> A major benefit of doing this is that we can remove the explicit calls
>> to
>> >> db.flush(), which forcibly flushes memtables to disk on-commit. It
>> turns
>> >> out, RocksDB memtable flushes are largely dictated by Kafka Streams
>> >> commits, *not* RocksDB configuration, which could be a major source of
>> >> confusion. Atomic checkpointing makes it safe to remove these explicit
>> >> flushes, because it no longer matters exactly when RocksDB flushes
>> data to
>> >> disk; since the data and corresponding checkpoint offsets will always
>> be
>> >> flushed together, the local store is always in a consistent state, and
>> >> on-restart, it can always safely resume restoration from the on-disk
>> >> offsets, restoring the small amount of data that hadn't been flushed
>> when
>> >> the app exited/crashed.
>> >>
>> >> Regards,
>> >> Nick
>> >>
>> >> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io>
>> wrote:
>> >>
>> >>> Nick,
>> >>>
>> >>> Thanks for your reply. Ack to A) and B).
>> >>>
>> >>> For item C), I see what you're referring to. Your proposed solution
>> will
>> >>> work, so no need to change it. What I was suggesting was that it
>> might be
>> >>> possible to achieve this with only one column family. So long as:
>> >>>
>> >>>     - No uncommitted records (i.e. not committed to the changelog) are
>> >>>     *committed* to the state store, AND
>> >>>     - The Checkpoint offset (which refers to the changelog topic) is
>> less
>> >>>     than or equal to the last written changelog offset in rocksdb
>> >>>
>> >>> I don't see the need to do the full restoration from scratch. My
>> >>> understanding was that prior to 844/892, full restorations were
>> required
>> >>> because there could be uncommitted records written to RocksDB;
>> however,
>> >>> given your use of RocksDB transactions, that can be avoided with the
>> >>> pattern of 1) commit Kafka transaction, 2) commit RocksDB
>> transaction, 3)
>> >>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <ni...@gmail.com>
>> >>> wrote:
>> >>>
>> >>>> Hi Colt,
>> >>>>
>> >>>> A. I've done my best to de-couple the StateStore stuff from the rest
>> of
>> >>> the
>> >>>> Streams engine. The fact that there will be only one ongoing (write)
>> >>>> transaction at a time is not guaranteed by any API, and is just a
>> >>>> consequence of the way Streams operates. To that end, I tried to
>> ensure
>> >>> the
>> >>>> documentation and guarantees provided by the new APIs are
>> independent of
>> >>>> this incidental behaviour. In practice, you're right, this
>> essentially
>> >>>> refers to "interactive queries", which are technically "read
>> >>> transactions",
>> >>>> even if they don't actually use the transaction API to isolate
>> >>> themselves.
>> >>>>
>> >>>> B. Yes, although not ideal. This is for backwards compatibility,
>> >>> because:
>> >>>>      1) Existing custom StateStore implementations will implement
>> >>> flush(),
>> >>>> and not commit(), but the Streams engine now calls commit(), so those
>> >>> calls
>> >>>> need to be forwarded to flush() for these legacy stores.
>> >>>>      2) Existing StateStore *users*, i.e. outside of the Streams
>> engine
>> >>>> itself, may depend on explicitly calling flush(), so for these cases,
>> >>>> flush() needs to be redirected to call commit().
>> >>>> If anyone has a better way to guarantee compatibility without
>> >>> introducing
>> >>>> this potential recursion loop, I'm open to changes!
>> >>>>
>> >>>> C. This is described in the "Atomic Checkpointing" section. Offsets
>> are
>> >>>> stored in a separate RocksDB column family, which is guaranteed to be
>> >>>> atomically flushed to disk with all other column families. The issue
>> of
>> >>>> checkpoints being written to disk after commit causing inconsistency
>> if
>> >>> it
>> >>>> crashes in between is the reason why, under EOS, checkpoint files are
>> >>> only
>> >>>> written on clean shutdown. This is one of the major causes of "full
>> >>>> restorations", so moving the offsets into a place where they can be
>> >>>> guaranteed to be atomically written with the data they checkpoint
>> >>> allows us
>> >>>> to write the checkpoint offsets *on every commit*, not just on clean
>> >>>> shutdown.
>> >>>>
>> >>>> Regards,
>> >>>> Nick
>> >>>>
>> >>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io>
>> wrote:
>> >>>>
>> >>>>> Nick,
>> >>>>>
>> >>>>> Thank you for continuing this work. I have a few minor clarifying
>> >>>>> questions.
>> >>>>>
>> >>>>> A) "Records written to any transaction are visible to all other
>> >>>>> transactions immediately." I am confused here—I thought there could
>> >>> only
>> >>>> be
>> >>>>> one transaction going on at a time for a given state store given the
>> >>>>> threading model for processing records on a Task. Do you mean
>> >>> Interactive
>> >>>>> Queries by "other transactions"? (If so, then everything makes
>> sense—I
>> >>>>> thought that since IQ were read-only then they didn't count as
>> >>>>> transactions).
>> >>>>>
>> >>>>> B) Is it intentional that the default implementations of the flush()
>> >>> and
>> >>>>> commit() methods in the StateStore class refer to each other in some
>> >>> sort
>> >>>>> of unbounded recursion?
>> >>>>>
>> >>>>> C) How will the getCommittedOffset() method work? At first I thought
>> >>> the
>> >>>>> way to do it would be using a special key in the RocksDB store to
>> >>> store
>> >>>> the
>> >>>>> offset, and committing that with the transaction. But upon second
>> >>>> thought,
>> >>>>> since restoration from the changelog is an idempotent procedure, I
>> >>> think
>> >>>> it
>> >>>>> would be fine to 1) commit the RocksDB transaction and then 2) write
>> >>> the
>> >>>>> offset to disk in a checkpoint file. If there is a crash between 1)
>> >>> and
>> >>>> 2),
>> >>>>> I think the only downside is 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 <
>> nick.telford@gmail.com>
>> >>>>> wrote:
>> >>>>>
>> >>>>>> Hi everyone,
>> >>>>>>
>> >>>>>> I've updated the KIP to reflect the latest version of the design:
>> >>>>>>
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> >>>>>>
>> >>>>>> There are several changes in there that reflect feedback from this
>> >>>>> thread,
>> >>>>>> and there's a new section and a bunch of interface changes relating
>> >>> to
>> >>>>>> Atomic Checkpointing, which is the final piece of the puzzle to
>> >>> making
>> >>>>>> everything robust.
>> >>>>>>
>> >>>>>> Let me know what you think!
>> >>>>>>
>> >>>>>> Regards,
>> >>>>>> Nick
>> >>>>>>
>> >>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <ni...@gmail.com>
>> >>>>> wrote:
>> >>>>>>
>> >>>>>>> Hi Lucas,
>> >>>>>>>
>> >>>>>>> Thanks for looking over my KIP.
>> >>>>>>>
>> >>>>>>> A) The bound is per-instance, not per-Task. This was a typo in the
>> >>>> KIP
>> >>>>>>> that I've now corrected. It was originally per-Task, but I
>> >>> changed it
>> >>>>> to
>> >>>>>>> per-instance for exactly the reason you highlighted.
>> >>>>>>> B) It's worth noting that transactionality is only enabled under
>> >>> EOS,
>> >>>>> and
>> >>>>>>> in the default mode of operation (ALOS), there should be no
>> >>> change in
>> >>>>>>> behavior at all. I think, under EOS, we can mitigate the impact on
>> >>>>> users
>> >>>>>> by
>> >>>>>>> sufficiently low default values for the memory bound
>> >>> configuration. I
>> >>>>>>> understand your hesitation to include a significant change of
>> >>>>> behaviour,
>> >>>>>>> especially in a minor release, but I suspect that most users will
>> >>>>> prefer
>> >>>>>>> the memory impact (under EOS) to the existing behaviour of
>> >>> frequent
>> >>>>> state
>> >>>>>>> restorations! If this is a problem, the changes can wait until the
>> >>>> next
>> >>>>>>> major release. I'll be running a patched version of streams in
>> >>>>> production
>> >>>>>>> with these changes as soon as they're ready, so it won't disrupt
>> >>> me
>> >>>> :-D
>> >>>>>>> C) The main purpose of this sentence was just to note that some
>> >>>> changes
>> >>>>>>> will need to be made to the way Segments are handled in order to
>> >>>> ensure
>> >>>>>>> they also benefit from transactions. At the time I wrote it, I
>> >>> hadn't
>> >>>>>>> figured out the specific changes necessary, so it was deliberately
>> >>>>> vague.
>> >>>>>>> This is the one outstanding problem I'm currently working on, and
>> >>>> I'll
>> >>>>>>> update this section with more detail once I have figured out the
>> >>>> exact
>> >>>>>>> changes required.
>> >>>>>>> D) newTransaction() provides the necessary isolation guarantees.
>> >>>> While
>> >>>>>>> the RocksDB implementation of transactions doesn't technically
>> >>> *need*
>> >>>>>>> read-only users to call newTransaction(), other implementations
>> >>>> (e.g. a
>> >>>>>>> hypothetical PostgresStore) may require it. Calling
>> >>> newTransaction()
>> >>>>> when
>> >>>>>>> no transaction is necessary is essentially free, as it will just
>> >>>> return
>> >>>>>>> this.
>> >>>>>>>
>> >>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think it
>> >>> should
>> >>>> be
>> >>>>>>> fairly obvious where the performance problems stem from: writes
>> >>> under
>> >>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with the
>> >>>>>>> tombstone/record flag, 1 to decode it from the tombstone/record
>> >>> flag,
>> >>>>>> and 1
>> >>>>>>> to copy the record from the "temporary" store to the "main" store,
>> >>>> when
>> >>>>>> the
>> >>>>>>> transaction commits. The different approach taken by KIP-869
>> >>> should
>> >>>>>> perform
>> >>>>>>> much better, as it avoids all these copies, and may actually
>> >>> perform
>> >>>>>>> slightly better than trunk, due to batched writes in RocksDB
>> >>>> performing
>> >>>>>>> better than non-batched writes.[1]
>> >>>>>>>
>> >>>>>>> Regards,
>> >>>>>>> Nick
>> >>>>>>>
>> >>>>>>> 1:
>> >>>>>>
>> >>>>
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>> >>>>>>>
>> >>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
>> >>> lbrutschy@confluent.io
>> >>>>>> .invalid>
>> >>>>>>> wrote:
>> >>>>>>>
>> >>>>>>>> Hi Nick,
>> >>>>>>>>
>> >>>>>>>> I'm just starting to read up on the whole discussion about
>> >>> KIP-892
>> >>>> and
>> >>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
>> >>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do have some
>> >>>>>>>> questions about the latest draft.
>> >>>>>>>>
>> >>>>>>>>   A) If I understand correctly, you propose to put a bound on the
>> >>>>>>>> (native) memory consumed by each task. However, I wonder if this
>> >>> is
>> >>>>>>>> sufficient if we have temporary imbalances in the cluster. For
>> >>>>>>>> example, depending on the timing of rebalances during a cluster
>> >>>>>>>> restart, it could happen that a single streams node is assigned a
>> >>>> lot
>> >>>>>>>> more tasks than expected. With your proposed change, this would
>> >>> mean
>> >>>>>>>> that the memory required by this one node could be a multiple of
>> >>>> what
>> >>>>>>>> is required during normal operation. I wonder if it wouldn't be
>> >>>> safer
>> >>>>>>>> to put a global bound on the memory use, across all tasks.
>> >>>>>>>>   B) Generally, the memory concerns still give me the feeling
>> that
>> >>>> this
>> >>>>>>>> should not be enabled by default for all users in a minor
>> >>> release.
>> >>>>>>>>   C) In section "Transaction Management": the sentence "A similar
>> >>>>>>>> analogue will be created to automatically manage `Segment`
>> >>>>>>>> transactions.". Maybe this is just me lacking some background,
>> >>> 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 you to find out why the KIP-844 PoC is 20x slower
>> >>> (e.g.
>> >>>>>>>> by providing a flame graph), that would be quite interesting.
>> >>>>>>>>
>> >>>>>>>> Cheers,
>> >>>>>>>> Lucas
>> >>>>>>>>
>> >>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
>> >>>> nick.telford@gmail.com>
>> >>>>>>>> wrote:
>> >>>>>>>>>
>> >>>>>>>>> Hi everyone,
>> >>>>>>>>>
>> >>>>>>>>> I've updated the KIP with a more detailed design, which
>> >>> reflects
>> >>>> the
>> >>>>>>>>> implementation I've been working on:
>> >>>>>>>>>
>> >>>>>>>>
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> >>>>>>>>>
>> >>>>>>>>> This new design should address the outstanding points already
>> >>> made
>> >>>>> in
>> >>>>>>>> the
>> >>>>>>>>> thread.
>> >>>>>>>>>
>> >>>>>>>>> Please let me know if there are areas that are unclear or need
>> >>>> more
>> >>>>>>>>> clarification.
>> >>>>>>>>>
>> >>>>>>>>> I have a (nearly) working implementation. I'm confident that
>> >>> the
>> >>>>>>>> remaining
>> >>>>>>>>> work (making Segments behave) will not impact the documented
>> >>>> design.
>> >>>>>>>>>
>> >>>>>>>>> Regards,
>> >>>>>>>>>
>> >>>>>>>>> Nick
>> >>>>>>>>>
>> >>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <colt@littlehorse.io
>> >>>>
>> >>>>>> wrote:
>> >>>>>>>>>
>> >>>>>>>>>> Nick,
>> >>>>>>>>>>
>> >>>>>>>>>> Thank you for the reply; that makes sense. I was hoping that,
>> >>>>> since
>> >>>>>>>> reading
>> >>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
>> >>> documented
>> >>>>> API,
>> >>>>>>>> maybe
>> >>>>>>>>>> you *wouldn't* have to wait for the next major release to
>> >>> make
>> >>>>> that
>> >>>>>>>> change;
>> >>>>>>>>>> but given 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 <
>> >>>>>> nick.telford@gmail.com>
>> >>>>>>>>>> wrote:
>> >>>>>>>>>>
>> >>>>>>>>>>> Hi Colt,
>> >>>>>>>>>>>
>> >>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended to
>> >>> try
>> >>>> to
>> >>>>>>>> keep the
>> >>>>>>>>>>> behaviour unchanged as much as possible, otherwise we'd
>> >>> have
>> >>>> to
>> >>>>>>>> wait for
>> >>>>>>>>>> a
>> >>>>>>>>>>> major version release to land these changes.
>> >>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
>> >>> guarantee,
>> >>>>> and
>> >>>>>>>> the
>> >>>>>>>>>>> typically longer commit intervals would be problematic when
>> >>>>>> reading
>> >>>>>>>> only
>> >>>>>>>>>>> "committed" records.
>> >>>>>>>>>>>
>> >>>>>>>>>>> I've been away for 5 days recovering from minor surgery,
>> >>> but I
>> >>>>>>>> spent a
>> >>>>>>>>>>> considerable amount of that time working through ideas for
>> >>>>>> possible
>> >>>>>>>>>>> solutions in my head. I think your suggestion of keeping
>> >>> ALOS
>> >>>>>>>> as-is, but
>> >>>>>>>>>>> buffering writes for EOS is the right path forwards,
>> >>> although
>> >>>> I
>> >>>>>>>> have a
>> >>>>>>>>>>> solution that both expands on this, and provides for some
>> >>> more
>> >>>>>>>> formal
>> >>>>>>>>>>> guarantees.
>> >>>>>>>>>>>
>> >>>>>>>>>>> Essentially, adding support to KeyValueStores for
>> >>>>> "Transactions",
>> >>>>>>>> with
>> >>>>>>>>>>> clearly defined IsolationLevels. Using "Read Committed"
>> >>> when
>> >>>>> under
>> >>>>>>>> EOS,
>> >>>>>>>>>> and
>> >>>>>>>>>>> "Read Uncommitted" under ALOS.
>> >>>>>>>>>>>
>> >>>>>>>>>>> The nice thing about this approach is that it gives us much
>> >>>> more
>> >>>>>>>> clearly
>> >>>>>>>>>>> defined isolation behaviour that can be properly
>> >>> documented to
>> >>>>>>>> ensure
>> >>>>>>>>>> users
>> >>>>>>>>>>> know what to expect.
>> >>>>>>>>>>>
>> >>>>>>>>>>> I'm still working out the kinks in the design, and will
>> >>> update
>> >>>>> the
>> >>>>>>>> KIP
>> >>>>>>>>>> when
>> >>>>>>>>>>> I have something. The main struggle is trying to implement
>> >>>> this
>> >>>>>>>> without
>> >>>>>>>>>>> making any major changes to the existing interfaces or
>> >>>> breaking
>> >>>>>>>> existing
>> >>>>>>>>>>> implementations, because currently everything expects to
>> >>>> operate
>> >>>>>>>> directly
>> >>>>>>>>>>> on a StateStore, and not a Transaction of that store. I
>> >>> think
>> >>>>> I'm
>> >>>>>>>> getting
>> >>>>>>>>>>> close, although sadly I won't be able to progress much
>> >>> until
>> >>>>> next
>> >>>>>>>> week
>> >>>>>>>>>> due
>> >>>>>>>>>>> to some work commitments.
>> >>>>>>>>>>>
>> >>>>>>>>>>> Regards,
>> >>>>>>>>>>> Nick
>> >>>>>>>>>>>
>> >>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
>> >>>> colt@littlehorse.io>
>> >>>>>>>> wrote:
>> >>>>>>>>>>>
>> >>>>>>>>>>>> Nick,
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Thank you for the explanation, and also for the updated
>> >>>> KIP. I
>> >>>>>> am
>> >>>>>>>> quite
>> >>>>>>>>>>>> eager for this improvement to be released as it would
>> >>>> greatly
>> >>>>>>>> reduce
>> >>>>>>>>>> the
>> >>>>>>>>>>>> operational difficulties of EOS streams apps.
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> Two questions:
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> 10)
>> >>>>>>>>>>>>> When reading records, we will use the
>> >>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
>> >>>>>>>>>>>>   and WriteBatchWithIndex#newIteratorWithBase utilities in
>> >>>>> order
>> >>>>>> to
>> >>>>>>>>>> ensure
>> >>>>>>>>>>>> that uncommitted writes are available to query.
>> >>>>>>>>>>>> Why do extra work to enable the reading of uncommitted
>> >>>> writes
>> >>>>>>>> during
>> >>>>>>>>>> IQ?
>> >>>>>>>>>>>> Code complexity aside, reading uncommitted writes is, in
>> >>> my
>> >>>>>>>> opinion, a
>> >>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have the
>> >>>>>> guarantee
>> >>>>>>>> that,
>> >>>>>>>>>>>> with EOS, IQ only reads committed records. In order to
>> >>> avoid
>> >>>>>> dirty
>> >>>>>>>>>> reads,
>> >>>>>>>>>>>> one currently must query a standby replica (but this
>> >>> still
>> >>>>>> doesn't
>> >>>>>>>>>> fully
>> >>>>>>>>>>>> guarantee monotonic reads).
>> >>>>>>>>>>>>
>> >>>>>>>>>>>> 20) Is it also necessary to enable this optimization on
>> >>> ALOS
>> >>>>>>>> stores?
>> >>>>>>>>>> The
>> >>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need to
>> >>>> restore
>> >>>>>>>> state
>> >>>>>>>>>> from
>> >>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
>> >>>> acceptable
>> >>>>> to
>> >>>>>>>> accept
>> >>>>>>>>>>>> that there may have been uncommitted writes on disk. On a
>> >>>> side
>> >>>>>>>> note, if
>> >>>>>>>>>>> you
>> >>>>>>>>>>>> enable this type of store on ALOS processors, the
>> >>> community
>> >>>>>> would
>> >>>>>>>>>>>> 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 <
>> >>>>>>>> nick.telford@gmail.com>
>> >>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>
>> >>>>>>>>>>>>> Hi everyone,
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
>> >>>> longer
>> >>>>>>>> include
>> >>>>>>>>>> the
>> >>>>>>>>>>>>> StateStore management of checkpointing. This can be
>> >>> added
>> >>>>> as a
>> >>>>>>>> KIP
>> >>>>>>>>>>> later
>> >>>>>>>>>>>> on
>> >>>>>>>>>>>>> to further optimize the consistency and performance of
>> >>>> state
>> >>>>>>>> stores.
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>> I've also added a section discussing some of the
>> >>> concerns
>> >>>>>> 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 2022 at 16:36, Nick Telford <
>> >>>>>>>> nick.telford@gmail.com>
>> >>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>>> Hi Colt,
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> I didn't do any profiling, but the 844
>> >>> implementation:
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>     - Writes uncommitted records to a temporary
>> >>> RocksDB
>> >>>>>>>> instance
>> >>>>>>>>>>>>>>        - Since tombstones need to be flagged, all
>> >>> record
>> >>>>>>>> values are
>> >>>>>>>>>>>>>>        prefixed with a value/tombstone marker. This
>> >>>>>>>> necessitates a
>> >>>>>>>>>>>> memory
>> >>>>>>>>>>>>> copy.
>> >>>>>>>>>>>>>>     - On-commit, iterates all records in this
>> >>> temporary
>> >>>>>>>> instance and
>> >>>>>>>>>>>>>>     writes them to the main RocksDB store.
>> >>>>>>>>>>>>>>     - While iterating, the value/tombstone marker
>> >>> needs
>> >>>> to
>> >>>>> be
>> >>>>>>>> parsed
>> >>>>>>>>>>> and
>> >>>>>>>>>>>>>>     the real value extracted. This necessitates
>> >>> another
>> >>>>>> memory
>> >>>>>>>> copy.
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> My guess is that the cost of iterating the temporary
>> >>>>> RocksDB
>> >>>>>>>> store
>> >>>>>>>>>> is
>> >>>>>>>>>>>> the
>> >>>>>>>>>>>>>> major factor, with the 2 extra memory copies
>> >>> per-Record
>> >>>>>>>>>> contributing
>> >>>>>>>>>>> a
>> >>>>>>>>>>>>>> significant amount too.
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> Regards,
>> >>>>>>>>>>>>>> Nick
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>> On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
>> >>>>>>>> colt@littlehorse.io>
>> >>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> Hi all,
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> Out of curiosity, why does the performance of the
>> >>> store
>> >>>>>>>> degrade so
>> >>>>>>>>>>>>>>> significantly with the 844 implementation? I
>> >>> wouldn't
>> >>>> be
>> >>>>>> too
>> >>>>>>>>>>> surprised
>> >>>>>>>>>>>>> by
>> >>>>>>>>>>>>>>> a
>> >>>>>>>>>>>>>>> 50-60% drop (caused by each record being written
>> >>>> twice),
>> >>>>>> but
>> >>>>>>>> 96%
>> >>>>>>>>>> is
>> >>>>>>>>>>>>>>> extreme.
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>> The only thing I can think of which could create
>> >>> such a
>> >>>>>>>> bottleneck
>> >>>>>>>>>>>> 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 Nick Telford <
>> >>>>>>>>>>> nick.telford@gmail.com>
>> >>>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> Hi everyone,
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> I've updated the KIP to resolve all the points
>> >>> that
>> >>>>> have
>> >>>>>>>> been
>> >>>>>>>>>>> raised
>> >>>>>>>>>>>>> so
>> >>>>>>>>>>>>>>>> far, with one exception: the ALOS default commit
>> >>>>> interval
>> >>>>>>>> of 5
>> >>>>>>>>>>>> minutes
>> >>>>>>>>>>>>>>> is
>> >>>>>>>>>>>>>>>> likely to cause WriteBatchWithIndex memory to grow
>> >>>> too
>> >>>>>>>> large.
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>> There's a couple of different things I can think
>> >>> of
>> >>>> to
>> >>>>>>>> solve
>> >>>>>>>>>> this:
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>     - We already have a memory/record limit in the
>> >>> KIP
>> >>>>> to
>> >>>>>>>> prevent
>> >>>>>>>>>>> OOM
>> >>>>>>>>>>>>>>>>     errors. Should we choose a default value for
>> >>>> these?
>> >>>>> My
>> >>>>>>>>>> concern
>> >>>>>>>>>>>> here
>> >>>>>>>>>>>>>>> is
>> >>>>>>>>>>>>>>>> that
>> >>>>>>>>>>>>>>>>     anything we choose might seem rather
>> >>> arbitrary. We
>> >>>>>> could
>> >>>>>>>>>> change
>> >>>>>>>>>>>>>>>>     its behaviour such that under ALOS, it only
>> >>>> triggers
>> >>>>>> the
>> >>>>>>>>>> commit
>> >>>>>>>>>>>> of
>> >>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>     StateStore, but under EOS, it triggers a
>> >>> commit of
>> >>>>> the
>> >>>>>>>> Kafka
>> >>>>>>>>>>>>>>>> transaction.
>> >>>>>>>>>>>>>>>>     - We could introduce a separate `
>> >>>>>> checkpoint.interval.ms`
>> >>>>>>>> to
>> >>>>>>>>>>>> allow
>> >>>>>>>>>>>>>>> ALOS
>> >>>>>>>>>>>>>>>>     to commit the StateStores more frequently than
>> >>> the
>> >>>>>>>> general
>> >>>>>>>>>>>>>>>>     commit.interval.ms? My concern here is that
>> >>> the
>> >>>>>>>> semantics of
>> >>>>>>>>>>>> this
>> >>>>>>>>>>>>>>>> config
>> >>>>>>>>>>>>>>>>     would depend 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 <
>> >>>>>>>>>>> nick.telford@gmail.com>
>> >>>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Hi Alex,
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Thanks for the feedback.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> I've updated the discussion of OOM issues by
>> >>>>> describing
>> >>>>>>>> how
>> >>>>>>>>>>> we'll
>> >>>>>>>>>>>>>>> handle
>> >>>>>>>>>>>>>>>>> it. Here's the new text:
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> To mitigate this, we will automatically force a
>> >>>> Task
>> >>>>>>>> commit if
>> >>>>>>>>>>> the
>> >>>>>>>>>>>>>>> total
>> >>>>>>>>>>>>>>>>>> uncommitted records returned by
>> >>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedEntries()
>> >>>>>> exceeds a
>> >>>>>>>>>>>> threshold,
>> >>>>>>>>>>>>>>>>>> configured by
>> >>>>> max.uncommitted.state.entries.per.task;
>> >>>>>>>> or the
>> >>>>>>>>>>>> total
>> >>>>>>>>>>>>>>>>>> memory used for buffering uncommitted records
>> >>>>> returned
>> >>>>>>>> by
>> >>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
>> >>>> exceeds
>> >>>>>> the
>> >>>>>>>>>>> threshold
>> >>>>>>>>>>>>>>>>>> configured by
>> >>>> max.uncommitted.state.bytes.per.task.
>> >>>>>>>> This will
>> >>>>>>>>>>>>> roughly
>> >>>>>>>>>>>>>>>>>> bound the memory required per-Task for
>> >>> buffering
>> >>>>>>>> uncommitted
>> >>>>>>>>>>>>> records,
>> >>>>>>>>>>>>>>>>>> irrespective of the commit.interval.ms, and
>> >>> will
>> >>>>>>>> effectively
>> >>>>>>>>>>>> bound
>> >>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>> number of records that will need to be
>> >>> restored in
>> >>>>> the
>> >>>>>>>> event
>> >>>>>>>>>>> of a
>> >>>>>>>>>>>>>>>> failure.
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> These limits will be checked in
>> >>> StreamTask#process
>> >>>>> and
>> >>>>>> a
>> >>>>>>>>>>> premature
>> >>>>>>>>>>>>>>> commit
>> >>>>>>>>>>>>>>>>>> will be requested via Task#requestCommit().
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Note that these new methods provide default
>> >>>>>>>> implementations
>> >>>>>>>>>> that
>> >>>>>>>>>>>>>>> ensure
>> >>>>>>>>>>>>>>>>>> existing custom stores and non-transactional
>> >>>> stores
>> >>>>>>>> (e.g.
>> >>>>>>>>>>>>>>>>>> InMemoryKeyValueStore) do not force any early
>> >>>>> commits.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> I've chosen to have the StateStore expose
>> >>>>>> approximations
>> >>>>>>>> of
>> >>>>>>>>>> its
>> >>>>>>>>>>>>> buffer
>> >>>>>>>>>>>>>>>>> size/count instead of opaquely requesting a
>> >>> commit
>> >>>> in
>> >>>>>>>> order to
>> >>>>>>>>>>>>>>> delegate
>> >>>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>> decision making to the Task itself. This enables
>> >>>>> Tasks
>> >>>>>>>> to look
>> >>>>>>>>>>> at
>> >>>>>>>>>>>>>>> *all*
>> >>>>>>>>>>>>>>>> of
>> >>>>>>>>>>>>>>>>> their StateStores, and determine whether an
>> >>> early
>> >>>>>> commit
>> >>>>>>>> is
>> >>>>>>>>>>>>> necessary.
>> >>>>>>>>>>>>>>>>> Notably, it enables pre-Task thresholds,
>> >>> instead of
>> >>>>>>>> per-Store,
>> >>>>>>>>>>>> which
>> >>>>>>>>>>>>>>>>> prevents Tasks with many StateStores from using
>> >>>> much
>> >>>>>> more
>> >>>>>>>>>> memory
>> >>>>>>>>>>>>> than
>> >>>>>>>>>>>>>>>> Tasks
>> >>>>>>>>>>>>>>>>> with one StateStore. This makes sense, since
>> >>>> commits
>> >>>>>> are
>> >>>>>>>> done
>> >>>>>>>>>>>>> by-Task,
>> >>>>>>>>>>>>>>>> not
>> >>>>>>>>>>>>>>>>> by-Store.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Prizes* for anyone who can come up with a better
>> >>>> name
>> >>>>>>>> for the
>> >>>>>>>>>>> new
>> >>>>>>>>>>>>>>> config
>> >>>>>>>>>>>>>>>>> properties!
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> Thanks for pointing out the potential
>> >>> performance
>> >>>>>> issues
>> >>>>>>>> of
>> >>>>>>>>>>> WBWI.
>> >>>>>>>>>>>>> From
>> >>>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>> benchmarks that user posted[1], it looks like
>> >>> WBWI
>> >>>>>> still
>> >>>>>>>>>>> performs
>> >>>>>>>>>>>>>>>>> considerably better than individual puts, which
>> >>> is
>> >>>>> the
>> >>>>>>>>>> existing
>> >>>>>>>>>>>>>>> design,
>> >>>>>>>>>>>>>>>> so
>> >>>>>>>>>>>>>>>>> I'd actually expect a performance boost from
>> >>> WBWI,
>> >>>>> just
>> >>>>>>>> not as
>> >>>>>>>>>>>> great
>> >>>>>>>>>>>>>>> as
>> >>>>>>>>>>>>>>>>> we'd get from a plain WriteBatch. This does
>> >>> suggest
>> >>>>>> that
>> >>>>>>>> a
>> >>>>>>>>>> good
>> >>>>>>>>>>>>>>>>> optimization would be to use a regular
>> >>> WriteBatch
>> >>>> for
>> >>>>>>>>>>> restoration
>> >>>>>>>>>>>>> (in
>> >>>>>>>>>>>>>>>>> RocksDBStore#restoreBatch), since we know that
>> >>>> those
>> >>>>>>>> records
>> >>>>>>>>>>> will
>> >>>>>>>>>>>>>>> never
>> >>>>>>>>>>>>>>>> be
>> >>>>>>>>>>>>>>>>> queried before they're committed.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> 1:
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>
>> >>>>>
>> >>>
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> * Just kidding, no prizes, sadly.
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>> On Wed, 23 Nov 2022 at 12:28, Alexander
>> >>> Sorokoumov
>> >>>>>>>>>>>>>>>>> <as...@confluent.io.invalid> wrote:
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> 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 to keep the current
>> >>>>> checkpointing
>> >>>>>>>> as a
>> >>>>>>>>>>>>> fallback
>> >>>>>>>>>>>>>>>>>> mechanism. We want to keep existing users'
>> >>>> workflows
>> >>>>>>>> intact
>> >>>>>>>>>> if
>> >>>>>>>>>>> we
>> >>>>>>>>>>>>>>> can. A
>> >>>>>>>>>>>>>>>>>> non-intrusive way would be to add a separate
>> >>>>>> StateStore
>> >>>>>>>>>> method,
>> >>>>>>>>>>>>> say,
>> >>>>>>>>>>>>>>>>>> StateStore#managesCheckpointing(), that
>> >>> controls
>> >>>>>>>> whether the
>> >>>>>>>>>>>> state
>> >>>>>>>>>>>>>>> store
>> >>>>>>>>>>>>>>>>>> implementation owns checkpointing.
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> I think that a solution to the transactional
>> >>>> writes
>> >>>>>>>> should
>> >>>>>>>>>>>> address
>> >>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>> OOMEs. One possible way to address that is to
>> >>> wire
>> >>>>>>>>>> StateStore's
>> >>>>>>>>>>>>>>> commit
>> >>>>>>>>>>>>>>>>>> request by adding, say, StateStore#commitNeeded
>> >>>> that
>> >>>>>> is
>> >>>>>>>>>> checked
>> >>>>>>>>>>>> in
>> >>>>>>>>>>>>>>>>>> StreamTask#commitNeeded via the corresponding
>> >>>>>>>>>>>>> ProcessorStateManager.
>> >>>>>>>>>>>>>>>> With
>> >>>>>>>>>>>>>>>>>> that change, RocksDBStore will have to track
>> >>> the
>> >>>>>> current
>> >>>>>>>>>>>>> transaction
>> >>>>>>>>>>>>>>>> size
>> >>>>>>>>>>>>>>>>>> and request a commit when the size goes over a
>> >>>>>>>> (configurable)
>> >>>>>>>>>>>>>>> threshold.
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> AFAIU WriteBatchWithIndex might perform
>> >>>>> significantly
>> >>>>>>>> slower
>> >>>>>>>>>>> than
>> >>>>>>>>>>>>>>>> non-txn
>> >>>>>>>>>>>>>>>>>> puts as the batch size grows [1]. We should
>> >>> have a
>> >>>>>>>>>>> configuration
>> >>>>>>>>>>>> to
>> >>>>>>>>>>>>>>> fall
>> >>>>>>>>>>>>>>>>>> back to the current behavior (and/or disable
>> >>> txn
>> >>>>>> stores
>> >>>>>>>> for
>> >>>>>>>>>>> ALOS)
>> >>>>>>>>>>>>>>> unless
>> >>>>>>>>>>>>>>>>>> the benchmarks show negligible overhead for
>> >>> longer
>> >>>>>>>> commits /
>> >>>>>>>>>>>>>>>> large-enough
>> >>>>>>>>>>>>>>>>>> batch sizes.
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> If you prefer to keep the KIP smaller, I would
>> >>>>> rather
>> >>>>>>>> cut out
>> >>>>>>>>>>>>>>>>>> state-store-managed checkpointing rather than
>> >>>> proper
>> >>>>>>>> OOMe
>> >>>>>>>>>>>> handling
>> >>>>>>>>>>>>>>> and
>> >>>>>>>>>>>>>>>>>> being able to switch to non-txn behavior. The
>> >>>>>>>> checkpointing
>> >>>>>>>>>> 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 configuration parts as well as review
>> >>> and
>> >>>>>> test
>> >>>>>>>> the
>> >>>>>>>>>>>> patch.
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> Best,
>> >>>>>>>>>>>>>>>>>> Alex
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> 1.
>> >>> https://github.com/facebook/rocksdb/issues/608
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
>> >>>>>>>>>>>>> nick.telford@gmail.com
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Hi John,
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Thanks for the review and feedback!
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> 1. Custom Stores: I've been mulling over this
>> >>>>>> problem
>> >>>>>>>>>> myself.
>> >>>>>>>>>>>> As
>> >>>>>>>>>>>>> it
>> >>>>>>>>>>>>>>>>>> stands,
>> >>>>>>>>>>>>>>>>>>> custom stores would essentially lose
>> >>>> checkpointing
>> >>>>>>>> with no
>> >>>>>>>>>>>>>>> indication
>> >>>>>>>>>>>>>>>>>> that
>> >>>>>>>>>>>>>>>>>>> they're expected to make changes, besides a
>> >>> line
>> >>>>> in
>> >>>>>>>> the
>> >>>>>>>>>>> release
>> >>>>>>>>>>>>>>>> notes. I
>> >>>>>>>>>>>>>>>>>>> agree that the best solution would be to
>> >>>> provide a
>> >>>>>>>> default
>> >>>>>>>>>>> that
>> >>>>>>>>>>>>>>>>>> checkpoints
>> >>>>>>>>>>>>>>>>>>> to a file. The one thing I would change is
>> >>> that
>> >>>>> the
>> >>>>>>>>>>>> checkpointing
>> >>>>>>>>>>>>>>> is
>> >>>>>>>>>>>>>>>> to
>> >>>>>>>>>>>>>>>>>> a
>> >>>>>>>>>>>>>>>>>>> store-local file, instead of a per-Task file.
>> >>>> This
>> >>>>>>>> way the
>> >>>>>>>>>>>>>>> StateStore
>> >>>>>>>>>>>>>>>>>> still
>> >>>>>>>>>>>>>>>>>>> technically owns its own checkpointing (via a
>> >>>>>> default
>> >>>>>>>>>>>>>>> implementation),
>> >>>>>>>>>>>>>>>>>> and
>> >>>>>>>>>>>>>>>>>>> the StateManager/Task execution engine
>> >>> doesn't
>> >>>>> need
>> >>>>>>>> to know
>> >>>>>>>>>>>>>>> anything
>> >>>>>>>>>>>>>>>>>> about
>> >>>>>>>>>>>>>>>>>>> checkpointing, which greatly simplifies some
>> >>> of
>> >>>>> the
>> >>>>>>>> logic.
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> 2. OOME errors: The main reasons why I didn't
>> >>>>>> explore
>> >>>>>>>> a
>> >>>>>>>>>>>> solution
>> >>>>>>>>>>>>> to
>> >>>>>>>>>>>>>>>>>> this is
>> >>>>>>>>>>>>>>>>>>> a) to keep this KIP as simple as possible,
>> >>> and
>> >>>> b)
>> >>>>>>>> because
>> >>>>>>>>>> I'm
>> >>>>>>>>>>>> not
>> >>>>>>>>>>>>>>>>>> exactly
>> >>>>>>>>>>>>>>>>>>> how to signal that a Task should commit
>> >>>>> prematurely.
>> >>>>>>>> I'm
>> >>>>>>>>>>>>> confident
>> >>>>>>>>>>>>>>>> it's
>> >>>>>>>>>>>>>>>>>>> possible, and I think it's worth adding a
>> >>>> section
>> >>>>> on
>> >>>>>>>>>> handling
>> >>>>>>>>>>>>> this.
>> >>>>>>>>>>>>>>>>>> Besides
>> >>>>>>>>>>>>>>>>>>> my proposal to force an early commit once
>> >>> memory
>> >>>>>> usage
>> >>>>>>>>>>> reaches
>> >>>>>>>>>>>> a
>> >>>>>>>>>>>>>>>>>> threshold,
>> >>>>>>>>>>>>>>>>>>> is there any other approach that you might
>> >>>> suggest
>> >>>>>> for
>> >>>>>>>>>>> tackling
>> >>>>>>>>>>>>>>> this
>> >>>>>>>>>>>>>>>>>>> problem?
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> 3. ALOS: I can add in an explicit paragraph,
>> >>> but
>> >>>>> my
>> >>>>>>>>>>> assumption
>> >>>>>>>>>>>> is
>> >>>>>>>>>>>>>>> that
>> >>>>>>>>>>>>>>>>>>> since transactional behaviour comes at
>> >>> little/no
>> >>>>>>>> cost, that
>> >>>>>>>>>>> it
>> >>>>>>>>>>>>>>> should
>> >>>>>>>>>>>>>>>> be
>> >>>>>>>>>>>>>>>>>>> available by default on all stores,
>> >>> irrespective
>> >>>>> of
>> >>>>>>>> the
>> >>>>>>>>>>>>> processing
>> >>>>>>>>>>>>>>>> mode.
>> >>>>>>>>>>>>>>>>>>> While ALOS doesn't use transactions, the Task
>> >>>>> itself
>> >>>>>>>> still
>> >>>>>>>>>>>>>>> "commits",
>> >>>>>>>>>>>>>>>> so
>> >>>>>>>>>>>>>>>>>>> the behaviour should be correct under ALOS
>> >>> too.
>> >>>>> I'm
>> >>>>>>>> not
>> >>>>>>>>>>>> convinced
>> >>>>>>>>>>>>>>> that
>> >>>>>>>>>>>>>>>>>> it's
>> >>>>>>>>>>>>>>>>>>> worth having both
>> >>>> transactional/non-transactional
>> >>>>>>>> stores
>> >>>>>>>>>>>>>>> available, as
>> >>>>>>>>>>>>>>>>>> it
>> >>>>>>>>>>>>>>>>>>> would considerably increase the complexity of
>> >>>> the
>> >>>>>>>> codebase,
>> >>>>>>>>>>> for
>> >>>>>>>>>>>>>>> very
>> >>>>>>>>>>>>>>>>>> little
>> >>>>>>>>>>>>>>>>>>> benefit.
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> 4. Method deprecation: Are you referring to
>> >>>>>>>>>>>>>>> StateStore#getPosition()?
>> >>>>>>>>>>>>>>>>>> As I
>> >>>>>>>>>>>>>>>>>>> understand it, Position contains the
>> >>> position of
>> >>>>> the
>> >>>>>>>>>> *source*
>> >>>>>>>>>>>>>>> topics,
>> >>>>>>>>>>>>>>>>>>> whereas the commit offsets would be the
>> >>>>> *changelog*
>> >>>>>>>>>> offsets.
>> >>>>>>>>>>> So
>> >>>>>>>>>>>>>>> it's
>> >>>>>>>>>>>>>>>>>> still
>> >>>>>>>>>>>>>>>>>>> necessary to retain the Position data, as
>> >>> well
>> >>>> as
>> >>>>>> the
>> >>>>>>>>>>> changelog
>> >>>>>>>>>>>>>>>> offsets.
>> >>>>>>>>>>>>>>>>>>> What I meant in the KIP is that Position
>> >>> offsets
>> >>>>> are
>> >>>>>>>>>>> currently
>> >>>>>>>>>>>>>>> stored
>> >>>>>>>>>>>>>>>>>> in a
>> >>>>>>>>>>>>>>>>>>> file, and since we can atomically store
>> >>> metadata
>> >>>>>>>> along with
>> >>>>>>>>>>> the
>> >>>>>>>>>>>>>>> record
>> >>>>>>>>>>>>>>>>>>> batch we commit to RocksDB, we can move our
>> >>>>> Position
>> >>>>>>>>>> offsets
>> >>>>>>>>>>> in
>> >>>>>>>>>>>>> to
>> >>>>>>>>>>>>>>>> this
>> >>>>>>>>>>>>>>>>>>> metadata too, and gain the same transactional
>> >>>>>>>> guarantees
>> >>>>>>>>>> that
>> >>>>>>>>>>>> we
>> >>>>>>>>>>>>>>> will
>> >>>>>>>>>>>>>>>>>> for
>> >>>>>>>>>>>>>>>>>>> changelog offsets, ensuring that the Position
>> >>>>>> offsets
>> >>>>>>>> are
>> >>>>>>>>>>>>>>> consistent
>> >>>>>>>>>>>>>>>>>> with
>> >>>>>>>>>>>>>>>>>>> the records that are read from the database.
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> Regards,
>> >>>>>>>>>>>>>>>>>>> Nick
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>> On Tue, 22 Nov 2022 at 16:25, John Roesler <
>> >>>>>>>>>>>> vvcephei@apache.org>
>> >>>>>>>>>>>>>>>> wrote:
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Thanks for publishing this alternative,
>> >>> Nick!
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> The benchmark you mentioned in the KIP-844
>> >>>>>>>> discussion
>> >>>>>>>>>> seems
>> >>>>>>>>>>>>> like
>> >>>>>>>>>>>>>>> a
>> >>>>>>>>>>>>>>>>>>>> compelling reason to revisit the built-in
>> >>>>>>>>>> transactionality
>> >>>>>>>>>>>>>>>> mechanism.
>> >>>>>>>>>>>>>>>>>> I
>> >>>>>>>>>>>>>>>>>>>> also appreciate you analysis, showing that
>> >>> for
>> >>>>>> most
>> >>>>>>>> use
>> >>>>>>>>>>>> cases,
>> >>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>> write
>> >>>>>>>>>>>>>>>>>>>> batch approach should be just fine.
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> There are a couple of points that would
>> >>> hold
>> >>>> me
>> >>>>>>>> back from
>> >>>>>>>>>>>>>>> approving
>> >>>>>>>>>>>>>>>>>> this
>> >>>>>>>>>>>>>>>>>>>> KIP right now:
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> 1. Loss of coverage for custom stores.
>> >>>>>>>>>>>>>>>>>>>> The fact that you can plug in a
>> >>> (relatively)
>> >>>>>> simple
>> >>>>>>>>>>>>>>> implementation
>> >>>>>>>>>>>>>>>> of
>> >>>>>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>>>> XStateStore interfaces and automagically
>> >>> get a
>> >>>>>>>>>> distributed
>> >>>>>>>>>>>>>>> database
>> >>>>>>>>>>>>>>>>>> out
>> >>>>>>>>>>>>>>>>>>> of
>> >>>>>>>>>>>>>>>>>>>> it is a significant benefit of Kafka
>> >>> Streams.
>> >>>>> I'd
>> >>>>>>>> hate to
>> >>>>>>>>>>>> lose
>> >>>>>>>>>>>>>>> it,
>> >>>>>>>>>>>>>>>> so
>> >>>>>>>>>>>>>>>>>> it
>> >>>>>>>>>>>>>>>>>>>> would be better to spend some time and
>> >>> come up
>> >>>>>> with
>> >>>>>>>> a way
>> >>>>>>>>>>> to
>> >>>>>>>>>>>>>>>> preserve
>> >>>>>>>>>>>>>>>>>>> that
>> >>>>>>>>>>>>>>>>>>>> property. For example, can we provide a
>> >>>> default
>> >>>>>>>>>>>> implementation
>> >>>>>>>>>>>>> of
>> >>>>>>>>>>>>>>>>>>>> `commit(..)` that re-implements the
>> >>> existing
>> >>>>>>>>>>> checkpoint-file
>> >>>>>>>>>>>>>>>>>> approach? Or
>> >>>>>>>>>>>>>>>>>>>> perhaps add an `isTransactional()` flag to
>> >>> the
>> >>>>>> state
>> >>>>>>>>>> store
>> >>>>>>>>>>>>>>> interface
>> >>>>>>>>>>>>>>>>>> so
>> >>>>>>>>>>>>>>>>>>>> that the runtime can decide whether to
>> >>>> continue
>> >>>>> to
>> >>>>>>>> manage
>> >>>>>>>>>>>>>>> checkpoint
>> >>>>>>>>>>>>>>>>>>> files
>> >>>>>>>>>>>>>>>>>>>> vs delegating transactionality to the
>> >>> stores?
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> 2. Guarding against OOME
>> >>>>>>>>>>>>>>>>>>>> I appreciate your analysis, but I don't
>> >>> think
>> >>>>> it's
>> >>>>>>>>>>> sufficient
>> >>>>>>>>>>>>> to
>> >>>>>>>>>>>>>>> say
>> >>>>>>>>>>>>>>>>>> that
>> >>>>>>>>>>>>>>>>>>>> we will solve the memory problem later if
>> >>> it
>> >>>>>> becomes
>> >>>>>>>>>>>> necessary.
>> >>>>>>>>>>>>>>> The
>> >>>>>>>>>>>>>>>>>>>> experience leading to that situation would
>> >>> be
>> >>>>>> quite
>> >>>>>>>> bad:
>> >>>>>>>>>>>>> Imagine,
>> >>>>>>>>>>>>>>>> you
>> >>>>>>>>>>>>>>>>>>>> upgrade to AK 3.next, your tests pass, so
>> >>> you
>> >>>>>>>> deploy to
>> >>>>>>>>>>>>>>> production.
>> >>>>>>>>>>>>>>>>>> That
>> >>>>>>>>>>>>>>>>>>>> night, you get paged because your app is
>> >>> now
>> >>>>>>>> crashing
>> >>>>>>>>>> with
>> >>>>>>>>>>>>>>> OOMEs. As
>> >>>>>>>>>>>>>>>>>> with
>> >>>>>>>>>>>>>>>>>>>> all OOMEs, you'll have a really hard time
>> >>>>> finding
>> >>>>>>>> the
>> >>>>>>>>>> root
>> >>>>>>>>>>>>> cause,
>> >>>>>>>>>>>>>>>> and
>> >>>>>>>>>>>>>>>>>>> once
>> >>>>>>>>>>>>>>>>>>>> you do, you won't have a clear path to
>> >>> resolve
>> >>>>> the
>> >>>>>>>> issue.
>> >>>>>>>>>>> You
>> >>>>>>>>>>>>>>> could
>> >>>>>>>>>>>>>>>>>> only
>> >>>>>>>>>>>>>>>>>>>> tune down the commit interval and cache
>> >>> buffer
>> >>>>>> size
>> >>>>>>>> until
>> >>>>>>>>>>> you
>> >>>>>>>>>>>>>>> stop
>> >>>>>>>>>>>>>>>>>>> getting
>> >>>>>>>>>>>>>>>>>>>> crashes.
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> FYI, I know of multiple cases where people
>> >>> run
>> >>>>> EOS
>> >>>>>>>> with
>> >>>>>>>>>>> much
>> >>>>>>>>>>>>>>> larger
>> >>>>>>>>>>>>>>>>>>> commit
>> >>>>>>>>>>>>>>>>>>>> intervals to get better batching than the
>> >>>>> default,
>> >>>>>>>> so I
>> >>>>>>>>>>> don't
>> >>>>>>>>>>>>>>> think
>> >>>>>>>>>>>>>>>>>> this
>> >>>>>>>>>>>>>>>>>>>> pathological case would be as rare as you
>> >>>>> suspect.
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Given that we already have the rudiments
>> >>> of an
>> >>>>>> idea
>> >>>>>>>> of
>> >>>>>>>>>> what
>> >>>>>>>>>>>> we
>> >>>>>>>>>>>>>>> could
>> >>>>>>>>>>>>>>>>>> do
>> >>>>>>>>>>>>>>>>>>> to
>> >>>>>>>>>>>>>>>>>>>> prevent this downside, we should take the
>> >>> time
>> >>>>> to
>> >>>>>>>> design
>> >>>>>>>>>> a
>> >>>>>>>>>>>>>>> solution.
>> >>>>>>>>>>>>>>>>>> We
>> >>>>>>>>>>>>>>>>>>> owe
>> >>>>>>>>>>>>>>>>>>>> it to our users to ensure that awesome new
>> >>>>>> features
>> >>>>>>>> don't
>> >>>>>>>>>>>> come
>> >>>>>>>>>>>>>>> with
>> >>>>>>>>>>>>>>>>>>> bitter
>> >>>>>>>>>>>>>>>>>>>> pills unless we can't avoid it.
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> 3. ALOS mode.
>> >>>>>>>>>>>>>>>>>>>> On the other hand, I didn't see an
>> >>> indication
>> >>>> of
>> >>>>>> how
>> >>>>>>>>>> stores
>> >>>>>>>>>>>>> will
>> >>>>>>>>>>>>>>> be
>> >>>>>>>>>>>>>>>>>>>> handled under ALOS (aka non-EOS) mode.
>> >>>>>>>> Theoretically, the
>> >>>>>>>>>>>>>>>>>>> transactionality
>> >>>>>>>>>>>>>>>>>>>> of the store and the processing mode are
>> >>>>>>>> orthogonal. A
>> >>>>>>>>>>>>>>> transactional
>> >>>>>>>>>>>>>>>>>>> store
>> >>>>>>>>>>>>>>>>>>>> would serve ALOS just as well as a
>> >>>>>>>> non-transactional one
>> >>>>>>>>>>> (if
>> >>>>>>>>>>>>> not
>> >>>>>>>>>>>>>>>>>> better).
>> >>>>>>>>>>>>>>>>>>>> Under ALOS, though, the default commit
>> >>>> interval
>> >>>>> is
>> >>>>>>>> five
>> >>>>>>>>>>>>> minutes,
>> >>>>>>>>>>>>>>> so
>> >>>>>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>>>> memory issue is far more pressing.
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> As I see it, we have several options to
>> >>>> resolve
>> >>>>>> this
>> >>>>>>>>>> point.
>> >>>>>>>>>>>> We
>> >>>>>>>>>>>>>>> could
>> >>>>>>>>>>>>>>>>>>>> demonstrate that transactional stores work
>> >>>> just
>> >>>>>>>> fine for
>> >>>>>>>>>>> ALOS
>> >>>>>>>>>>>>>>> and we
>> >>>>>>>>>>>>>>>>>> can
>> >>>>>>>>>>>>>>>>>>>> therefore just swap over unconditionally.
>> >>> We
>> >>>>> could
>> >>>>>>>> also
>> >>>>>>>>>>>> disable
>> >>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>>>> transactional mechanism under ALOS so that
>> >>>>> stores
>> >>>>>>>> operate
>> >>>>>>>>>>>> just
>> >>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>> same
>> >>>>>>>>>>>>>>>>>>> as
>> >>>>>>>>>>>>>>>>>>>> they do today when run in ALOS mode.
>> >>> Finally,
>> >>>> we
>> >>>>>>>> could do
>> >>>>>>>>>>> the
>> >>>>>>>>>>>>>>> same
>> >>>>>>>>>>>>>>>> as
>> >>>>>>>>>>>>>>>>>> in
>> >>>>>>>>>>>>>>>>>>>> KIP-844 and make transactional stores
>> >>> opt-in
>> >>>>> (it'd
>> >>>>>>>> be
>> >>>>>>>>>>> better
>> >>>>>>>>>>>> to
>> >>>>>>>>>>>>>>>> avoid
>> >>>>>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>>>> extra opt-in mechanism, but it's a good
>> >>>>>>>>>>> get-out-of-jail-free
>> >>>>>>>>>>>>>>> card).
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> 4. (minor point) Deprecation of methods
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> You mentioned that the new `commit` 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?
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> Thanks again for the KIP! It's really nice
>> >>>> that
>> >>>>>> you
>> >>>>>>>> and
>> >>>>>>>>>>> Alex
>> >>>>>>>>>>>>> will
>> >>>>>>>>>>>>>>>> get
>> >>>>>>>>>>>>>>>>>> the
>> >>>>>>>>>>>>>>>>>>>> chance to collaborate on both directions so
>> >>>> that
>> >>>>>> we
>> >>>>>>>> can
>> >>>>>>>>>> get
>> >>>>>>>>>>>> the
>> >>>>>>>>>>>>>>> best
>> >>>>>>>>>>>>>>>>>>>> outcome for Streams and its users.
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> -John
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>> On 2022/11/21 15:02:15 Nick Telford wrote:
>> >>>>>>>>>>>>>>>>>>>>> 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
>> >>>>>>>>>>>>>>>>>>>>> <
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> >>>>>>>>>>>>>>>>>>>>> ,
>> >>>>>>>>>>>>>>>>>>>>> so that it can be discussed/reviewed
>> >>>>> separately
>> >>>>>>>> from
>> >>>>>>>>>>>> KIP-844.
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> Alex: I'm especially interested in what
>> >>> you
>> >>>>>> think!
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> I have a nearly complete implementation
>> >>> of
>> >>>> the
>> >>>>>>>> changes
>> >>>>>>>>>>>>>>> outlined in
>> >>>>>>>>>>>>>>>>>> this
>> >>>>>>>>>>>>>>>>>>>>> KIP, please let me know if you'd like me
>> >>> to
>> >>>>> push
>> >>>>>>>> them
>> >>>>>>>>>> for
>> >>>>>>>>>>>>>>> review
>> >>>>>>>>>>>>>>>> in
>> >>>>>>>>>>>>>>>>>>>> advance
>> >>>>>>>>>>>>>>>>>>>>> of a vote.
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> Regards,
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>> Nick
>> >>>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>>
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>>
>> >>>>>
>> >>>>
>> >>>
>> >>
>> >
>>
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Bruno,

Thanks for reviewing my proposal.

1.
The main reason I added it was because it was easy to do. If we see no
value in it, I can remove it.

2.
Global StateStores can have multiple partitions in their input topics
(which function as their changelogs), so they would have more than one
partition.

3.
That's a good point. At present, the only method it adds is
isolationLevel(), which is likely not necessary outside of StateStores.
It *does* provide slightly different guarantees in the documentation to
several of the methods (hence the overrides). I'm not sure if this is
enough to warrant a new interface though.
I think the question that remains is whether this interface makes it easier
to implement custom transactional StateStores than if we were to remove it?
Probably not.

4.
The main motivation for the Atomic Checkpointing is actually performance.
My team has been testing out an implementation of this KIP without it, and
we had problems with RocksDB doing *much* more compaction, due to the
significantly increased flush rate. It was enough of a problem that (for
the time being), we had to revert back to Kafka Streams proper.
I think the best way to solve this, as you say, is to keep the .checkpoint
files *in addition* to the offsets being stored within the store itself.
Essentially, when closing StateStores, we force a memtable flush, then call
getCommittedOffsets and write those out to the .checkpoint file. That would
ensure the metadata is available to the StreamsPartitionAssignor for all
closed stores.
If there's a crash (no clean close), then we won't be able to guarantee
which offsets were flushed to disk by RocksDB, so we'd need to open (init()),
read offsets, and then close() those stores. But since this is the
exception, and will only occur once (provided it doesn't crash every
time!), I think the performance impact here would be acceptable.

Thanks for the feedback, please let me know if you have any more comments
or questions!

I'm currently working on rebasing against trunk. This involves adding
support for transactionality to VersionedStateStores. I will probably need
to revise my implementation for transactional "segmented" stores, both to
accommodate VersionedStateStore, and to clean up some other stuff.

Regards,
Nick


On Tue, 2 May 2023 at 13:45, Bruno Cadonna <ca...@apache.org> wrote:

> Hi Nick,
>
> Thanks for the updates!
>
> I have a couple of questions/comments.
>
> 1.
> Why do you propose a configuration that involves max. bytes and max.
> reords? I think we are mainly concerned about memory consumption because
> we want to limit the off-heap memory used. I cannot think of a case
> where one would want to set the max. number of records.
>
>
> 2.
> Why does
>
>   default void commit(final Map<TopicPartition, Long> changelogOffsets) {
>       flush();
>   }
>
> take a map of partitions to changelog offsets?
> The mapping between state stores to partitions is a 1:1 relationship.
> Passing in a single changelog offset should suffice.
>
>
> 3.
> Why do we need the Transaction interface? It should be possible to hide
> beginning and committing a transactions withing the state store
> implementation, so that from outside the state store, it does not matter
> whether the state store is transactional or not. What would be the
> advantage of using the Transaction interface?
>
>
> 4.
> Regarding checkpointing offsets, I think we should keep the checkpoint
> file in any case for the reason you mentioned about rebalancing. Even if
> that would not be an issue, I would propose to move the change to offset
> management to a new KIP and to not add more complexity than needed to
> this one. I would not be too concerned about the consistency violation
> you mention. As far as I understand, with transactional state stores
> Streams would write the checkpoint file during every commit even under
> EOS. In the failure case you describe, Streams would restore the state
> stores from the offsets found in the checkpoint file written during the
> penultimate commit instead of during the last commit. Basically, Streams
> would overwrite the records written to the state store between the last
> two commits with the same records read from the changelogs. While I
> understand that this is wasteful, it is -- at the same time --
> acceptable and most importantly it does not break EOS.
>
> Best,
> Bruno
>
>
> On 27.04.23 12:34, Nick Telford wrote:
> > Hi everyone,
> >
> > I find myself (again) considering removing the offset management from
> > StateStores, and keeping the old checkpoint file system. The reason is
> that
> > the StreamPartitionAssignor directly reads checkpoint files in order to
> > determine which instance has the most up-to-date copy of the local state.
> > If we move offsets into the StateStore itself, then we will need to open,
> > initialize, read offsets and then close each StateStore (that is not
> > already assigned and open) for which we have *any* local state, on every
> > rebalance.
> >
> > Generally, I don't think there are many "orphan" stores like this sitting
> > around on most instances, but even a few would introduce additional
> latency
> > to an already somewhat lengthy rebalance procedure.
> >
> > I'm leaning towards Colt's (Slack) suggestion of just keeping things in
> the
> > checkpoint file(s) for now, and not worrying about the race. The downside
> > is that we wouldn't be able to remove the explicit RocksDB flush
> on-commit,
> > which likely hurts performance.
> >
> > If anyone has any thoughts or ideas on this subject, I would appreciate
> it!
> >
> > Regards,
> > Nick
> >
> > On Wed, 19 Apr 2023 at 15:05, Nick Telford <ni...@gmail.com>
> wrote:
> >
> >> Hi Colt,
> >>
> >> The issue is that if there's a crash between 2 and 3, then you still end
> >> up with inconsistent data in RocksDB. The only way to guarantee that
> your
> >> checkpoint offsets and locally stored data are consistent with each
> other
> >> are to atomically commit them, which can be achieved by having the
> offsets
> >> stored in RocksDB.
> >>
> >> The offsets column family is likely to be extremely small (one
> >> per-changelog partition + one per Topology input partition for regular
> >> stores, one per input partition for global stores). So the overhead
> will be
> >> minimal.
> >>
> >> A major benefit of doing this is that we can remove the explicit calls
> to
> >> db.flush(), which forcibly flushes memtables to disk on-commit. It turns
> >> out, RocksDB memtable flushes are largely dictated by Kafka Streams
> >> commits, *not* RocksDB configuration, which could be a major source of
> >> confusion. Atomic checkpointing makes it safe to remove these explicit
> >> flushes, because it no longer matters exactly when RocksDB flushes data
> to
> >> disk; since the data and corresponding checkpoint offsets will always be
> >> flushed together, the local store is always in a consistent state, and
> >> on-restart, it can always safely resume restoration from the on-disk
> >> offsets, restoring the small amount of data that hadn't been flushed
> when
> >> the app exited/crashed.
> >>
> >> Regards,
> >> Nick
> >>
> >> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io> wrote:
> >>
> >>> Nick,
> >>>
> >>> Thanks for your reply. Ack to A) and B).
> >>>
> >>> For item C), I see what you're referring to. Your proposed solution
> will
> >>> work, so no need to change it. What I was suggesting was that it might
> be
> >>> possible to achieve this with only one column family. So long as:
> >>>
> >>>     - No uncommitted records (i.e. not committed to the changelog) are
> >>>     *committed* to the state store, AND
> >>>     - The Checkpoint offset (which refers to the changelog topic) is
> less
> >>>     than or equal to the last written changelog offset in rocksdb
> >>>
> >>> I don't see the need to do the full restoration from scratch. My
> >>> understanding was that prior to 844/892, full restorations were
> required
> >>> because there could be uncommitted records written to RocksDB; however,
> >>> given your use of RocksDB transactions, that can be avoided with the
> >>> pattern of 1) commit Kafka transaction, 2) commit RocksDB transaction,
> 3)
> >>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <ni...@gmail.com>
> >>> wrote:
> >>>
> >>>> Hi Colt,
> >>>>
> >>>> A. I've done my best to de-couple the StateStore stuff from the rest
> of
> >>> the
> >>>> Streams engine. The fact that there will be only one ongoing (write)
> >>>> transaction at a time is not guaranteed by any API, and is just a
> >>>> consequence of the way Streams operates. To that end, I tried to
> ensure
> >>> the
> >>>> documentation and guarantees provided by the new APIs are independent
> of
> >>>> this incidental behaviour. In practice, you're right, this essentially
> >>>> refers to "interactive queries", which are technically "read
> >>> transactions",
> >>>> even if they don't actually use the transaction API to isolate
> >>> themselves.
> >>>>
> >>>> B. Yes, although not ideal. This is for backwards compatibility,
> >>> because:
> >>>>      1) Existing custom StateStore implementations will implement
> >>> flush(),
> >>>> and not commit(), but the Streams engine now calls commit(), so those
> >>> calls
> >>>> need to be forwarded to flush() for these legacy stores.
> >>>>      2) Existing StateStore *users*, i.e. outside of the Streams
> engine
> >>>> itself, may depend on explicitly calling flush(), so for these cases,
> >>>> flush() needs to be redirected to call commit().
> >>>> If anyone has a better way to guarantee compatibility without
> >>> introducing
> >>>> this potential recursion loop, I'm open to changes!
> >>>>
> >>>> C. This is described in the "Atomic Checkpointing" section. Offsets
> are
> >>>> stored in a separate RocksDB column family, which is guaranteed to be
> >>>> atomically flushed to disk with all other column families. The issue
> of
> >>>> checkpoints being written to disk after commit causing inconsistency
> if
> >>> it
> >>>> crashes in between is the reason why, under EOS, checkpoint files are
> >>> only
> >>>> written on clean shutdown. This is one of the major causes of "full
> >>>> restorations", so moving the offsets into a place where they can be
> >>>> guaranteed to be atomically written with the data they checkpoint
> >>> allows us
> >>>> to write the checkpoint offsets *on every commit*, not just on clean
> >>>> shutdown.
> >>>>
> >>>> Regards,
> >>>> Nick
> >>>>
> >>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io>
> wrote:
> >>>>
> >>>>> Nick,
> >>>>>
> >>>>> Thank you for continuing this work. I have a few minor clarifying
> >>>>> questions.
> >>>>>
> >>>>> A) "Records written to any transaction are visible to all other
> >>>>> transactions immediately." I am confused here—I thought there could
> >>> only
> >>>> be
> >>>>> one transaction going on at a time for a given state store given the
> >>>>> threading model for processing records on a Task. Do you mean
> >>> Interactive
> >>>>> Queries by "other transactions"? (If so, then everything makes
> sense—I
> >>>>> thought that since IQ were read-only then they didn't count as
> >>>>> transactions).
> >>>>>
> >>>>> B) Is it intentional that the default implementations of the flush()
> >>> and
> >>>>> commit() methods in the StateStore class refer to each other in some
> >>> sort
> >>>>> of unbounded recursion?
> >>>>>
> >>>>> C) How will the getCommittedOffset() method work? At first I thought
> >>> the
> >>>>> way to do it would be using a special key in the RocksDB store to
> >>> store
> >>>> the
> >>>>> offset, and committing that with the transaction. But upon second
> >>>> thought,
> >>>>> since restoration from the changelog is an idempotent procedure, I
> >>> think
> >>>> it
> >>>>> would be fine to 1) commit the RocksDB transaction and then 2) write
> >>> the
> >>>>> offset to disk in a checkpoint file. If there is a crash between 1)
> >>> and
> >>>> 2),
> >>>>> I think the only downside is 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 <nick.telford@gmail.com
> >
> >>>>> wrote:
> >>>>>
> >>>>>> Hi everyone,
> >>>>>>
> >>>>>> I've updated the KIP to reflect the latest version of the design:
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>
> >>>>>> There are several changes in there that reflect feedback from this
> >>>>> thread,
> >>>>>> and there's a new section and a bunch of interface changes relating
> >>> to
> >>>>>> Atomic Checkpointing, which is the final piece of the puzzle to
> >>> making
> >>>>>> everything robust.
> >>>>>>
> >>>>>> Let me know what you think!
> >>>>>>
> >>>>>> Regards,
> >>>>>> Nick
> >>>>>>
> >>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <ni...@gmail.com>
> >>>>> wrote:
> >>>>>>
> >>>>>>> Hi Lucas,
> >>>>>>>
> >>>>>>> Thanks for looking over my KIP.
> >>>>>>>
> >>>>>>> A) The bound is per-instance, not per-Task. This was a typo in the
> >>>> KIP
> >>>>>>> that I've now corrected. It was originally per-Task, but I
> >>> changed it
> >>>>> to
> >>>>>>> per-instance for exactly the reason you highlighted.
> >>>>>>> B) It's worth noting that transactionality is only enabled under
> >>> EOS,
> >>>>> and
> >>>>>>> in the default mode of operation (ALOS), there should be no
> >>> change in
> >>>>>>> behavior at all. I think, under EOS, we can mitigate the impact on
> >>>>> users
> >>>>>> by
> >>>>>>> sufficiently low default values for the memory bound
> >>> configuration. I
> >>>>>>> understand your hesitation to include a significant change of
> >>>>> behaviour,
> >>>>>>> especially in a minor release, but I suspect that most users will
> >>>>> prefer
> >>>>>>> the memory impact (under EOS) to the existing behaviour of
> >>> frequent
> >>>>> state
> >>>>>>> restorations! If this is a problem, the changes can wait until the
> >>>> next
> >>>>>>> major release. I'll be running a patched version of streams in
> >>>>> production
> >>>>>>> with these changes as soon as they're ready, so it won't disrupt
> >>> me
> >>>> :-D
> >>>>>>> C) The main purpose of this sentence was just to note that some
> >>>> changes
> >>>>>>> will need to be made to the way Segments are handled in order to
> >>>> ensure
> >>>>>>> they also benefit from transactions. At the time I wrote it, I
> >>> hadn't
> >>>>>>> figured out the specific changes necessary, so it was deliberately
> >>>>> vague.
> >>>>>>> This is the one outstanding problem I'm currently working on, and
> >>>> I'll
> >>>>>>> update this section with more detail once I have figured out the
> >>>> exact
> >>>>>>> changes required.
> >>>>>>> D) newTransaction() provides the necessary isolation guarantees.
> >>>> While
> >>>>>>> the RocksDB implementation of transactions doesn't technically
> >>> *need*
> >>>>>>> read-only users to call newTransaction(), other implementations
> >>>> (e.g. a
> >>>>>>> hypothetical PostgresStore) may require it. Calling
> >>> newTransaction()
> >>>>> when
> >>>>>>> no transaction is necessary is essentially free, as it will just
> >>>> return
> >>>>>>> this.
> >>>>>>>
> >>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think it
> >>> should
> >>>> be
> >>>>>>> fairly obvious where the performance problems stem from: writes
> >>> under
> >>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with the
> >>>>>>> tombstone/record flag, 1 to decode it from the tombstone/record
> >>> flag,
> >>>>>> and 1
> >>>>>>> to copy the record from the "temporary" store to the "main" store,
> >>>> when
> >>>>>> the
> >>>>>>> transaction commits. The different approach taken by KIP-869
> >>> should
> >>>>>> perform
> >>>>>>> much better, as it avoids all these copies, and may actually
> >>> perform
> >>>>>>> slightly better than trunk, due to batched writes in RocksDB
> >>>> performing
> >>>>>>> better than non-batched writes.[1]
> >>>>>>>
> >>>>>>> Regards,
> >>>>>>> Nick
> >>>>>>>
> >>>>>>> 1:
> >>>>>>
> >>>>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>
> >>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
> >>> lbrutschy@confluent.io
> >>>>>> .invalid>
> >>>>>>> wrote:
> >>>>>>>
> >>>>>>>> Hi Nick,
> >>>>>>>>
> >>>>>>>> I'm just starting to read up on the whole discussion about
> >>> KIP-892
> >>>> and
> >>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
> >>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do have some
> >>>>>>>> questions about the latest draft.
> >>>>>>>>
> >>>>>>>>   A) If I understand correctly, you propose to put a bound on the
> >>>>>>>> (native) memory consumed by each task. However, I wonder if this
> >>> is
> >>>>>>>> sufficient if we have temporary imbalances in the cluster. For
> >>>>>>>> example, depending on the timing of rebalances during a cluster
> >>>>>>>> restart, it could happen that a single streams node is assigned a
> >>>> lot
> >>>>>>>> more tasks than expected. With your proposed change, this would
> >>> mean
> >>>>>>>> that the memory required by this one node could be a multiple of
> >>>> what
> >>>>>>>> is required during normal operation. I wonder if it wouldn't be
> >>>> safer
> >>>>>>>> to put a global bound on the memory use, across all tasks.
> >>>>>>>>   B) Generally, the memory concerns still give me the feeling that
> >>>> this
> >>>>>>>> should not be enabled by default for all users in a minor
> >>> release.
> >>>>>>>>   C) In section "Transaction Management": the sentence "A similar
> >>>>>>>> analogue will be created to automatically manage `Segment`
> >>>>>>>> transactions.". Maybe this is just me lacking some background,
> >>> 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 you to find out why the KIP-844 PoC is 20x slower
> >>> (e.g.
> >>>>>>>> by providing a flame graph), that would be quite interesting.
> >>>>>>>>
> >>>>>>>> Cheers,
> >>>>>>>> Lucas
> >>>>>>>>
> >>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
> >>>> nick.telford@gmail.com>
> >>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>> Hi everyone,
> >>>>>>>>>
> >>>>>>>>> I've updated the KIP with a more detailed design, which
> >>> reflects
> >>>> the
> >>>>>>>>> implementation I've been working on:
> >>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>
> >>>>>>>>> This new design should address the outstanding points already
> >>> made
> >>>>> in
> >>>>>>>> the
> >>>>>>>>> thread.
> >>>>>>>>>
> >>>>>>>>> Please let me know if there are areas that are unclear or need
> >>>> more
> >>>>>>>>> clarification.
> >>>>>>>>>
> >>>>>>>>> I have a (nearly) working implementation. I'm confident that
> >>> the
> >>>>>>>> remaining
> >>>>>>>>> work (making Segments behave) will not impact the documented
> >>>> design.
> >>>>>>>>>
> >>>>>>>>> Regards,
> >>>>>>>>>
> >>>>>>>>> Nick
> >>>>>>>>>
> >>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <colt@littlehorse.io
> >>>>
> >>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Nick,
> >>>>>>>>>>
> >>>>>>>>>> Thank you for the reply; that makes sense. I was hoping that,
> >>>>> since
> >>>>>>>> reading
> >>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
> >>> documented
> >>>>> API,
> >>>>>>>> maybe
> >>>>>>>>>> you *wouldn't* have to wait for the next major release to
> >>> make
> >>>>> that
> >>>>>>>> change;
> >>>>>>>>>> but given 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 <
> >>>>>> nick.telford@gmail.com>
> >>>>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>
> >>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended to
> >>> try
> >>>> to
> >>>>>>>> keep the
> >>>>>>>>>>> behaviour unchanged as much as possible, otherwise we'd
> >>> have
> >>>> to
> >>>>>>>> wait for
> >>>>>>>>>> a
> >>>>>>>>>>> major version release to land these changes.
> >>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
> >>> guarantee,
> >>>>> and
> >>>>>>>> the
> >>>>>>>>>>> typically longer commit intervals would be problematic when
> >>>>>> reading
> >>>>>>>> only
> >>>>>>>>>>> "committed" records.
> >>>>>>>>>>>
> >>>>>>>>>>> I've been away for 5 days recovering from minor surgery,
> >>> but I
> >>>>>>>> spent a
> >>>>>>>>>>> considerable amount of that time working through ideas for
> >>>>>> possible
> >>>>>>>>>>> solutions in my head. I think your suggestion of keeping
> >>> ALOS
> >>>>>>>> as-is, but
> >>>>>>>>>>> buffering writes for EOS is the right path forwards,
> >>> although
> >>>> I
> >>>>>>>> have a
> >>>>>>>>>>> solution that both expands on this, and provides for some
> >>> more
> >>>>>>>> formal
> >>>>>>>>>>> guarantees.
> >>>>>>>>>>>
> >>>>>>>>>>> Essentially, adding support to KeyValueStores for
> >>>>> "Transactions",
> >>>>>>>> with
> >>>>>>>>>>> clearly defined IsolationLevels. Using "Read Committed"
> >>> when
> >>>>> under
> >>>>>>>> EOS,
> >>>>>>>>>> and
> >>>>>>>>>>> "Read Uncommitted" under ALOS.
> >>>>>>>>>>>
> >>>>>>>>>>> The nice thing about this approach is that it gives us much
> >>>> more
> >>>>>>>> clearly
> >>>>>>>>>>> defined isolation behaviour that can be properly
> >>> documented to
> >>>>>>>> ensure
> >>>>>>>>>> users
> >>>>>>>>>>> know what to expect.
> >>>>>>>>>>>
> >>>>>>>>>>> I'm still working out the kinks in the design, and will
> >>> update
> >>>>> the
> >>>>>>>> KIP
> >>>>>>>>>> when
> >>>>>>>>>>> I have something. The main struggle is trying to implement
> >>>> this
> >>>>>>>> without
> >>>>>>>>>>> making any major changes to the existing interfaces or
> >>>> breaking
> >>>>>>>> existing
> >>>>>>>>>>> implementations, because currently everything expects to
> >>>> operate
> >>>>>>>> directly
> >>>>>>>>>>> on a StateStore, and not a Transaction of that store. I
> >>> think
> >>>>> I'm
> >>>>>>>> getting
> >>>>>>>>>>> close, although sadly I won't be able to progress much
> >>> until
> >>>>> next
> >>>>>>>> week
> >>>>>>>>>> due
> >>>>>>>>>>> to some work commitments.
> >>>>>>>>>>>
> >>>>>>>>>>> Regards,
> >>>>>>>>>>> Nick
> >>>>>>>>>>>
> >>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
> >>>> colt@littlehorse.io>
> >>>>>>>> wrote:
> >>>>>>>>>>>
> >>>>>>>>>>>> Nick,
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thank you for the explanation, and also for the updated
> >>>> KIP. I
> >>>>>> am
> >>>>>>>> quite
> >>>>>>>>>>>> eager for this improvement to be released as it would
> >>>> greatly
> >>>>>>>> reduce
> >>>>>>>>>> the
> >>>>>>>>>>>> operational difficulties of EOS streams apps.
> >>>>>>>>>>>>
> >>>>>>>>>>>> Two questions:
> >>>>>>>>>>>>
> >>>>>>>>>>>> 10)
> >>>>>>>>>>>>> When reading records, we will use the
> >>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
> >>>>>>>>>>>>   and WriteBatchWithIndex#newIteratorWithBase utilities in
> >>>>> order
> >>>>>> to
> >>>>>>>>>> ensure
> >>>>>>>>>>>> that uncommitted writes are available to query.
> >>>>>>>>>>>> Why do extra work to enable the reading of uncommitted
> >>>> writes
> >>>>>>>> during
> >>>>>>>>>> IQ?
> >>>>>>>>>>>> Code complexity aside, reading uncommitted writes is, in
> >>> my
> >>>>>>>> opinion, a
> >>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have the
> >>>>>> guarantee
> >>>>>>>> that,
> >>>>>>>>>>>> with EOS, IQ only reads committed records. In order to
> >>> avoid
> >>>>>> dirty
> >>>>>>>>>> reads,
> >>>>>>>>>>>> one currently must query a standby replica (but this
> >>> still
> >>>>>> doesn't
> >>>>>>>>>> fully
> >>>>>>>>>>>> guarantee monotonic reads).
> >>>>>>>>>>>>
> >>>>>>>>>>>> 20) Is it also necessary to enable this optimization on
> >>> ALOS
> >>>>>>>> stores?
> >>>>>>>>>> The
> >>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need to
> >>>> restore
> >>>>>>>> state
> >>>>>>>>>> from
> >>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
> >>>> acceptable
> >>>>> to
> >>>>>>>> accept
> >>>>>>>>>>>> that there may have been uncommitted writes on disk. On a
> >>>> side
> >>>>>>>> note, if
> >>>>>>>>>>> you
> >>>>>>>>>>>> enable this type of store on ALOS processors, the
> >>> community
> >>>>>> would
> >>>>>>>>>>>> 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 <
> >>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
> >>>> longer
> >>>>>>>> include
> >>>>>>>>>> the
> >>>>>>>>>>>>> StateStore management of checkpointing. This can be
> >>> added
> >>>>> as a
> >>>>>>>> KIP
> >>>>>>>>>>> later
> >>>>>>>>>>>> on
> >>>>>>>>>>>>> to further optimize the consistency and performance of
> >>>> state
> >>>>>>>> stores.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I've also added a section discussing some of the
> >>> concerns
> >>>>>> 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 2022 at 16:36, Nick Telford <
> >>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> Hi Colt,
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I didn't do any profiling, but the 844
> >>> implementation:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>     - Writes uncommitted records to a temporary
> >>> RocksDB
> >>>>>>>> instance
> >>>>>>>>>>>>>>        - Since tombstones need to be flagged, all
> >>> record
> >>>>>>>> values are
> >>>>>>>>>>>>>>        prefixed with a value/tombstone marker. This
> >>>>>>>> necessitates a
> >>>>>>>>>>>> memory
> >>>>>>>>>>>>> copy.
> >>>>>>>>>>>>>>     - On-commit, iterates all records in this
> >>> temporary
> >>>>>>>> instance and
> >>>>>>>>>>>>>>     writes them to the main RocksDB store.
> >>>>>>>>>>>>>>     - While iterating, the value/tombstone marker
> >>> needs
> >>>> to
> >>>>> be
> >>>>>>>> parsed
> >>>>>>>>>>> and
> >>>>>>>>>>>>>>     the real value extracted. This necessitates
> >>> another
> >>>>>> memory
> >>>>>>>> copy.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> My guess is that the cost of iterating the temporary
> >>>>> RocksDB
> >>>>>>>> store
> >>>>>>>>>> is
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>> major factor, with the 2 extra memory copies
> >>> per-Record
> >>>>>>>>>> contributing
> >>>>>>>>>>> a
> >>>>>>>>>>>>>> significant amount too.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
> >>>>>>>> colt@littlehorse.io>
> >>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Hi all,
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Out of curiosity, why does the performance of the
> >>> store
> >>>>>>>> degrade so
> >>>>>>>>>>>>>>> significantly with the 844 implementation? I
> >>> wouldn't
> >>>> be
> >>>>>> too
> >>>>>>>>>>> surprised
> >>>>>>>>>>>>> by
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>> 50-60% drop (caused by each record being written
> >>>> twice),
> >>>>>> but
> >>>>>>>> 96%
> >>>>>>>>>> is
> >>>>>>>>>>>>>>> extreme.
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> The only thing I can think of which could create
> >>> such a
> >>>>>>>> bottleneck
> >>>>>>>>>>>> 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 Nick Telford <
> >>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi everyone,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> I've updated the KIP to resolve all the points
> >>> that
> >>>>> have
> >>>>>>>> been
> >>>>>>>>>>> raised
> >>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>> far, with one exception: the ALOS default commit
> >>>>> interval
> >>>>>>>> of 5
> >>>>>>>>>>>> minutes
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>> likely to cause WriteBatchWithIndex memory to grow
> >>>> too
> >>>>>>>> large.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> There's a couple of different things I can think
> >>> of
> >>>> to
> >>>>>>>> solve
> >>>>>>>>>> this:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>     - We already have a memory/record limit in the
> >>> KIP
> >>>>> to
> >>>>>>>> prevent
> >>>>>>>>>>> OOM
> >>>>>>>>>>>>>>>>     errors. Should we choose a default value for
> >>>> these?
> >>>>> My
> >>>>>>>>>> concern
> >>>>>>>>>>>> here
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>     anything we choose might seem rather
> >>> arbitrary. We
> >>>>>> could
> >>>>>>>>>> change
> >>>>>>>>>>>>>>>>     its behaviour such that under ALOS, it only
> >>>> triggers
> >>>>>> the
> >>>>>>>>>> commit
> >>>>>>>>>>>> of
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>     StateStore, but under EOS, it triggers a
> >>> commit of
> >>>>> the
> >>>>>>>> Kafka
> >>>>>>>>>>>>>>>> transaction.
> >>>>>>>>>>>>>>>>     - We could introduce a separate `
> >>>>>> checkpoint.interval.ms`
> >>>>>>>> to
> >>>>>>>>>>>> allow
> >>>>>>>>>>>>>>> ALOS
> >>>>>>>>>>>>>>>>     to commit the StateStores more frequently than
> >>> the
> >>>>>>>> general
> >>>>>>>>>>>>>>>>     commit.interval.ms? My concern here is that
> >>> the
> >>>>>>>> semantics of
> >>>>>>>>>>>> this
> >>>>>>>>>>>>>>>> config
> >>>>>>>>>>>>>>>>     would depend 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 <
> >>>>>>>>>>> nick.telford@gmail.com>
> >>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Hi Alex,
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks for the feedback.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I've updated the discussion of OOM issues by
> >>>>> describing
> >>>>>>>> how
> >>>>>>>>>>> we'll
> >>>>>>>>>>>>>>> handle
> >>>>>>>>>>>>>>>>> it. Here's the new text:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> To mitigate this, we will automatically force a
> >>>> Task
> >>>>>>>> commit if
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>> total
> >>>>>>>>>>>>>>>>>> uncommitted records returned by
> >>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedEntries()
> >>>>>> exceeds a
> >>>>>>>>>>>> threshold,
> >>>>>>>>>>>>>>>>>> configured by
> >>>>> max.uncommitted.state.entries.per.task;
> >>>>>>>> or the
> >>>>>>>>>>>> total
> >>>>>>>>>>>>>>>>>> memory used for buffering uncommitted records
> >>>>> returned
> >>>>>>>> by
> >>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
> >>>> exceeds
> >>>>>> the
> >>>>>>>>>>> threshold
> >>>>>>>>>>>>>>>>>> configured by
> >>>> max.uncommitted.state.bytes.per.task.
> >>>>>>>> This will
> >>>>>>>>>>>>> roughly
> >>>>>>>>>>>>>>>>>> bound the memory required per-Task for
> >>> buffering
> >>>>>>>> uncommitted
> >>>>>>>>>>>>> records,
> >>>>>>>>>>>>>>>>>> irrespective of the commit.interval.ms, and
> >>> will
> >>>>>>>> effectively
> >>>>>>>>>>>> bound
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> number of records that will need to be
> >>> restored in
> >>>>> the
> >>>>>>>> event
> >>>>>>>>>>> of a
> >>>>>>>>>>>>>>>> failure.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> These limits will be checked in
> >>> StreamTask#process
> >>>>> and
> >>>>>> a
> >>>>>>>>>>> premature
> >>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>> will be requested via Task#requestCommit().
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Note that these new methods provide default
> >>>>>>>> implementations
> >>>>>>>>>> that
> >>>>>>>>>>>>>>> ensure
> >>>>>>>>>>>>>>>>>> existing custom stores and non-transactional
> >>>> stores
> >>>>>>>> (e.g.
> >>>>>>>>>>>>>>>>>> InMemoryKeyValueStore) do not force any early
> >>>>> commits.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I've chosen to have the StateStore expose
> >>>>>> approximations
> >>>>>>>> of
> >>>>>>>>>> its
> >>>>>>>>>>>>> buffer
> >>>>>>>>>>>>>>>>> size/count instead of opaquely requesting a
> >>> commit
> >>>> in
> >>>>>>>> order to
> >>>>>>>>>>>>>>> delegate
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> decision making to the Task itself. This enables
> >>>>> Tasks
> >>>>>>>> to look
> >>>>>>>>>>> at
> >>>>>>>>>>>>>>> *all*
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> their StateStores, and determine whether an
> >>> early
> >>>>>> commit
> >>>>>>>> is
> >>>>>>>>>>>>> necessary.
> >>>>>>>>>>>>>>>>> Notably, it enables pre-Task thresholds,
> >>> instead of
> >>>>>>>> per-Store,
> >>>>>>>>>>>> which
> >>>>>>>>>>>>>>>>> prevents Tasks with many StateStores from using
> >>>> much
> >>>>>> more
> >>>>>>>>>> memory
> >>>>>>>>>>>>> than
> >>>>>>>>>>>>>>>> Tasks
> >>>>>>>>>>>>>>>>> with one StateStore. This makes sense, since
> >>>> commits
> >>>>>> are
> >>>>>>>> done
> >>>>>>>>>>>>> by-Task,
> >>>>>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>> by-Store.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Prizes* for anyone who can come up with a better
> >>>> name
> >>>>>>>> for the
> >>>>>>>>>>> new
> >>>>>>>>>>>>>>> config
> >>>>>>>>>>>>>>>>> properties!
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Thanks for pointing out the potential
> >>> performance
> >>>>>> issues
> >>>>>>>> of
> >>>>>>>>>>> WBWI.
> >>>>>>>>>>>>> From
> >>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> benchmarks that user posted[1], it looks like
> >>> WBWI
> >>>>>> still
> >>>>>>>>>>> performs
> >>>>>>>>>>>>>>>>> considerably better than individual puts, which
> >>> is
> >>>>> the
> >>>>>>>>>> existing
> >>>>>>>>>>>>>>> design,
> >>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>> I'd actually expect a performance boost from
> >>> WBWI,
> >>>>> just
> >>>>>>>> not as
> >>>>>>>>>>>> great
> >>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>> we'd get from a plain WriteBatch. This does
> >>> suggest
> >>>>>> that
> >>>>>>>> a
> >>>>>>>>>> good
> >>>>>>>>>>>>>>>>> optimization would be to use a regular
> >>> WriteBatch
> >>>> for
> >>>>>>>>>>> restoration
> >>>>>>>>>>>>> (in
> >>>>>>>>>>>>>>>>> RocksDBStore#restoreBatch), since we know that
> >>>> those
> >>>>>>>> records
> >>>>>>>>>>> will
> >>>>>>>>>>>>>>> never
> >>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>> queried before they're committed.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> 1:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>
> >>>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> * Just kidding, no prizes, sadly.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Wed, 23 Nov 2022 at 12:28, Alexander
> >>> Sorokoumov
> >>>>>>>>>>>>>>>>> <as...@confluent.io.invalid> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 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 to keep the current
> >>>>> checkpointing
> >>>>>>>> as a
> >>>>>>>>>>>>> fallback
> >>>>>>>>>>>>>>>>>> mechanism. We want to keep existing users'
> >>>> workflows
> >>>>>>>> intact
> >>>>>>>>>> if
> >>>>>>>>>>> we
> >>>>>>>>>>>>>>> can. A
> >>>>>>>>>>>>>>>>>> non-intrusive way would be to add a separate
> >>>>>> StateStore
> >>>>>>>>>> method,
> >>>>>>>>>>>>> say,
> >>>>>>>>>>>>>>>>>> StateStore#managesCheckpointing(), that
> >>> controls
> >>>>>>>> whether the
> >>>>>>>>>>>> state
> >>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>> implementation owns checkpointing.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I think that a solution to the transactional
> >>>> writes
> >>>>>>>> should
> >>>>>>>>>>>> address
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> OOMEs. One possible way to address that is to
> >>> wire
> >>>>>>>>>> StateStore's
> >>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>> request by adding, say, StateStore#commitNeeded
> >>>> that
> >>>>>> is
> >>>>>>>>>> checked
> >>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>> StreamTask#commitNeeded via the corresponding
> >>>>>>>>>>>>> ProcessorStateManager.
> >>>>>>>>>>>>>>>> With
> >>>>>>>>>>>>>>>>>> that change, RocksDBStore will have to track
> >>> the
> >>>>>> current
> >>>>>>>>>>>>> transaction
> >>>>>>>>>>>>>>>> size
> >>>>>>>>>>>>>>>>>> and request a commit when the size goes over a
> >>>>>>>> (configurable)
> >>>>>>>>>>>>>>> threshold.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> AFAIU WriteBatchWithIndex might perform
> >>>>> significantly
> >>>>>>>> slower
> >>>>>>>>>>> than
> >>>>>>>>>>>>>>>> non-txn
> >>>>>>>>>>>>>>>>>> puts as the batch size grows [1]. We should
> >>> have a
> >>>>>>>>>>> configuration
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>> fall
> >>>>>>>>>>>>>>>>>> back to the current behavior (and/or disable
> >>> txn
> >>>>>> stores
> >>>>>>>> for
> >>>>>>>>>>> ALOS)
> >>>>>>>>>>>>>>> unless
> >>>>>>>>>>>>>>>>>> the benchmarks show negligible overhead for
> >>> longer
> >>>>>>>> commits /
> >>>>>>>>>>>>>>>> large-enough
> >>>>>>>>>>>>>>>>>> batch sizes.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> If you prefer to keep the KIP smaller, I would
> >>>>> rather
> >>>>>>>> cut out
> >>>>>>>>>>>>>>>>>> state-store-managed checkpointing rather than
> >>>> proper
> >>>>>>>> OOMe
> >>>>>>>>>>>> handling
> >>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> being able to switch to non-txn behavior. The
> >>>>>>>> checkpointing
> >>>>>>>>>> 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 configuration parts as well as review
> >>> and
> >>>>>> test
> >>>>>>>> the
> >>>>>>>>>>>> patch.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Best,
> >>>>>>>>>>>>>>>>>> Alex
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> 1.
> >>> https://github.com/facebook/rocksdb/issues/608
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
> >>>>>>>>>>>>> nick.telford@gmail.com
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Hi John,
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks for the review and feedback!
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 1. Custom Stores: I've been mulling over this
> >>>>>> problem
> >>>>>>>>>> myself.
> >>>>>>>>>>>> As
> >>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>> stands,
> >>>>>>>>>>>>>>>>>>> custom stores would essentially lose
> >>>> checkpointing
> >>>>>>>> with no
> >>>>>>>>>>>>>>> indication
> >>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> they're expected to make changes, besides a
> >>> line
> >>>>> in
> >>>>>>>> the
> >>>>>>>>>>> release
> >>>>>>>>>>>>>>>> notes. I
> >>>>>>>>>>>>>>>>>>> agree that the best solution would be to
> >>>> provide a
> >>>>>>>> default
> >>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> checkpoints
> >>>>>>>>>>>>>>>>>>> to a file. The one thing I would change is
> >>> that
> >>>>> the
> >>>>>>>>>>>> checkpointing
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>> store-local file, instead of a per-Task file.
> >>>> This
> >>>>>>>> way the
> >>>>>>>>>>>>>>> StateStore
> >>>>>>>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>>> technically owns its own checkpointing (via a
> >>>>>> default
> >>>>>>>>>>>>>>> implementation),
> >>>>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> the StateManager/Task execution engine
> >>> doesn't
> >>>>> need
> >>>>>>>> to know
> >>>>>>>>>>>>>>> anything
> >>>>>>>>>>>>>>>>>> about
> >>>>>>>>>>>>>>>>>>> checkpointing, which greatly simplifies some
> >>> of
> >>>>> the
> >>>>>>>> logic.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 2. OOME errors: The main reasons why I didn't
> >>>>>> explore
> >>>>>>>> a
> >>>>>>>>>>>> solution
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> this is
> >>>>>>>>>>>>>>>>>>> a) to keep this KIP as simple as possible,
> >>> and
> >>>> b)
> >>>>>>>> because
> >>>>>>>>>> I'm
> >>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>> exactly
> >>>>>>>>>>>>>>>>>>> how to signal that a Task should commit
> >>>>> prematurely.
> >>>>>>>> I'm
> >>>>>>>>>>>>> confident
> >>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>> possible, and I think it's worth adding a
> >>>> section
> >>>>> on
> >>>>>>>>>> handling
> >>>>>>>>>>>>> this.
> >>>>>>>>>>>>>>>>>> Besides
> >>>>>>>>>>>>>>>>>>> my proposal to force an early commit once
> >>> memory
> >>>>>> usage
> >>>>>>>>>>> reaches
> >>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>> threshold,
> >>>>>>>>>>>>>>>>>>> is there any other approach that you might
> >>>> suggest
> >>>>>> for
> >>>>>>>>>>> tackling
> >>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> problem?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 3. ALOS: I can add in an explicit paragraph,
> >>> but
> >>>>> my
> >>>>>>>>>>> assumption
> >>>>>>>>>>>> is
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>> since transactional behaviour comes at
> >>> little/no
> >>>>>>>> cost, that
> >>>>>>>>>>> it
> >>>>>>>>>>>>>>> should
> >>>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>> available by default on all stores,
> >>> irrespective
> >>>>> of
> >>>>>>>> the
> >>>>>>>>>>>>> processing
> >>>>>>>>>>>>>>>> mode.
> >>>>>>>>>>>>>>>>>>> While ALOS doesn't use transactions, the Task
> >>>>> itself
> >>>>>>>> still
> >>>>>>>>>>>>>>> "commits",
> >>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>> the behaviour should be correct under ALOS
> >>> too.
> >>>>> I'm
> >>>>>>>> not
> >>>>>>>>>>>> convinced
> >>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>>> worth having both
> >>>> transactional/non-transactional
> >>>>>>>> stores
> >>>>>>>>>>>>>>> available, as
> >>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>> would considerably increase the complexity of
> >>>> the
> >>>>>>>> codebase,
> >>>>>>>>>>> for
> >>>>>>>>>>>>>>> very
> >>>>>>>>>>>>>>>>>> little
> >>>>>>>>>>>>>>>>>>> benefit.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> 4. Method deprecation: Are you referring to
> >>>>>>>>>>>>>>> StateStore#getPosition()?
> >>>>>>>>>>>>>>>>>> As I
> >>>>>>>>>>>>>>>>>>> understand it, Position contains the
> >>> position of
> >>>>> the
> >>>>>>>>>> *source*
> >>>>>>>>>>>>>>> topics,
> >>>>>>>>>>>>>>>>>>> whereas the commit offsets would be the
> >>>>> *changelog*
> >>>>>>>>>> offsets.
> >>>>>>>>>>> So
> >>>>>>>>>>>>>>> it's
> >>>>>>>>>>>>>>>>>> still
> >>>>>>>>>>>>>>>>>>> necessary to retain the Position data, as
> >>> well
> >>>> as
> >>>>>> the
> >>>>>>>>>>> changelog
> >>>>>>>>>>>>>>>> offsets.
> >>>>>>>>>>>>>>>>>>> What I meant in the KIP is that Position
> >>> offsets
> >>>>> are
> >>>>>>>>>>> currently
> >>>>>>>>>>>>>>> stored
> >>>>>>>>>>>>>>>>>> in a
> >>>>>>>>>>>>>>>>>>> file, and since we can atomically store
> >>> metadata
> >>>>>>>> along with
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>> record
> >>>>>>>>>>>>>>>>>>> batch we commit to RocksDB, we can move our
> >>>>> Position
> >>>>>>>>>> offsets
> >>>>>>>>>>> in
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>> metadata too, and gain the same transactional
> >>>>>>>> guarantees
> >>>>>>>>>> that
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>>> for
> >>>>>>>>>>>>>>>>>>> changelog offsets, ensuring that the Position
> >>>>>> offsets
> >>>>>>>> are
> >>>>>>>>>>>>>>> consistent
> >>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>> the records that are read from the database.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On Tue, 22 Nov 2022 at 16:25, John Roesler <
> >>>>>>>>>>>> vvcephei@apache.org>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks for publishing this alternative,
> >>> Nick!
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> The benchmark you mentioned in the KIP-844
> >>>>>>>> discussion
> >>>>>>>>>> seems
> >>>>>>>>>>>>> like
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>>>>> compelling reason to revisit the built-in
> >>>>>>>>>> transactionality
> >>>>>>>>>>>>>>>> mechanism.
> >>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>> also appreciate you analysis, showing that
> >>> for
> >>>>>> most
> >>>>>>>> use
> >>>>>>>>>>>> cases,
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> write
> >>>>>>>>>>>>>>>>>>>> batch approach should be just fine.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> There are a couple of points that would
> >>> hold
> >>>> me
> >>>>>>>> back from
> >>>>>>>>>>>>>>> approving
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>> KIP right now:
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 1. Loss of coverage for custom stores.
> >>>>>>>>>>>>>>>>>>>> The fact that you can plug in a
> >>> (relatively)
> >>>>>> simple
> >>>>>>>>>>>>>>> implementation
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> XStateStore interfaces and automagically
> >>> get a
> >>>>>>>>>> distributed
> >>>>>>>>>>>>>>> database
> >>>>>>>>>>>>>>>>>> out
> >>>>>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> it is a significant benefit of Kafka
> >>> Streams.
> >>>>> I'd
> >>>>>>>> hate to
> >>>>>>>>>>>> lose
> >>>>>>>>>>>>>>> it,
> >>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>>>>>> would be better to spend some time and
> >>> come up
> >>>>>> with
> >>>>>>>> a way
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>>> preserve
> >>>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> property. For example, can we provide a
> >>>> default
> >>>>>>>>>>>> implementation
> >>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>>>>> `commit(..)` that re-implements the
> >>> existing
> >>>>>>>>>>> checkpoint-file
> >>>>>>>>>>>>>>>>>> approach? Or
> >>>>>>>>>>>>>>>>>>>> perhaps add an `isTransactional()` flag to
> >>> the
> >>>>>> state
> >>>>>>>>>> store
> >>>>>>>>>>>>>>> interface
> >>>>>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>>>> that the runtime can decide whether to
> >>>> continue
> >>>>> to
> >>>>>>>> manage
> >>>>>>>>>>>>>>> checkpoint
> >>>>>>>>>>>>>>>>>>> files
> >>>>>>>>>>>>>>>>>>>> vs delegating transactionality to the
> >>> stores?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 2. Guarding against OOME
> >>>>>>>>>>>>>>>>>>>> I appreciate your analysis, but I don't
> >>> think
> >>>>> it's
> >>>>>>>>>>> sufficient
> >>>>>>>>>>>>> to
> >>>>>>>>>>>>>>> say
> >>>>>>>>>>>>>>>>>> that
> >>>>>>>>>>>>>>>>>>>> we will solve the memory problem later if
> >>> it
> >>>>>> becomes
> >>>>>>>>>>>> necessary.
> >>>>>>>>>>>>>>> The
> >>>>>>>>>>>>>>>>>>>> experience leading to that situation would
> >>> be
> >>>>>> quite
> >>>>>>>> bad:
> >>>>>>>>>>>>> Imagine,
> >>>>>>>>>>>>>>>> you
> >>>>>>>>>>>>>>>>>>>> upgrade to AK 3.next, your tests pass, so
> >>> you
> >>>>>>>> deploy to
> >>>>>>>>>>>>>>> production.
> >>>>>>>>>>>>>>>>>> That
> >>>>>>>>>>>>>>>>>>>> night, you get paged because your app is
> >>> now
> >>>>>>>> crashing
> >>>>>>>>>> with
> >>>>>>>>>>>>>>> OOMEs. As
> >>>>>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>>> all OOMEs, you'll have a really hard time
> >>>>> finding
> >>>>>>>> the
> >>>>>>>>>> root
> >>>>>>>>>>>>> cause,
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>>> once
> >>>>>>>>>>>>>>>>>>>> you do, you won't have a clear path to
> >>> resolve
> >>>>> the
> >>>>>>>> issue.
> >>>>>>>>>>> You
> >>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>> only
> >>>>>>>>>>>>>>>>>>>> tune down the commit interval and cache
> >>> buffer
> >>>>>> size
> >>>>>>>> until
> >>>>>>>>>>> you
> >>>>>>>>>>>>>>> stop
> >>>>>>>>>>>>>>>>>>> getting
> >>>>>>>>>>>>>>>>>>>> crashes.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> FYI, I know of multiple cases where people
> >>> run
> >>>>> EOS
> >>>>>>>> with
> >>>>>>>>>>> much
> >>>>>>>>>>>>>>> larger
> >>>>>>>>>>>>>>>>>>> commit
> >>>>>>>>>>>>>>>>>>>> intervals to get better batching than the
> >>>>> default,
> >>>>>>>> so I
> >>>>>>>>>>> don't
> >>>>>>>>>>>>>>> think
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>> pathological case would be as rare as you
> >>>>> suspect.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Given that we already have the rudiments
> >>> of an
> >>>>>> idea
> >>>>>>>> of
> >>>>>>>>>> what
> >>>>>>>>>>>> we
> >>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>> do
> >>>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>> prevent this downside, we should take the
> >>> time
> >>>>> to
> >>>>>>>> design
> >>>>>>>>>> a
> >>>>>>>>>>>>>>> solution.
> >>>>>>>>>>>>>>>>>> We
> >>>>>>>>>>>>>>>>>>> owe
> >>>>>>>>>>>>>>>>>>>> it to our users to ensure that awesome new
> >>>>>> features
> >>>>>>>> don't
> >>>>>>>>>>>> come
> >>>>>>>>>>>>>>> with
> >>>>>>>>>>>>>>>>>>> bitter
> >>>>>>>>>>>>>>>>>>>> pills unless we can't avoid it.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 3. ALOS mode.
> >>>>>>>>>>>>>>>>>>>> On the other hand, I didn't see an
> >>> indication
> >>>> of
> >>>>>> how
> >>>>>>>>>> stores
> >>>>>>>>>>>>> will
> >>>>>>>>>>>>>>> be
> >>>>>>>>>>>>>>>>>>>> handled under ALOS (aka non-EOS) mode.
> >>>>>>>> Theoretically, the
> >>>>>>>>>>>>>>>>>>> transactionality
> >>>>>>>>>>>>>>>>>>>> of the store and the processing mode are
> >>>>>>>> orthogonal. A
> >>>>>>>>>>>>>>> transactional
> >>>>>>>>>>>>>>>>>>> store
> >>>>>>>>>>>>>>>>>>>> would serve ALOS just as well as a
> >>>>>>>> non-transactional one
> >>>>>>>>>>> (if
> >>>>>>>>>>>>> not
> >>>>>>>>>>>>>>>>>> better).
> >>>>>>>>>>>>>>>>>>>> Under ALOS, though, the default commit
> >>>> interval
> >>>>> is
> >>>>>>>> five
> >>>>>>>>>>>>> minutes,
> >>>>>>>>>>>>>>> so
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> memory issue is far more pressing.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> As I see it, we have several options to
> >>>> resolve
> >>>>>> this
> >>>>>>>>>> point.
> >>>>>>>>>>>> We
> >>>>>>>>>>>>>>> could
> >>>>>>>>>>>>>>>>>>>> demonstrate that transactional stores work
> >>>> just
> >>>>>>>> fine for
> >>>>>>>>>>> ALOS
> >>>>>>>>>>>>>>> and we
> >>>>>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>>> therefore just swap over unconditionally.
> >>> We
> >>>>> could
> >>>>>>>> also
> >>>>>>>>>>>> disable
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> transactional mechanism under ALOS so that
> >>>>> stores
> >>>>>>>> operate
> >>>>>>>>>>>> just
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>> they do today when run in ALOS mode.
> >>> Finally,
> >>>> we
> >>>>>>>> could do
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>> same
> >>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>> KIP-844 and make transactional stores
> >>> opt-in
> >>>>> (it'd
> >>>>>>>> be
> >>>>>>>>>>> better
> >>>>>>>>>>>> to
> >>>>>>>>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> extra opt-in mechanism, but it's a good
> >>>>>>>>>>> get-out-of-jail-free
> >>>>>>>>>>>>>>> card).
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> 4. (minor point) Deprecation of methods
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> You mentioned that the new `commit` 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?
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Thanks again for the KIP! It's really nice
> >>>> that
> >>>>>> you
> >>>>>>>> and
> >>>>>>>>>>> Alex
> >>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>> get
> >>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>> chance to collaborate on both directions so
> >>>> that
> >>>>>> we
> >>>>>>>> can
> >>>>>>>>>> get
> >>>>>>>>>>>> the
> >>>>>>>>>>>>>>> best
> >>>>>>>>>>>>>>>>>>>> outcome for Streams and its users.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> -John
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> On 2022/11/21 15:02:15 Nick Telford wrote:
> >>>>>>>>>>>>>>>>>>>>> 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
> >>>>>>>>>>>>>>>>>>>>> <
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >>>>>>>>>>>>>>>>>>>>> ,
> >>>>>>>>>>>>>>>>>>>>> so that it can be discussed/reviewed
> >>>>> separately
> >>>>>>>> from
> >>>>>>>>>>>> KIP-844.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Alex: I'm especially interested in what
> >>> you
> >>>>>> think!
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> I have a nearly complete implementation
> >>> of
> >>>> the
> >>>>>>>> changes
> >>>>>>>>>>>>>>> outlined in
> >>>>>>>>>>>>>>>>>> this
> >>>>>>>>>>>>>>>>>>>>> KIP, please let me know if you'd like me
> >>> to
> >>>>> push
> >>>>>>>> them
> >>>>>>>>>> for
> >>>>>>>>>>>>>>> review
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>>>>> advance
> >>>>>>>>>>>>>>>>>>>>> of a vote.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Regards,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Nick
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>
> >>
> >
>

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

Posted by Bruno Cadonna <ca...@apache.org>.
Hi Nick,

Thanks for the updates!

I have a couple of questions/comments.

1.
Why do you propose a configuration that involves max. bytes and max. 
reords? I think we are mainly concerned about memory consumption because 
we want to limit the off-heap memory used. I cannot think of a case 
where one would want to set the max. number of records.


2.
Why does

  default void commit(final Map<TopicPartition, Long> changelogOffsets) {
      flush();
  }

take a map of partitions to changelog offsets?
The mapping between state stores to partitions is a 1:1 relationship. 
Passing in a single changelog offset should suffice.


3.
Why do we need the Transaction interface? It should be possible to hide 
beginning and committing a transactions withing the state store 
implementation, so that from outside the state store, it does not matter 
whether the state store is transactional or not. What would be the 
advantage of using the Transaction interface?


4.
Regarding checkpointing offsets, I think we should keep the checkpoint 
file in any case for the reason you mentioned about rebalancing. Even if 
that would not be an issue, I would propose to move the change to offset 
management to a new KIP and to not add more complexity than needed to 
this one. I would not be too concerned about the consistency violation 
you mention. As far as I understand, with transactional state stores 
Streams would write the checkpoint file during every commit even under 
EOS. In the failure case you describe, Streams would restore the state 
stores from the offsets found in the checkpoint file written during the 
penultimate commit instead of during the last commit. Basically, Streams 
would overwrite the records written to the state store between the last 
two commits with the same records read from the changelogs. While I 
understand that this is wasteful, it is -- at the same time -- 
acceptable and most importantly it does not break EOS.

Best,
Bruno


On 27.04.23 12:34, Nick Telford wrote:
> Hi everyone,
> 
> I find myself (again) considering removing the offset management from
> StateStores, and keeping the old checkpoint file system. The reason is that
> the StreamPartitionAssignor directly reads checkpoint files in order to
> determine which instance has the most up-to-date copy of the local state.
> If we move offsets into the StateStore itself, then we will need to open,
> initialize, read offsets and then close each StateStore (that is not
> already assigned and open) for which we have *any* local state, on every
> rebalance.
> 
> Generally, I don't think there are many "orphan" stores like this sitting
> around on most instances, but even a few would introduce additional latency
> to an already somewhat lengthy rebalance procedure.
> 
> I'm leaning towards Colt's (Slack) suggestion of just keeping things in the
> checkpoint file(s) for now, and not worrying about the race. The downside
> is that we wouldn't be able to remove the explicit RocksDB flush on-commit,
> which likely hurts performance.
> 
> If anyone has any thoughts or ideas on this subject, I would appreciate it!
> 
> Regards,
> Nick
> 
> On Wed, 19 Apr 2023 at 15:05, Nick Telford <ni...@gmail.com> wrote:
> 
>> Hi Colt,
>>
>> The issue is that if there's a crash between 2 and 3, then you still end
>> up with inconsistent data in RocksDB. The only way to guarantee that your
>> checkpoint offsets and locally stored data are consistent with each other
>> are to atomically commit them, which can be achieved by having the offsets
>> stored in RocksDB.
>>
>> The offsets column family is likely to be extremely small (one
>> per-changelog partition + one per Topology input partition for regular
>> stores, one per input partition for global stores). So the overhead will be
>> minimal.
>>
>> A major benefit of doing this is that we can remove the explicit calls to
>> db.flush(), which forcibly flushes memtables to disk on-commit. It turns
>> out, RocksDB memtable flushes are largely dictated by Kafka Streams
>> commits, *not* RocksDB configuration, which could be a major source of
>> confusion. Atomic checkpointing makes it safe to remove these explicit
>> flushes, because it no longer matters exactly when RocksDB flushes data to
>> disk; since the data and corresponding checkpoint offsets will always be
>> flushed together, the local store is always in a consistent state, and
>> on-restart, it can always safely resume restoration from the on-disk
>> offsets, restoring the small amount of data that hadn't been flushed when
>> the app exited/crashed.
>>
>> Regards,
>> Nick
>>
>> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io> wrote:
>>
>>> Nick,
>>>
>>> Thanks for your reply. Ack to A) and B).
>>>
>>> For item C), I see what you're referring to. Your proposed solution will
>>> work, so no need to change it. What I was suggesting was that it might be
>>> possible to achieve this with only one column family. So long as:
>>>
>>>     - No uncommitted records (i.e. not committed to the changelog) are
>>>     *committed* to the state store, AND
>>>     - The Checkpoint offset (which refers to the changelog topic) is less
>>>     than or equal to the last written changelog offset in rocksdb
>>>
>>> I don't see the need to do the full restoration from scratch. My
>>> understanding was that prior to 844/892, full restorations were required
>>> because there could be uncommitted records written to RocksDB; however,
>>> given your use of RocksDB transactions, that can be avoided with the
>>> pattern of 1) commit Kafka transaction, 2) commit RocksDB transaction, 3)
>>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <ni...@gmail.com>
>>> wrote:
>>>
>>>> Hi Colt,
>>>>
>>>> A. I've done my best to de-couple the StateStore stuff from the rest of
>>> the
>>>> Streams engine. The fact that there will be only one ongoing (write)
>>>> transaction at a time is not guaranteed by any API, and is just a
>>>> consequence of the way Streams operates. To that end, I tried to ensure
>>> the
>>>> documentation and guarantees provided by the new APIs are independent of
>>>> this incidental behaviour. In practice, you're right, this essentially
>>>> refers to "interactive queries", which are technically "read
>>> transactions",
>>>> even if they don't actually use the transaction API to isolate
>>> themselves.
>>>>
>>>> B. Yes, although not ideal. This is for backwards compatibility,
>>> because:
>>>>      1) Existing custom StateStore implementations will implement
>>> flush(),
>>>> and not commit(), but the Streams engine now calls commit(), so those
>>> calls
>>>> need to be forwarded to flush() for these legacy stores.
>>>>      2) Existing StateStore *users*, i.e. outside of the Streams engine
>>>> itself, may depend on explicitly calling flush(), so for these cases,
>>>> flush() needs to be redirected to call commit().
>>>> If anyone has a better way to guarantee compatibility without
>>> introducing
>>>> this potential recursion loop, I'm open to changes!
>>>>
>>>> C. This is described in the "Atomic Checkpointing" section. Offsets are
>>>> stored in a separate RocksDB column family, which is guaranteed to be
>>>> atomically flushed to disk with all other column families. The issue of
>>>> checkpoints being written to disk after commit causing inconsistency if
>>> it
>>>> crashes in between is the reason why, under EOS, checkpoint files are
>>> only
>>>> written on clean shutdown. This is one of the major causes of "full
>>>> restorations", so moving the offsets into a place where they can be
>>>> guaranteed to be atomically written with the data they checkpoint
>>> allows us
>>>> to write the checkpoint offsets *on every commit*, not just on clean
>>>> shutdown.
>>>>
>>>> Regards,
>>>> Nick
>>>>
>>>> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io> wrote:
>>>>
>>>>> Nick,
>>>>>
>>>>> Thank you for continuing this work. I have a few minor clarifying
>>>>> questions.
>>>>>
>>>>> A) "Records written to any transaction are visible to all other
>>>>> transactions immediately." I am confused here—I thought there could
>>> only
>>>> be
>>>>> one transaction going on at a time for a given state store given the
>>>>> threading model for processing records on a Task. Do you mean
>>> Interactive
>>>>> Queries by "other transactions"? (If so, then everything makes sense—I
>>>>> thought that since IQ were read-only then they didn't count as
>>>>> transactions).
>>>>>
>>>>> B) Is it intentional that the default implementations of the flush()
>>> and
>>>>> commit() methods in the StateStore class refer to each other in some
>>> sort
>>>>> of unbounded recursion?
>>>>>
>>>>> C) How will the getCommittedOffset() method work? At first I thought
>>> the
>>>>> way to do it would be using a special key in the RocksDB store to
>>> store
>>>> the
>>>>> offset, and committing that with the transaction. But upon second
>>>> thought,
>>>>> since restoration from the changelog is an idempotent procedure, I
>>> think
>>>> it
>>>>> would be fine to 1) commit the RocksDB transaction and then 2) write
>>> the
>>>>> offset to disk in a checkpoint file. If there is a crash between 1)
>>> and
>>>> 2),
>>>>> I think the only downside is 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 <ni...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hi everyone,
>>>>>>
>>>>>> I've updated the KIP to reflect the latest version of the design:
>>>>>>
>>>>>>
>>>>>
>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>
>>>>>> There are several changes in there that reflect feedback from this
>>>>> thread,
>>>>>> and there's a new section and a bunch of interface changes relating
>>> to
>>>>>> Atomic Checkpointing, which is the final piece of the puzzle to
>>> making
>>>>>> everything robust.
>>>>>>
>>>>>> Let me know what you think!
>>>>>>
>>>>>> Regards,
>>>>>> Nick
>>>>>>
>>>>>> On Tue, 3 Jan 2023 at 11:33, Nick Telford <ni...@gmail.com>
>>>>> wrote:
>>>>>>
>>>>>>> Hi Lucas,
>>>>>>>
>>>>>>> Thanks for looking over my KIP.
>>>>>>>
>>>>>>> A) The bound is per-instance, not per-Task. This was a typo in the
>>>> KIP
>>>>>>> that I've now corrected. It was originally per-Task, but I
>>> changed it
>>>>> to
>>>>>>> per-instance for exactly the reason you highlighted.
>>>>>>> B) It's worth noting that transactionality is only enabled under
>>> EOS,
>>>>> and
>>>>>>> in the default mode of operation (ALOS), there should be no
>>> change in
>>>>>>> behavior at all. I think, under EOS, we can mitigate the impact on
>>>>> users
>>>>>> by
>>>>>>> sufficiently low default values for the memory bound
>>> configuration. I
>>>>>>> understand your hesitation to include a significant change of
>>>>> behaviour,
>>>>>>> especially in a minor release, but I suspect that most users will
>>>>> prefer
>>>>>>> the memory impact (under EOS) to the existing behaviour of
>>> frequent
>>>>> state
>>>>>>> restorations! If this is a problem, the changes can wait until the
>>>> next
>>>>>>> major release. I'll be running a patched version of streams in
>>>>> production
>>>>>>> with these changes as soon as they're ready, so it won't disrupt
>>> me
>>>> :-D
>>>>>>> C) The main purpose of this sentence was just to note that some
>>>> changes
>>>>>>> will need to be made to the way Segments are handled in order to
>>>> ensure
>>>>>>> they also benefit from transactions. At the time I wrote it, I
>>> hadn't
>>>>>>> figured out the specific changes necessary, so it was deliberately
>>>>> vague.
>>>>>>> This is the one outstanding problem I'm currently working on, and
>>>> I'll
>>>>>>> update this section with more detail once I have figured out the
>>>> exact
>>>>>>> changes required.
>>>>>>> D) newTransaction() provides the necessary isolation guarantees.
>>>> While
>>>>>>> the RocksDB implementation of transactions doesn't technically
>>> *need*
>>>>>>> read-only users to call newTransaction(), other implementations
>>>> (e.g. a
>>>>>>> hypothetical PostgresStore) may require it. Calling
>>> newTransaction()
>>>>> when
>>>>>>> no transaction is necessary is essentially free, as it will just
>>>> return
>>>>>>> this.
>>>>>>>
>>>>>>> I didn't do any profiling of the KIP-844 PoC, but I think it
>>> should
>>>> be
>>>>>>> fairly obvious where the performance problems stem from: writes
>>> under
>>>>>>> KIP-844 require 3 extra memory-copies: 1 to encode it with the
>>>>>>> tombstone/record flag, 1 to decode it from the tombstone/record
>>> flag,
>>>>>> and 1
>>>>>>> to copy the record from the "temporary" store to the "main" store,
>>>> when
>>>>>> the
>>>>>>> transaction commits. The different approach taken by KIP-869
>>> should
>>>>>> perform
>>>>>>> much better, as it avoids all these copies, and may actually
>>> perform
>>>>>>> slightly better than trunk, due to batched writes in RocksDB
>>>> performing
>>>>>>> better than non-batched writes.[1]
>>>>>>>
>>>>>>> Regards,
>>>>>>> Nick
>>>>>>>
>>>>>>> 1:
>>>>>>
>>>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>
>>>>>>> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
>>> lbrutschy@confluent.io
>>>>>> .invalid>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Nick,
>>>>>>>>
>>>>>>>> I'm just starting to read up on the whole discussion about
>>> KIP-892
>>>> and
>>>>>>>> KIP-844. Thanks a lot for your work on this, I do think
>>>>>>>> `WriteBatchWithIndex` may be the way to go here. I do have some
>>>>>>>> questions about the latest draft.
>>>>>>>>
>>>>>>>>   A) If I understand correctly, you propose to put a bound on the
>>>>>>>> (native) memory consumed by each task. However, I wonder if this
>>> is
>>>>>>>> sufficient if we have temporary imbalances in the cluster. For
>>>>>>>> example, depending on the timing of rebalances during a cluster
>>>>>>>> restart, it could happen that a single streams node is assigned a
>>>> lot
>>>>>>>> more tasks than expected. With your proposed change, this would
>>> mean
>>>>>>>> that the memory required by this one node could be a multiple of
>>>> what
>>>>>>>> is required during normal operation. I wonder if it wouldn't be
>>>> safer
>>>>>>>> to put a global bound on the memory use, across all tasks.
>>>>>>>>   B) Generally, the memory concerns still give me the feeling that
>>>> this
>>>>>>>> should not be enabled by default for all users in a minor
>>> release.
>>>>>>>>   C) In section "Transaction Management": the sentence "A similar
>>>>>>>> analogue will be created to automatically manage `Segment`
>>>>>>>> transactions.". Maybe this is just me lacking some background,
>>> 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 you to find out why the KIP-844 PoC is 20x slower
>>> (e.g.
>>>>>>>> by providing a flame graph), that would be quite interesting.
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Lucas
>>>>>>>>
>>>>>>>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
>>>> nick.telford@gmail.com>
>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> Hi everyone,
>>>>>>>>>
>>>>>>>>> I've updated the KIP with a more detailed design, which
>>> reflects
>>>> the
>>>>>>>>> implementation I've been working on:
>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>
>>>>>>>>> This new design should address the outstanding points already
>>> made
>>>>> in
>>>>>>>> the
>>>>>>>>> thread.
>>>>>>>>>
>>>>>>>>> Please let me know if there are areas that are unclear or need
>>>> more
>>>>>>>>> clarification.
>>>>>>>>>
>>>>>>>>> I have a (nearly) working implementation. I'm confident that
>>> the
>>>>>>>> remaining
>>>>>>>>> work (making Segments behave) will not impact the documented
>>>> design.
>>>>>>>>>
>>>>>>>>> Regards,
>>>>>>>>>
>>>>>>>>> Nick
>>>>>>>>>
>>>>>>>>> On Tue, 6 Dec 2022 at 19:24, Colt McNealy <colt@littlehorse.io
>>>>
>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Nick,
>>>>>>>>>>
>>>>>>>>>> Thank you for the reply; that makes sense. I was hoping that,
>>>>> since
>>>>>>>> reading
>>>>>>>>>> uncommitted records from IQ in EOS isn't part of the
>>> documented
>>>>> API,
>>>>>>>> maybe
>>>>>>>>>> you *wouldn't* have to wait for the next major release to
>>> make
>>>>> that
>>>>>>>> change;
>>>>>>>>>> but given 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 <
>>>>>> nick.telford@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>
>>>>>>>>>>> 10: Yes, I agree it's not ideal. I originally intended to
>>> try
>>>> to
>>>>>>>> keep the
>>>>>>>>>>> behaviour unchanged as much as possible, otherwise we'd
>>> have
>>>> to
>>>>>>>> wait for
>>>>>>>>>> a
>>>>>>>>>>> major version release to land these changes.
>>>>>>>>>>> 20: Good point, ALOS doesn't need the same level of
>>> guarantee,
>>>>> and
>>>>>>>> the
>>>>>>>>>>> typically longer commit intervals would be problematic when
>>>>>> reading
>>>>>>>> only
>>>>>>>>>>> "committed" records.
>>>>>>>>>>>
>>>>>>>>>>> I've been away for 5 days recovering from minor surgery,
>>> but I
>>>>>>>> spent a
>>>>>>>>>>> considerable amount of that time working through ideas for
>>>>>> possible
>>>>>>>>>>> solutions in my head. I think your suggestion of keeping
>>> ALOS
>>>>>>>> as-is, but
>>>>>>>>>>> buffering writes for EOS is the right path forwards,
>>> although
>>>> I
>>>>>>>> have a
>>>>>>>>>>> solution that both expands on this, and provides for some
>>> more
>>>>>>>> formal
>>>>>>>>>>> guarantees.
>>>>>>>>>>>
>>>>>>>>>>> Essentially, adding support to KeyValueStores for
>>>>> "Transactions",
>>>>>>>> with
>>>>>>>>>>> clearly defined IsolationLevels. Using "Read Committed"
>>> when
>>>>> under
>>>>>>>> EOS,
>>>>>>>>>> and
>>>>>>>>>>> "Read Uncommitted" under ALOS.
>>>>>>>>>>>
>>>>>>>>>>> The nice thing about this approach is that it gives us much
>>>> more
>>>>>>>> clearly
>>>>>>>>>>> defined isolation behaviour that can be properly
>>> documented to
>>>>>>>> ensure
>>>>>>>>>> users
>>>>>>>>>>> know what to expect.
>>>>>>>>>>>
>>>>>>>>>>> I'm still working out the kinks in the design, and will
>>> update
>>>>> the
>>>>>>>> KIP
>>>>>>>>>> when
>>>>>>>>>>> I have something. The main struggle is trying to implement
>>>> this
>>>>>>>> without
>>>>>>>>>>> making any major changes to the existing interfaces or
>>>> breaking
>>>>>>>> existing
>>>>>>>>>>> implementations, because currently everything expects to
>>>> operate
>>>>>>>> directly
>>>>>>>>>>> on a StateStore, and not a Transaction of that store. I
>>> think
>>>>> I'm
>>>>>>>> getting
>>>>>>>>>>> close, although sadly I won't be able to progress much
>>> until
>>>>> next
>>>>>>>> week
>>>>>>>>>> due
>>>>>>>>>>> to some work commitments.
>>>>>>>>>>>
>>>>>>>>>>> Regards,
>>>>>>>>>>> Nick
>>>>>>>>>>>
>>>>>>>>>>> On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
>>>> colt@littlehorse.io>
>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Nick,
>>>>>>>>>>>>
>>>>>>>>>>>> Thank you for the explanation, and also for the updated
>>>> KIP. I
>>>>>> am
>>>>>>>> quite
>>>>>>>>>>>> eager for this improvement to be released as it would
>>>> greatly
>>>>>>>> reduce
>>>>>>>>>> the
>>>>>>>>>>>> operational difficulties of EOS streams apps.
>>>>>>>>>>>>
>>>>>>>>>>>> Two questions:
>>>>>>>>>>>>
>>>>>>>>>>>> 10)
>>>>>>>>>>>>> When reading records, we will use the
>>>>>>>>>>>> WriteBatchWithIndex#getFromBatchAndDB
>>>>>>>>>>>>   and WriteBatchWithIndex#newIteratorWithBase utilities in
>>>>> order
>>>>>> to
>>>>>>>>>> ensure
>>>>>>>>>>>> that uncommitted writes are available to query.
>>>>>>>>>>>> Why do extra work to enable the reading of uncommitted
>>>> writes
>>>>>>>> during
>>>>>>>>>> IQ?
>>>>>>>>>>>> Code complexity aside, reading uncommitted writes is, in
>>> my
>>>>>>>> opinion, a
>>>>>>>>>>>> minor flaw in EOS IQ; it would be very nice to have the
>>>>>> guarantee
>>>>>>>> that,
>>>>>>>>>>>> with EOS, IQ only reads committed records. In order to
>>> avoid
>>>>>> dirty
>>>>>>>>>> reads,
>>>>>>>>>>>> one currently must query a standby replica (but this
>>> still
>>>>>> doesn't
>>>>>>>>>> fully
>>>>>>>>>>>> guarantee monotonic reads).
>>>>>>>>>>>>
>>>>>>>>>>>> 20) Is it also necessary to enable this optimization on
>>> ALOS
>>>>>>>> stores?
>>>>>>>>>> The
>>>>>>>>>>>> motivation of KIP-844 was mainly to reduce the need to
>>>> restore
>>>>>>>> state
>>>>>>>>>> from
>>>>>>>>>>>> scratch on unclean EOS shutdowns; with ALOS it was
>>>> acceptable
>>>>> to
>>>>>>>> accept
>>>>>>>>>>>> that there may have been uncommitted writes on disk. On a
>>>> side
>>>>>>>> note, if
>>>>>>>>>>> you
>>>>>>>>>>>> enable this type of store on ALOS processors, the
>>> community
>>>>>> would
>>>>>>>>>>>> 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 <
>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>
>>>>>>>>>>>>> I've drastically reduced the scope of this KIP to no
>>>> longer
>>>>>>>> include
>>>>>>>>>> the
>>>>>>>>>>>>> StateStore management of checkpointing. This can be
>>> added
>>>>> as a
>>>>>>>> KIP
>>>>>>>>>>> later
>>>>>>>>>>>> on
>>>>>>>>>>>>> to further optimize the consistency and performance of
>>>> state
>>>>>>>> stores.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I've also added a section discussing some of the
>>> concerns
>>>>>> 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 2022 at 16:36, Nick Telford <
>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hi Colt,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I didn't do any profiling, but the 844
>>> implementation:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>     - Writes uncommitted records to a temporary
>>> RocksDB
>>>>>>>> instance
>>>>>>>>>>>>>>        - Since tombstones need to be flagged, all
>>> record
>>>>>>>> values are
>>>>>>>>>>>>>>        prefixed with a value/tombstone marker. This
>>>>>>>> necessitates a
>>>>>>>>>>>> memory
>>>>>>>>>>>>> copy.
>>>>>>>>>>>>>>     - On-commit, iterates all records in this
>>> temporary
>>>>>>>> instance and
>>>>>>>>>>>>>>     writes them to the main RocksDB store.
>>>>>>>>>>>>>>     - While iterating, the value/tombstone marker
>>> needs
>>>> to
>>>>> be
>>>>>>>> parsed
>>>>>>>>>>> and
>>>>>>>>>>>>>>     the real value extracted. This necessitates
>>> another
>>>>>> memory
>>>>>>>> copy.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> My guess is that the cost of iterating the temporary
>>>>> RocksDB
>>>>>>>> store
>>>>>>>>>> is
>>>>>>>>>>>> the
>>>>>>>>>>>>>> major factor, with the 2 extra memory copies
>>> per-Record
>>>>>>>>>> contributing
>>>>>>>>>>> a
>>>>>>>>>>>>>> significant amount too.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
>>>>>>>> colt@littlehorse.io>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi all,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Out of curiosity, why does the performance of the
>>> store
>>>>>>>> degrade so
>>>>>>>>>>>>>>> significantly with the 844 implementation? I
>>> wouldn't
>>>> be
>>>>>> too
>>>>>>>>>>> surprised
>>>>>>>>>>>>> by
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>> 50-60% drop (caused by each record being written
>>>> twice),
>>>>>> but
>>>>>>>> 96%
>>>>>>>>>> is
>>>>>>>>>>>>>>> extreme.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The only thing I can think of which could create
>>> such a
>>>>>>>> bottleneck
>>>>>>>>>>>> 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 Nick Telford <
>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hi everyone,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I've updated the KIP to resolve all the points
>>> that
>>>>> have
>>>>>>>> been
>>>>>>>>>>> raised
>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>> far, with one exception: the ALOS default commit
>>>>> interval
>>>>>>>> of 5
>>>>>>>>>>>> minutes
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> likely to cause WriteBatchWithIndex memory to grow
>>>> too
>>>>>>>> large.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> There's a couple of different things I can think
>>> of
>>>> to
>>>>>>>> solve
>>>>>>>>>> this:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>     - We already have a memory/record limit in the
>>> KIP
>>>>> to
>>>>>>>> prevent
>>>>>>>>>>> OOM
>>>>>>>>>>>>>>>>     errors. Should we choose a default value for
>>>> these?
>>>>> My
>>>>>>>>>> concern
>>>>>>>>>>>> here
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>     anything we choose might seem rather
>>> arbitrary. We
>>>>>> could
>>>>>>>>>> change
>>>>>>>>>>>>>>>>     its behaviour such that under ALOS, it only
>>>> triggers
>>>>>> the
>>>>>>>>>> commit
>>>>>>>>>>>> of
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>     StateStore, but under EOS, it triggers a
>>> commit of
>>>>> the
>>>>>>>> Kafka
>>>>>>>>>>>>>>>> transaction.
>>>>>>>>>>>>>>>>     - We could introduce a separate `
>>>>>> checkpoint.interval.ms`
>>>>>>>> to
>>>>>>>>>>>> allow
>>>>>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>>>     to commit the StateStores more frequently than
>>> the
>>>>>>>> general
>>>>>>>>>>>>>>>>     commit.interval.ms? My concern here is that
>>> the
>>>>>>>> semantics of
>>>>>>>>>>>> this
>>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>     would depend 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 <
>>>>>>>>>>> nick.telford@gmail.com>
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hi Alex,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks for the feedback.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I've updated the discussion of OOM issues by
>>>>> describing
>>>>>>>> how
>>>>>>>>>>> we'll
>>>>>>>>>>>>>>> handle
>>>>>>>>>>>>>>>>> it. Here's the new text:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> To mitigate this, we will automatically force a
>>>> Task
>>>>>>>> commit if
>>>>>>>>>>> the
>>>>>>>>>>>>>>> total
>>>>>>>>>>>>>>>>>> uncommitted records returned by
>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedEntries()
>>>>>> exceeds a
>>>>>>>>>>>> threshold,
>>>>>>>>>>>>>>>>>> configured by
>>>>> max.uncommitted.state.entries.per.task;
>>>>>>>> or the
>>>>>>>>>>>> total
>>>>>>>>>>>>>>>>>> memory used for buffering uncommitted records
>>>>> returned
>>>>>>>> by
>>>>>>>>>>>>>>>>>> StateStore#approximateNumUncommittedBytes()
>>>> exceeds
>>>>>> the
>>>>>>>>>>> threshold
>>>>>>>>>>>>>>>>>> configured by
>>>> max.uncommitted.state.bytes.per.task.
>>>>>>>> This will
>>>>>>>>>>>>> roughly
>>>>>>>>>>>>>>>>>> bound the memory required per-Task for
>>> buffering
>>>>>>>> uncommitted
>>>>>>>>>>>>> records,
>>>>>>>>>>>>>>>>>> irrespective of the commit.interval.ms, and
>>> will
>>>>>>>> effectively
>>>>>>>>>>>> bound
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> number of records that will need to be
>>> restored in
>>>>> the
>>>>>>>> event
>>>>>>>>>>> of a
>>>>>>>>>>>>>>>> failure.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> These limits will be checked in
>>> StreamTask#process
>>>>> and
>>>>>> a
>>>>>>>>>>> premature
>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>> will be requested via Task#requestCommit().
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Note that these new methods provide default
>>>>>>>> implementations
>>>>>>>>>> that
>>>>>>>>>>>>>>> ensure
>>>>>>>>>>>>>>>>>> existing custom stores and non-transactional
>>>> stores
>>>>>>>> (e.g.
>>>>>>>>>>>>>>>>>> InMemoryKeyValueStore) do not force any early
>>>>> commits.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I've chosen to have the StateStore expose
>>>>>> approximations
>>>>>>>> of
>>>>>>>>>> its
>>>>>>>>>>>>> buffer
>>>>>>>>>>>>>>>>> size/count instead of opaquely requesting a
>>> commit
>>>> in
>>>>>>>> order to
>>>>>>>>>>>>>>> delegate
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> decision making to the Task itself. This enables
>>>>> Tasks
>>>>>>>> to look
>>>>>>>>>>> at
>>>>>>>>>>>>>>> *all*
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> their StateStores, and determine whether an
>>> early
>>>>>> commit
>>>>>>>> is
>>>>>>>>>>>>> necessary.
>>>>>>>>>>>>>>>>> Notably, it enables pre-Task thresholds,
>>> instead of
>>>>>>>> per-Store,
>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>> prevents Tasks with many StateStores from using
>>>> much
>>>>>> more
>>>>>>>>>> memory
>>>>>>>>>>>>> than
>>>>>>>>>>>>>>>> Tasks
>>>>>>>>>>>>>>>>> with one StateStore. This makes sense, since
>>>> commits
>>>>>> are
>>>>>>>> done
>>>>>>>>>>>>> by-Task,
>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>> by-Store.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Prizes* for anyone who can come up with a better
>>>> name
>>>>>>>> for the
>>>>>>>>>>> new
>>>>>>>>>>>>>>> config
>>>>>>>>>>>>>>>>> properties!
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks for pointing out the potential
>>> performance
>>>>>> issues
>>>>>>>> of
>>>>>>>>>>> WBWI.
>>>>>>>>>>>>> From
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> benchmarks that user posted[1], it looks like
>>> WBWI
>>>>>> still
>>>>>>>>>>> performs
>>>>>>>>>>>>>>>>> considerably better than individual puts, which
>>> is
>>>>> the
>>>>>>>>>> existing
>>>>>>>>>>>>>>> design,
>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>> I'd actually expect a performance boost from
>>> WBWI,
>>>>> just
>>>>>>>> not as
>>>>>>>>>>>> great
>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>> we'd get from a plain WriteBatch. This does
>>> suggest
>>>>>> that
>>>>>>>> a
>>>>>>>>>> good
>>>>>>>>>>>>>>>>> optimization would be to use a regular
>>> WriteBatch
>>>> for
>>>>>>>>>>> restoration
>>>>>>>>>>>>> (in
>>>>>>>>>>>>>>>>> RocksDBStore#restoreBatch), since we know that
>>>> those
>>>>>>>> records
>>>>>>>>>>> will
>>>>>>>>>>>>>>> never
>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>> queried before they're committed.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> 1:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>
>>>>>
>>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> * Just kidding, no prizes, sadly.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, 23 Nov 2022 at 12:28, Alexander
>>> Sorokoumov
>>>>>>>>>>>>>>>>> <as...@confluent.io.invalid> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 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 to keep the current
>>>>> checkpointing
>>>>>>>> as a
>>>>>>>>>>>>> fallback
>>>>>>>>>>>>>>>>>> mechanism. We want to keep existing users'
>>>> workflows
>>>>>>>> intact
>>>>>>>>>> if
>>>>>>>>>>> we
>>>>>>>>>>>>>>> can. A
>>>>>>>>>>>>>>>>>> non-intrusive way would be to add a separate
>>>>>> StateStore
>>>>>>>>>> method,
>>>>>>>>>>>>> say,
>>>>>>>>>>>>>>>>>> StateStore#managesCheckpointing(), that
>>> controls
>>>>>>>> whether the
>>>>>>>>>>>> state
>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>> implementation owns checkpointing.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I think that a solution to the transactional
>>>> writes
>>>>>>>> should
>>>>>>>>>>>> address
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> OOMEs. One possible way to address that is to
>>> wire
>>>>>>>>>> StateStore's
>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>> request by adding, say, StateStore#commitNeeded
>>>> that
>>>>>> is
>>>>>>>>>> checked
>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> StreamTask#commitNeeded via the corresponding
>>>>>>>>>>>>> ProcessorStateManager.
>>>>>>>>>>>>>>>> With
>>>>>>>>>>>>>>>>>> that change, RocksDBStore will have to track
>>> the
>>>>>> current
>>>>>>>>>>>>> transaction
>>>>>>>>>>>>>>>> size
>>>>>>>>>>>>>>>>>> and request a commit when the size goes over a
>>>>>>>> (configurable)
>>>>>>>>>>>>>>> threshold.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> AFAIU WriteBatchWithIndex might perform
>>>>> significantly
>>>>>>>> slower
>>>>>>>>>>> than
>>>>>>>>>>>>>>>> non-txn
>>>>>>>>>>>>>>>>>> puts as the batch size grows [1]. We should
>>> have a
>>>>>>>>>>> configuration
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> fall
>>>>>>>>>>>>>>>>>> back to the current behavior (and/or disable
>>> txn
>>>>>> stores
>>>>>>>> for
>>>>>>>>>>> ALOS)
>>>>>>>>>>>>>>> unless
>>>>>>>>>>>>>>>>>> the benchmarks show negligible overhead for
>>> longer
>>>>>>>> commits /
>>>>>>>>>>>>>>>> large-enough
>>>>>>>>>>>>>>>>>> batch sizes.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> If you prefer to keep the KIP smaller, I would
>>>>> rather
>>>>>>>> cut out
>>>>>>>>>>>>>>>>>> state-store-managed checkpointing rather than
>>>> proper
>>>>>>>> OOMe
>>>>>>>>>>>> handling
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> being able to switch to non-txn behavior. The
>>>>>>>> checkpointing
>>>>>>>>>> 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 configuration parts as well as review
>>> and
>>>>>> test
>>>>>>>> the
>>>>>>>>>>>> patch.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>> Alex
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> 1.
>>> https://github.com/facebook/rocksdb/issues/608
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
>>>>>>>>>>>>> nick.telford@gmail.com
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hi John,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks for the review and feedback!
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 1. Custom Stores: I've been mulling over this
>>>>>> problem
>>>>>>>>>> myself.
>>>>>>>>>>>> As
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>> stands,
>>>>>>>>>>>>>>>>>>> custom stores would essentially lose
>>>> checkpointing
>>>>>>>> with no
>>>>>>>>>>>>>>> indication
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> they're expected to make changes, besides a
>>> line
>>>>> in
>>>>>>>> the
>>>>>>>>>>> release
>>>>>>>>>>>>>>>> notes. I
>>>>>>>>>>>>>>>>>>> agree that the best solution would be to
>>>> provide a
>>>>>>>> default
>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> checkpoints
>>>>>>>>>>>>>>>>>>> to a file. The one thing I would change is
>>> that
>>>>> the
>>>>>>>>>>>> checkpointing
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>> store-local file, instead of a per-Task file.
>>>> This
>>>>>>>> way the
>>>>>>>>>>>>>>> StateStore
>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>> technically owns its own checkpointing (via a
>>>>>> default
>>>>>>>>>>>>>>> implementation),
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> the StateManager/Task execution engine
>>> doesn't
>>>>> need
>>>>>>>> to know
>>>>>>>>>>>>>>> anything
>>>>>>>>>>>>>>>>>> about
>>>>>>>>>>>>>>>>>>> checkpointing, which greatly simplifies some
>>> of
>>>>> the
>>>>>>>> logic.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 2. OOME errors: The main reasons why I didn't
>>>>>> explore
>>>>>>>> a
>>>>>>>>>>>> solution
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> this is
>>>>>>>>>>>>>>>>>>> a) to keep this KIP as simple as possible,
>>> and
>>>> b)
>>>>>>>> because
>>>>>>>>>> I'm
>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> exactly
>>>>>>>>>>>>>>>>>>> how to signal that a Task should commit
>>>>> prematurely.
>>>>>>>> I'm
>>>>>>>>>>>>> confident
>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>> possible, and I think it's worth adding a
>>>> section
>>>>> on
>>>>>>>>>> handling
>>>>>>>>>>>>> this.
>>>>>>>>>>>>>>>>>> Besides
>>>>>>>>>>>>>>>>>>> my proposal to force an early commit once
>>> memory
>>>>>> usage
>>>>>>>>>>> reaches
>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>> threshold,
>>>>>>>>>>>>>>>>>>> is there any other approach that you might
>>>> suggest
>>>>>> for
>>>>>>>>>>> tackling
>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> problem?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 3. ALOS: I can add in an explicit paragraph,
>>> but
>>>>> my
>>>>>>>>>>> assumption
>>>>>>>>>>>> is
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>> since transactional behaviour comes at
>>> little/no
>>>>>>>> cost, that
>>>>>>>>>>> it
>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>> available by default on all stores,
>>> irrespective
>>>>> of
>>>>>>>> the
>>>>>>>>>>>>> processing
>>>>>>>>>>>>>>>> mode.
>>>>>>>>>>>>>>>>>>> While ALOS doesn't use transactions, the Task
>>>>> itself
>>>>>>>> still
>>>>>>>>>>>>>>> "commits",
>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>> the behaviour should be correct under ALOS
>>> too.
>>>>> I'm
>>>>>>>> not
>>>>>>>>>>>> convinced
>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>>> worth having both
>>>> transactional/non-transactional
>>>>>>>> stores
>>>>>>>>>>>>>>> available, as
>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> would considerably increase the complexity of
>>>> the
>>>>>>>> codebase,
>>>>>>>>>>> for
>>>>>>>>>>>>>>> very
>>>>>>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>>>>> benefit.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> 4. Method deprecation: Are you referring to
>>>>>>>>>>>>>>> StateStore#getPosition()?
>>>>>>>>>>>>>>>>>> As I
>>>>>>>>>>>>>>>>>>> understand it, Position contains the
>>> position of
>>>>> the
>>>>>>>>>> *source*
>>>>>>>>>>>>>>> topics,
>>>>>>>>>>>>>>>>>>> whereas the commit offsets would be the
>>>>> *changelog*
>>>>>>>>>> offsets.
>>>>>>>>>>> So
>>>>>>>>>>>>>>> it's
>>>>>>>>>>>>>>>>>> still
>>>>>>>>>>>>>>>>>>> necessary to retain the Position data, as
>>> well
>>>> as
>>>>>> the
>>>>>>>>>>> changelog
>>>>>>>>>>>>>>>> offsets.
>>>>>>>>>>>>>>>>>>> What I meant in the KIP is that Position
>>> offsets
>>>>> are
>>>>>>>>>>> currently
>>>>>>>>>>>>>>> stored
>>>>>>>>>>>>>>>>>> in a
>>>>>>>>>>>>>>>>>>> file, and since we can atomically store
>>> metadata
>>>>>>>> along with
>>>>>>>>>>> the
>>>>>>>>>>>>>>> record
>>>>>>>>>>>>>>>>>>> batch we commit to RocksDB, we can move our
>>>>> Position
>>>>>>>>>> offsets
>>>>>>>>>>> in
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>> metadata too, and gain the same transactional
>>>>>>>> guarantees
>>>>>>>>>> that
>>>>>>>>>>>> we
>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>> changelog offsets, ensuring that the Position
>>>>>> offsets
>>>>>>>> are
>>>>>>>>>>>>>>> consistent
>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> the records that are read from the database.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Tue, 22 Nov 2022 at 16:25, John Roesler <
>>>>>>>>>>>> vvcephei@apache.org>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks for publishing this alternative,
>>> Nick!
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> The benchmark you mentioned in the KIP-844
>>>>>>>> discussion
>>>>>>>>>> seems
>>>>>>>>>>>>> like
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>> compelling reason to revisit the built-in
>>>>>>>>>> transactionality
>>>>>>>>>>>>>>>> mechanism.
>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>> also appreciate you analysis, showing that
>>> for
>>>>>> most
>>>>>>>> use
>>>>>>>>>>>> cases,
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> write
>>>>>>>>>>>>>>>>>>>> batch approach should be just fine.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> There are a couple of points that would
>>> hold
>>>> me
>>>>>>>> back from
>>>>>>>>>>>>>>> approving
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>> KIP right now:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 1. Loss of coverage for custom stores.
>>>>>>>>>>>>>>>>>>>> The fact that you can plug in a
>>> (relatively)
>>>>>> simple
>>>>>>>>>>>>>>> implementation
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> XStateStore interfaces and automagically
>>> get a
>>>>>>>>>> distributed
>>>>>>>>>>>>>>> database
>>>>>>>>>>>>>>>>>> out
>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> it is a significant benefit of Kafka
>>> Streams.
>>>>> I'd
>>>>>>>> hate to
>>>>>>>>>>>> lose
>>>>>>>>>>>>>>> it,
>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>> would be better to spend some time and
>>> come up
>>>>>> with
>>>>>>>> a way
>>>>>>>>>>> to
>>>>>>>>>>>>>>>> preserve
>>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> property. For example, can we provide a
>>>> default
>>>>>>>>>>>> implementation
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> `commit(..)` that re-implements the
>>> existing
>>>>>>>>>>> checkpoint-file
>>>>>>>>>>>>>>>>>> approach? Or
>>>>>>>>>>>>>>>>>>>> perhaps add an `isTransactional()` flag to
>>> the
>>>>>> state
>>>>>>>>>> store
>>>>>>>>>>>>>>> interface
>>>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>>>> that the runtime can decide whether to
>>>> continue
>>>>> to
>>>>>>>> manage
>>>>>>>>>>>>>>> checkpoint
>>>>>>>>>>>>>>>>>>> files
>>>>>>>>>>>>>>>>>>>> vs delegating transactionality to the
>>> stores?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 2. Guarding against OOME
>>>>>>>>>>>>>>>>>>>> I appreciate your analysis, but I don't
>>> think
>>>>> it's
>>>>>>>>>>> sufficient
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> say
>>>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>> we will solve the memory problem later if
>>> it
>>>>>> becomes
>>>>>>>>>>>> necessary.
>>>>>>>>>>>>>>> The
>>>>>>>>>>>>>>>>>>>> experience leading to that situation would
>>> be
>>>>>> quite
>>>>>>>> bad:
>>>>>>>>>>>>> Imagine,
>>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>>>>> upgrade to AK 3.next, your tests pass, so
>>> you
>>>>>>>> deploy to
>>>>>>>>>>>>>>> production.
>>>>>>>>>>>>>>>>>> That
>>>>>>>>>>>>>>>>>>>> night, you get paged because your app is
>>> now
>>>>>>>> crashing
>>>>>>>>>> with
>>>>>>>>>>>>>>> OOMEs. As
>>>>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>>> all OOMEs, you'll have a really hard time
>>>>> finding
>>>>>>>> the
>>>>>>>>>> root
>>>>>>>>>>>>> cause,
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>> once
>>>>>>>>>>>>>>>>>>>> you do, you won't have a clear path to
>>> resolve
>>>>> the
>>>>>>>> issue.
>>>>>>>>>>> You
>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>>> tune down the commit interval and cache
>>> buffer
>>>>>> size
>>>>>>>> until
>>>>>>>>>>> you
>>>>>>>>>>>>>>> stop
>>>>>>>>>>>>>>>>>>> getting
>>>>>>>>>>>>>>>>>>>> crashes.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> FYI, I know of multiple cases where people
>>> run
>>>>> EOS
>>>>>>>> with
>>>>>>>>>>> much
>>>>>>>>>>>>>>> larger
>>>>>>>>>>>>>>>>>>> commit
>>>>>>>>>>>>>>>>>>>> intervals to get better batching than the
>>>>> default,
>>>>>>>> so I
>>>>>>>>>>> don't
>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>> pathological case would be as rare as you
>>>>> suspect.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Given that we already have the rudiments
>>> of an
>>>>>> idea
>>>>>>>> of
>>>>>>>>>> what
>>>>>>>>>>>> we
>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>> do
>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>> prevent this downside, we should take the
>>> time
>>>>> to
>>>>>>>> design
>>>>>>>>>> a
>>>>>>>>>>>>>>> solution.
>>>>>>>>>>>>>>>>>> We
>>>>>>>>>>>>>>>>>>> owe
>>>>>>>>>>>>>>>>>>>> it to our users to ensure that awesome new
>>>>>> features
>>>>>>>> don't
>>>>>>>>>>>> come
>>>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>>>>> bitter
>>>>>>>>>>>>>>>>>>>> pills unless we can't avoid it.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 3. ALOS mode.
>>>>>>>>>>>>>>>>>>>> On the other hand, I didn't see an
>>> indication
>>>> of
>>>>>> how
>>>>>>>>>> stores
>>>>>>>>>>>>> will
>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>> handled under ALOS (aka non-EOS) mode.
>>>>>>>> Theoretically, the
>>>>>>>>>>>>>>>>>>> transactionality
>>>>>>>>>>>>>>>>>>>> of the store and the processing mode are
>>>>>>>> orthogonal. A
>>>>>>>>>>>>>>> transactional
>>>>>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>>>> would serve ALOS just as well as a
>>>>>>>> non-transactional one
>>>>>>>>>>> (if
>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> better).
>>>>>>>>>>>>>>>>>>>> Under ALOS, though, the default commit
>>>> interval
>>>>> is
>>>>>>>> five
>>>>>>>>>>>>> minutes,
>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> memory issue is far more pressing.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> As I see it, we have several options to
>>>> resolve
>>>>>> this
>>>>>>>>>> point.
>>>>>>>>>>>> We
>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>>>>> demonstrate that transactional stores work
>>>> just
>>>>>>>> fine for
>>>>>>>>>>> ALOS
>>>>>>>>>>>>>>> and we
>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>> therefore just swap over unconditionally.
>>> We
>>>>> could
>>>>>>>> also
>>>>>>>>>>>> disable
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> transactional mechanism under ALOS so that
>>>>> stores
>>>>>>>> operate
>>>>>>>>>>>> just
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>> they do today when run in ALOS mode.
>>> Finally,
>>>> we
>>>>>>>> could do
>>>>>>>>>>> the
>>>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> KIP-844 and make transactional stores
>>> opt-in
>>>>> (it'd
>>>>>>>> be
>>>>>>>>>>> better
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> extra opt-in mechanism, but it's a good
>>>>>>>>>>> get-out-of-jail-free
>>>>>>>>>>>>>>> card).
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> 4. (minor point) Deprecation of methods
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> You mentioned that the new `commit` 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?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks again for the KIP! It's really nice
>>>> that
>>>>>> you
>>>>>>>> and
>>>>>>>>>>> Alex
>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>> chance to collaborate on both directions so
>>>> that
>>>>>> we
>>>>>>>> can
>>>>>>>>>> get
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> best
>>>>>>>>>>>>>>>>>>>> outcome for Streams and its users.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> -John
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On 2022/11/21 15:02:15 Nick Telford wrote:
>>>>>>>>>>>>>>>>>>>>> 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
>>>>>>>>>>>>>>>>>>>>> <
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>>>>>>>>>>>>>>>>>>>>> ,
>>>>>>>>>>>>>>>>>>>>> so that it can be discussed/reviewed
>>>>> separately
>>>>>>>> from
>>>>>>>>>>>> KIP-844.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Alex: I'm especially interested in what
>>> you
>>>>>> think!
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I have a nearly complete implementation
>>> of
>>>> the
>>>>>>>> changes
>>>>>>>>>>>>>>> outlined in
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>> KIP, please let me know if you'd like me
>>> to
>>>>> push
>>>>>>>> them
>>>>>>>>>> for
>>>>>>>>>>>>>>> review
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>> advance
>>>>>>>>>>>>>>>>>>>>> of a vote.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Regards,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Nick
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
> 

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

Posted by Colt McNealy <co...@littlehorse.io>.
Nick,

That is a good point, the Checkpoint Files have been a part of the State
Store implementation for a long time and there might be some risk involved
in removing it. And yes, if you go with the idea of moving offset
management out of the state store itself, then you'll need to flush RocksDB
before updating the Checkpoint file.

As far as I understand it, the commit procedure would look like:

1.  Commit Kafka Txn
2.  Commit RocksDB.
2a Flush RocksDB
3.  Update Checkpoint File

Lastly, the online migration process just might be a bit simpler and easier
to test/verify if we don't need to add a column family and copy over the
value from the checkpoint files.

If you're able to resolve the issue with the StreamPartitionAssignor,
however, then I would be totally fine with moving offset management into
the state store, which would allow you to skip steps 2a and 3. 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 <ni...@gmail.com> wrote:

> Hi everyone,
>
> I find myself (again) considering removing the offset management from
> StateStores, and keeping the old checkpoint file system. The reason is that
> the StreamPartitionAssignor directly reads checkpoint files in order to
> determine which instance has the most up-to-date copy of the local state.
> If we move offsets into the StateStore itself, then we will need to open,
> initialize, read offsets and then close each StateStore (that is not
> already assigned and open) for which we have *any* local state, on every
> rebalance.
>
> Generally, I don't think there are many "orphan" stores like this sitting
> around on most instances, but even a few would introduce additional latency
> to an already somewhat lengthy rebalance procedure.
>
> I'm leaning towards Colt's (Slack) suggestion of just keeping things in the
> checkpoint file(s) for now, and not worrying about the race. The downside
> is that we wouldn't be able to remove the explicit RocksDB flush on-commit,
> which likely hurts performance.
>
> If anyone has any thoughts or ideas on this subject, I would appreciate it!
>
> Regards,
> Nick
>
> On Wed, 19 Apr 2023 at 15:05, Nick Telford <ni...@gmail.com> wrote:
>
> > Hi Colt,
> >
> > The issue is that if there's a crash between 2 and 3, then you still end
> > up with inconsistent data in RocksDB. The only way to guarantee that your
> > checkpoint offsets and locally stored data are consistent with each other
> > are to atomically commit them, which can be achieved by having the
> offsets
> > stored in RocksDB.
> >
> > The offsets column family is likely to be extremely small (one
> > per-changelog partition + one per Topology input partition for regular
> > stores, one per input partition for global stores). So the overhead will
> be
> > minimal.
> >
> > A major benefit of doing this is that we can remove the explicit calls to
> > db.flush(), which forcibly flushes memtables to disk on-commit. It turns
> > out, RocksDB memtable flushes are largely dictated by Kafka Streams
> > commits, *not* RocksDB configuration, which could be a major source of
> > confusion. Atomic checkpointing makes it safe to remove these explicit
> > flushes, because it no longer matters exactly when RocksDB flushes data
> to
> > disk; since the data and corresponding checkpoint offsets will always be
> > flushed together, the local store is always in a consistent state, and
> > on-restart, it can always safely resume restoration from the on-disk
> > offsets, restoring the small amount of data that hadn't been flushed when
> > the app exited/crashed.
> >
> > Regards,
> > Nick
> >
> > On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io> wrote:
> >
> >> Nick,
> >>
> >> Thanks for your reply. Ack to A) and B).
> >>
> >> For item C), I see what you're referring to. Your proposed solution will
> >> work, so no need to change it. What I was suggesting was that it might
> be
> >> possible to achieve this with only one column family. So long as:
> >>
> >>    - No uncommitted records (i.e. not committed to the changelog) are
> >>    *committed* to the state store, AND
> >>    - The Checkpoint offset (which refers to the changelog topic) is less
> >>    than or equal to the last written changelog offset in rocksdb
> >>
> >> I don't see the need to do the full restoration from scratch. My
> >> understanding was that prior to 844/892, full restorations were required
> >> because there could be uncommitted records written to RocksDB; however,
> >> given your use of RocksDB transactions, that can be avoided with the
> >> pattern of 1) commit Kafka transaction, 2) commit RocksDB transaction,
> 3)
> >> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <ni...@gmail.com>
> >> wrote:
> >>
> >> > Hi Colt,
> >> >
> >> > A. I've done my best to de-couple the StateStore stuff from the rest
> of
> >> the
> >> > Streams engine. The fact that there will be only one ongoing (write)
> >> > transaction at a time is not guaranteed by any API, and is just a
> >> > consequence of the way Streams operates. To that end, I tried to
> ensure
> >> the
> >> > documentation and guarantees provided by the new APIs are independent
> of
> >> > this incidental behaviour. In practice, you're right, this essentially
> >> > refers to "interactive queries", which are technically "read
> >> transactions",
> >> > even if they don't actually use the transaction API to isolate
> >> themselves.
> >> >
> >> > B. Yes, although not ideal. This is for backwards compatibility,
> >> because:
> >> >     1) Existing custom StateStore implementations will implement
> >> flush(),
> >> > and not commit(), but the Streams engine now calls commit(), so those
> >> calls
> >> > need to be forwarded to flush() for these legacy stores.
> >> >     2) Existing StateStore *users*, i.e. outside of the Streams engine
> >> > itself, may depend on explicitly calling flush(), so for these cases,
> >> > flush() needs to be redirected to call commit().
> >> > If anyone has a better way to guarantee compatibility without
> >> introducing
> >> > this potential recursion loop, I'm open to changes!
> >> >
> >> > C. This is described in the "Atomic Checkpointing" section. Offsets
> are
> >> > stored in a separate RocksDB column family, which is guaranteed to be
> >> > atomically flushed to disk with all other column families. The issue
> of
> >> > checkpoints being written to disk after commit causing inconsistency
> if
> >> it
> >> > crashes in between is the reason why, under EOS, checkpoint files are
> >> only
> >> > written on clean shutdown. This is one of the major causes of "full
> >> > restorations", so moving the offsets into a place where they can be
> >> > guaranteed to be atomically written with the data they checkpoint
> >> allows us
> >> > to write the checkpoint offsets *on every commit*, not just on clean
> >> > shutdown.
> >> >
> >> > Regards,
> >> > Nick
> >> >
> >> > On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io>
> wrote:
> >> >
> >> > > Nick,
> >> > >
> >> > > Thank you for continuing this work. I have a few minor clarifying
> >> > > questions.
> >> > >
> >> > > A) "Records written to any transaction are visible to all other
> >> > > transactions immediately." I am confused here—I thought there could
> >> only
> >> > be
> >> > > one transaction going on at a time for a given state store given the
> >> > > threading model for processing records on a Task. Do you mean
> >> Interactive
> >> > > Queries by "other transactions"? (If so, then everything makes
> sense—I
> >> > > thought that since IQ were read-only then they didn't count as
> >> > > transactions).
> >> > >
> >> > > B) Is it intentional that the default implementations of the flush()
> >> and
> >> > > commit() methods in the StateStore class refer to each other in some
> >> sort
> >> > > of unbounded recursion?
> >> > >
> >> > > C) How will the getCommittedOffset() method work? At first I thought
> >> the
> >> > > way to do it would be using a special key in the RocksDB store to
> >> store
> >> > the
> >> > > offset, and committing that with the transaction. But upon second
> >> > thought,
> >> > > since restoration from the changelog is an idempotent procedure, I
> >> think
> >> > it
> >> > > would be fine to 1) commit the RocksDB transaction and then 2) write
> >> the
> >> > > offset to disk in a checkpoint file. If there is a crash between 1)
> >> and
> >> > 2),
> >> > > I think the only downside is 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 <
> nick.telford@gmail.com>
> >> > > wrote:
> >> > >
> >> > > > Hi everyone,
> >> > > >
> >> > > > I've updated the KIP to reflect the latest version of the design:
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >> > > >
> >> > > > There are several changes in there that reflect feedback from this
> >> > > thread,
> >> > > > and there's a new section and a bunch of interface changes
> relating
> >> to
> >> > > > Atomic Checkpointing, which is the final piece of the puzzle to
> >> making
> >> > > > everything robust.
> >> > > >
> >> > > > Let me know what you think!
> >> > > >
> >> > > > Regards,
> >> > > > Nick
> >> > > >
> >> > > > On Tue, 3 Jan 2023 at 11:33, Nick Telford <nick.telford@gmail.com
> >
> >> > > wrote:
> >> > > >
> >> > > > > Hi Lucas,
> >> > > > >
> >> > > > > Thanks for looking over my KIP.
> >> > > > >
> >> > > > > A) The bound is per-instance, not per-Task. This was a typo in
> the
> >> > KIP
> >> > > > > that I've now corrected. It was originally per-Task, but I
> >> changed it
> >> > > to
> >> > > > > per-instance for exactly the reason you highlighted.
> >> > > > > B) It's worth noting that transactionality is only enabled under
> >> EOS,
> >> > > and
> >> > > > > in the default mode of operation (ALOS), there should be no
> >> change in
> >> > > > > behavior at all. I think, under EOS, we can mitigate the impact
> on
> >> > > users
> >> > > > by
> >> > > > > sufficiently low default values for the memory bound
> >> configuration. I
> >> > > > > understand your hesitation to include a significant change of
> >> > > behaviour,
> >> > > > > especially in a minor release, but I suspect that most users
> will
> >> > > prefer
> >> > > > > the memory impact (under EOS) to the existing behaviour of
> >> frequent
> >> > > state
> >> > > > > restorations! If this is a problem, the changes can wait until
> the
> >> > next
> >> > > > > major release. I'll be running a patched version of streams in
> >> > > production
> >> > > > > with these changes as soon as they're ready, so it won't disrupt
> >> me
> >> > :-D
> >> > > > > C) The main purpose of this sentence was just to note that some
> >> > changes
> >> > > > > will need to be made to the way Segments are handled in order to
> >> > ensure
> >> > > > > they also benefit from transactions. At the time I wrote it, I
> >> hadn't
> >> > > > > figured out the specific changes necessary, so it was
> deliberately
> >> > > vague.
> >> > > > > This is the one outstanding problem I'm currently working on,
> and
> >> > I'll
> >> > > > > update this section with more detail once I have figured out the
> >> > exact
> >> > > > > changes required.
> >> > > > > D) newTransaction() provides the necessary isolation guarantees.
> >> > While
> >> > > > > the RocksDB implementation of transactions doesn't technically
> >> *need*
> >> > > > > read-only users to call newTransaction(), other implementations
> >> > (e.g. a
> >> > > > > hypothetical PostgresStore) may require it. Calling
> >> newTransaction()
> >> > > when
> >> > > > > no transaction is necessary is essentially free, as it will just
> >> > return
> >> > > > > this.
> >> > > > >
> >> > > > > I didn't do any profiling of the KIP-844 PoC, but I think it
> >> should
> >> > be
> >> > > > > fairly obvious where the performance problems stem from: writes
> >> under
> >> > > > > KIP-844 require 3 extra memory-copies: 1 to encode it with the
> >> > > > > tombstone/record flag, 1 to decode it from the tombstone/record
> >> flag,
> >> > > > and 1
> >> > > > > to copy the record from the "temporary" store to the "main"
> store,
> >> > when
> >> > > > the
> >> > > > > transaction commits. The different approach taken by KIP-869
> >> should
> >> > > > perform
> >> > > > > much better, as it avoids all these copies, and may actually
> >> perform
> >> > > > > slightly better than trunk, due to batched writes in RocksDB
> >> > performing
> >> > > > > better than non-batched writes.[1]
> >> > > > >
> >> > > > > Regards,
> >> > > > > Nick
> >> > > > >
> >> > > > > 1:
> >> > > >
> >> >
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >> > > > >
> >> > > > > On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
> >> lbrutschy@confluent.io
> >> > > > .invalid>
> >> > > > > wrote:
> >> > > > >
> >> > > > >> Hi Nick,
> >> > > > >>
> >> > > > >> I'm just starting to read up on the whole discussion about
> >> KIP-892
> >> > and
> >> > > > >> KIP-844. Thanks a lot for your work on this, I do think
> >> > > > >> `WriteBatchWithIndex` may be the way to go here. I do have some
> >> > > > >> questions about the latest draft.
> >> > > > >>
> >> > > > >>  A) If I understand correctly, you propose to put a bound on
> the
> >> > > > >> (native) memory consumed by each task. However, I wonder if
> this
> >> is
> >> > > > >> sufficient if we have temporary imbalances in the cluster. For
> >> > > > >> example, depending on the timing of rebalances during a cluster
> >> > > > >> restart, it could happen that a single streams node is
> assigned a
> >> > lot
> >> > > > >> more tasks than expected. With your proposed change, this would
> >> mean
> >> > > > >> that the memory required by this one node could be a multiple
> of
> >> > what
> >> > > > >> is required during normal operation. I wonder if it wouldn't be
> >> > safer
> >> > > > >> to put a global bound on the memory use, across all tasks.
> >> > > > >>  B) Generally, the memory concerns still give me the feeling
> that
> >> > this
> >> > > > >> should not be enabled by default for all users in a minor
> >> release.
> >> > > > >>  C) In section "Transaction Management": the sentence "A
> similar
> >> > > > >> analogue will be created to automatically manage `Segment`
> >> > > > >> transactions.". Maybe this is just me lacking some background,
> >> 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 you to find out why the KIP-844 PoC is 20x slower
> >> (e.g.
> >> > > > >> by providing a flame graph), that would be quite interesting.
> >> > > > >>
> >> > > > >> Cheers,
> >> > > > >> Lucas
> >> > > > >>
> >> > > > >> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
> >> > nick.telford@gmail.com>
> >> > > > >> wrote:
> >> > > > >> >
> >> > > > >> > Hi everyone,
> >> > > > >> >
> >> > > > >> > I've updated the KIP with a more detailed design, which
> >> reflects
> >> > the
> >> > > > >> > implementation I've been working on:
> >> > > > >> >
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >> > > > >> >
> >> > > > >> > This new design should address the outstanding points already
> >> made
> >> > > in
> >> > > > >> the
> >> > > > >> > thread.
> >> > > > >> >
> >> > > > >> > Please let me know if there are areas that are unclear or
> need
> >> > more
> >> > > > >> > clarification.
> >> > > > >> >
> >> > > > >> > I have a (nearly) working implementation. I'm confident that
> >> the
> >> > > > >> remaining
> >> > > > >> > work (making Segments behave) will not impact the documented
> >> > design.
> >> > > > >> >
> >> > > > >> > Regards,
> >> > > > >> >
> >> > > > >> > Nick
> >> > > > >> >
> >> > > > >> > On Tue, 6 Dec 2022 at 19:24, Colt McNealy <
> colt@littlehorse.io
> >> >
> >> > > > wrote:
> >> > > > >> >
> >> > > > >> > > Nick,
> >> > > > >> > >
> >> > > > >> > > Thank you for the reply; that makes sense. I was hoping
> that,
> >> > > since
> >> > > > >> reading
> >> > > > >> > > uncommitted records from IQ in EOS isn't part of the
> >> documented
> >> > > API,
> >> > > > >> maybe
> >> > > > >> > > you *wouldn't* have to wait for the next major release to
> >> make
> >> > > that
> >> > > > >> change;
> >> > > > >> > > but given 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 <
> >> > > > nick.telford@gmail.com>
> >> > > > >> > > wrote:
> >> > > > >> > >
> >> > > > >> > > > Hi Colt,
> >> > > > >> > > >
> >> > > > >> > > > 10: Yes, I agree it's not ideal. I originally intended to
> >> try
> >> > to
> >> > > > >> keep the
> >> > > > >> > > > behaviour unchanged as much as possible, otherwise we'd
> >> have
> >> > to
> >> > > > >> wait for
> >> > > > >> > > a
> >> > > > >> > > > major version release to land these changes.
> >> > > > >> > > > 20: Good point, ALOS doesn't need the same level of
> >> guarantee,
> >> > > and
> >> > > > >> the
> >> > > > >> > > > typically longer commit intervals would be problematic
> when
> >> > > > reading
> >> > > > >> only
> >> > > > >> > > > "committed" records.
> >> > > > >> > > >
> >> > > > >> > > > I've been away for 5 days recovering from minor surgery,
> >> but I
> >> > > > >> spent a
> >> > > > >> > > > considerable amount of that time working through ideas
> for
> >> > > > possible
> >> > > > >> > > > solutions in my head. I think your suggestion of keeping
> >> ALOS
> >> > > > >> as-is, but
> >> > > > >> > > > buffering writes for EOS is the right path forwards,
> >> although
> >> > I
> >> > > > >> have a
> >> > > > >> > > > solution that both expands on this, and provides for some
> >> more
> >> > > > >> formal
> >> > > > >> > > > guarantees.
> >> > > > >> > > >
> >> > > > >> > > > Essentially, adding support to KeyValueStores for
> >> > > "Transactions",
> >> > > > >> with
> >> > > > >> > > > clearly defined IsolationLevels. Using "Read Committed"
> >> when
> >> > > under
> >> > > > >> EOS,
> >> > > > >> > > and
> >> > > > >> > > > "Read Uncommitted" under ALOS.
> >> > > > >> > > >
> >> > > > >> > > > The nice thing about this approach is that it gives us
> much
> >> > more
> >> > > > >> clearly
> >> > > > >> > > > defined isolation behaviour that can be properly
> >> documented to
> >> > > > >> ensure
> >> > > > >> > > users
> >> > > > >> > > > know what to expect.
> >> > > > >> > > >
> >> > > > >> > > > I'm still working out the kinks in the design, and will
> >> update
> >> > > the
> >> > > > >> KIP
> >> > > > >> > > when
> >> > > > >> > > > I have something. The main struggle is trying to
> implement
> >> > this
> >> > > > >> without
> >> > > > >> > > > making any major changes to the existing interfaces or
> >> > breaking
> >> > > > >> existing
> >> > > > >> > > > implementations, because currently everything expects to
> >> > operate
> >> > > > >> directly
> >> > > > >> > > > on a StateStore, and not a Transaction of that store. I
> >> think
> >> > > I'm
> >> > > > >> getting
> >> > > > >> > > > close, although sadly I won't be able to progress much
> >> until
> >> > > next
> >> > > > >> week
> >> > > > >> > > due
> >> > > > >> > > > to some work commitments.
> >> > > > >> > > >
> >> > > > >> > > > Regards,
> >> > > > >> > > > Nick
> >> > > > >> > > >
> >> > > > >> > > > On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
> >> > colt@littlehorse.io>
> >> > > > >> wrote:
> >> > > > >> > > >
> >> > > > >> > > > > Nick,
> >> > > > >> > > > >
> >> > > > >> > > > > Thank you for the explanation, and also for the updated
> >> > KIP. I
> >> > > > am
> >> > > > >> quite
> >> > > > >> > > > > eager for this improvement to be released as it would
> >> > greatly
> >> > > > >> reduce
> >> > > > >> > > the
> >> > > > >> > > > > operational difficulties of EOS streams apps.
> >> > > > >> > > > >
> >> > > > >> > > > > Two questions:
> >> > > > >> > > > >
> >> > > > >> > > > > 10)
> >> > > > >> > > > > >When reading records, we will use the
> >> > > > >> > > > > WriteBatchWithIndex#getFromBatchAndDB
> >> > > > >> > > > >  and WriteBatchWithIndex#newIteratorWithBase utilities
> in
> >> > > order
> >> > > > to
> >> > > > >> > > ensure
> >> > > > >> > > > > that uncommitted writes are available to query.
> >> > > > >> > > > > Why do extra work to enable the reading of uncommitted
> >> > writes
> >> > > > >> during
> >> > > > >> > > IQ?
> >> > > > >> > > > > Code complexity aside, reading uncommitted writes is,
> in
> >> my
> >> > > > >> opinion, a
> >> > > > >> > > > > minor flaw in EOS IQ; it would be very nice to have the
> >> > > > guarantee
> >> > > > >> that,
> >> > > > >> > > > > with EOS, IQ only reads committed records. In order to
> >> avoid
> >> > > > dirty
> >> > > > >> > > reads,
> >> > > > >> > > > > one currently must query a standby replica (but this
> >> still
> >> > > > doesn't
> >> > > > >> > > fully
> >> > > > >> > > > > guarantee monotonic reads).
> >> > > > >> > > > >
> >> > > > >> > > > > 20) Is it also necessary to enable this optimization on
> >> ALOS
> >> > > > >> stores?
> >> > > > >> > > The
> >> > > > >> > > > > motivation of KIP-844 was mainly to reduce the need to
> >> > restore
> >> > > > >> state
> >> > > > >> > > from
> >> > > > >> > > > > scratch on unclean EOS shutdowns; with ALOS it was
> >> > acceptable
> >> > > to
> >> > > > >> accept
> >> > > > >> > > > > that there may have been uncommitted writes on disk.
> On a
> >> > side
> >> > > > >> note, if
> >> > > > >> > > > you
> >> > > > >> > > > > enable this type of store on ALOS processors, the
> >> community
> >> > > > would
> >> > > > >> > > > > 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 <
> >> > > > >> nick.telford@gmail.com>
> >> > > > >> > > > > wrote:
> >> > > > >> > > > >
> >> > > > >> > > > > > Hi everyone,
> >> > > > >> > > > > >
> >> > > > >> > > > > > I've drastically reduced the scope of this KIP to no
> >> > longer
> >> > > > >> include
> >> > > > >> > > the
> >> > > > >> > > > > > StateStore management of checkpointing. This can be
> >> added
> >> > > as a
> >> > > > >> KIP
> >> > > > >> > > > later
> >> > > > >> > > > > on
> >> > > > >> > > > > > to further optimize the consistency and performance
> of
> >> > state
> >> > > > >> stores.
> >> > > > >> > > > > >
> >> > > > >> > > > > > I've also added a section discussing some of the
> >> concerns
> >> > > > 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 2022 at 16:36, Nick Telford <
> >> > > > >> nick.telford@gmail.com>
> >> > > > >> > > > > wrote:
> >> > > > >> > > > > >
> >> > > > >> > > > > > > Hi Colt,
> >> > > > >> > > > > > >
> >> > > > >> > > > > > > I didn't do any profiling, but the 844
> >> implementation:
> >> > > > >> > > > > > >
> >> > > > >> > > > > > >    - Writes uncommitted records to a temporary
> >> RocksDB
> >> > > > >> instance
> >> > > > >> > > > > > >       - Since tombstones need to be flagged, all
> >> record
> >> > > > >> values are
> >> > > > >> > > > > > >       prefixed with a value/tombstone marker. This
> >> > > > >> necessitates a
> >> > > > >> > > > > memory
> >> > > > >> > > > > > copy.
> >> > > > >> > > > > > >    - On-commit, iterates all records in this
> >> temporary
> >> > > > >> instance and
> >> > > > >> > > > > > >    writes them to the main RocksDB store.
> >> > > > >> > > > > > >    - While iterating, the value/tombstone marker
> >> needs
> >> > to
> >> > > be
> >> > > > >> parsed
> >> > > > >> > > > and
> >> > > > >> > > > > > >    the real value extracted. This necessitates
> >> another
> >> > > > memory
> >> > > > >> copy.
> >> > > > >> > > > > > >
> >> > > > >> > > > > > > My guess is that the cost of iterating the
> temporary
> >> > > RocksDB
> >> > > > >> store
> >> > > > >> > > is
> >> > > > >> > > > > the
> >> > > > >> > > > > > > major factor, with the 2 extra memory copies
> >> per-Record
> >> > > > >> > > contributing
> >> > > > >> > > > a
> >> > > > >> > > > > > > significant amount too.
> >> > > > >> > > > > > >
> >> > > > >> > > > > > > Regards,
> >> > > > >> > > > > > > Nick
> >> > > > >> > > > > > >
> >> > > > >> > > > > > > On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
> >> > > > >> colt@littlehorse.io>
> >> > > > >> > > > > wrote:
> >> > > > >> > > > > > >
> >> > > > >> > > > > > >> Hi all,
> >> > > > >> > > > > > >>
> >> > > > >> > > > > > >> Out of curiosity, why does the performance of the
> >> store
> >> > > > >> degrade so
> >> > > > >> > > > > > >> significantly with the 844 implementation? I
> >> wouldn't
> >> > be
> >> > > > too
> >> > > > >> > > > surprised
> >> > > > >> > > > > > by
> >> > > > >> > > > > > >> a
> >> > > > >> > > > > > >> 50-60% drop (caused by each record being written
> >> > twice),
> >> > > > but
> >> > > > >> 96%
> >> > > > >> > > is
> >> > > > >> > > > > > >> extreme.
> >> > > > >> > > > > > >>
> >> > > > >> > > > > > >> The only thing I can think of which could create
> >> such a
> >> > > > >> bottleneck
> >> > > > >> > > > > 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 Nick Telford <
> >> > > > >> > > > nick.telford@gmail.com>
> >> > > > >> > > > > > >> wrote:
> >> > > > >> > > > > > >>
> >> > > > >> > > > > > >> > Hi everyone,
> >> > > > >> > > > > > >> >
> >> > > > >> > > > > > >> > I've updated the KIP to resolve all the points
> >> that
> >> > > have
> >> > > > >> been
> >> > > > >> > > > raised
> >> > > > >> > > > > > so
> >> > > > >> > > > > > >> > far, with one exception: the ALOS default commit
> >> > > interval
> >> > > > >> of 5
> >> > > > >> > > > > minutes
> >> > > > >> > > > > > >> is
> >> > > > >> > > > > > >> > likely to cause WriteBatchWithIndex memory to
> grow
> >> > too
> >> > > > >> large.
> >> > > > >> > > > > > >> >
> >> > > > >> > > > > > >> > There's a couple of different things I can think
> >> of
> >> > to
> >> > > > >> solve
> >> > > > >> > > this:
> >> > > > >> > > > > > >> >
> >> > > > >> > > > > > >> >    - We already have a memory/record limit in
> the
> >> KIP
> >> > > to
> >> > > > >> prevent
> >> > > > >> > > > OOM
> >> > > > >> > > > > > >> >    errors. Should we choose a default value for
> >> > these?
> >> > > My
> >> > > > >> > > concern
> >> > > > >> > > > > here
> >> > > > >> > > > > > >> is
> >> > > > >> > > > > > >> > that
> >> > > > >> > > > > > >> >    anything we choose might seem rather
> >> arbitrary. We
> >> > > > could
> >> > > > >> > > change
> >> > > > >> > > > > > >> >    its behaviour such that under ALOS, it only
> >> > triggers
> >> > > > the
> >> > > > >> > > commit
> >> > > > >> > > > > of
> >> > > > >> > > > > > >> the
> >> > > > >> > > > > > >> >    StateStore, but under EOS, it triggers a
> >> commit of
> >> > > the
> >> > > > >> Kafka
> >> > > > >> > > > > > >> > transaction.
> >> > > > >> > > > > > >> >    - We could introduce a separate `
> >> > > > checkpoint.interval.ms`
> >> > > > >> to
> >> > > > >> > > > > allow
> >> > > > >> > > > > > >> ALOS
> >> > > > >> > > > > > >> >    to commit the StateStores more frequently
> than
> >> the
> >> > > > >> general
> >> > > > >> > > > > > >> >    commit.interval.ms? My concern here is that
> >> the
> >> > > > >> semantics of
> >> > > > >> > > > > this
> >> > > > >> > > > > > >> > config
> >> > > > >> > > > > > >> >    would depend 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 <
> >> > > > >> > > > nick.telford@gmail.com>
> >> > > > >> > > > > > >> wrote:
> >> > > > >> > > > > > >> >
> >> > > > >> > > > > > >> > > Hi Alex,
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > > Thanks for the feedback.
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > > I've updated the discussion of OOM issues by
> >> > > describing
> >> > > > >> how
> >> > > > >> > > > we'll
> >> > > > >> > > > > > >> handle
> >> > > > >> > > > > > >> > > it. Here's the new text:
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > > To mitigate this, we will automatically force
> a
> >> > Task
> >> > > > >> commit if
> >> > > > >> > > > the
> >> > > > >> > > > > > >> total
> >> > > > >> > > > > > >> > >> uncommitted records returned by
> >> > > > >> > > > > > >> > >> StateStore#approximateNumUncommittedEntries()
> >> > > > exceeds a
> >> > > > >> > > > > threshold,
> >> > > > >> > > > > > >> > >> configured by
> >> > > max.uncommitted.state.entries.per.task;
> >> > > > >> or the
> >> > > > >> > > > > total
> >> > > > >> > > > > > >> > >> memory used for buffering uncommitted records
> >> > > returned
> >> > > > >> by
> >> > > > >> > > > > > >> > >> StateStore#approximateNumUncommittedBytes()
> >> > exceeds
> >> > > > the
> >> > > > >> > > > threshold
> >> > > > >> > > > > > >> > >> configured by
> >> > max.uncommitted.state.bytes.per.task.
> >> > > > >> This will
> >> > > > >> > > > > > roughly
> >> > > > >> > > > > > >> > >> bound the memory required per-Task for
> >> buffering
> >> > > > >> uncommitted
> >> > > > >> > > > > > records,
> >> > > > >> > > > > > >> > >> irrespective of the commit.interval.ms, and
> >> will
> >> > > > >> effectively
> >> > > > >> > > > > bound
> >> > > > >> > > > > > >> the
> >> > > > >> > > > > > >> > >> number of records that will need to be
> >> restored in
> >> > > the
> >> > > > >> event
> >> > > > >> > > > of a
> >> > > > >> > > > > > >> > failure.
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > > These limits will be checked in
> >> StreamTask#process
> >> > > and
> >> > > > a
> >> > > > >> > > > premature
> >> > > > >> > > > > > >> commit
> >> > > > >> > > > > > >> > >> will be requested via Task#requestCommit().
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > > Note that these new methods provide default
> >> > > > >> implementations
> >> > > > >> > > that
> >> > > > >> > > > > > >> ensure
> >> > > > >> > > > > > >> > >> existing custom stores and non-transactional
> >> > stores
> >> > > > >> (e.g.
> >> > > > >> > > > > > >> > >> InMemoryKeyValueStore) do not force any early
> >> > > commits.
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > > I've chosen to have the StateStore expose
> >> > > > approximations
> >> > > > >> of
> >> > > > >> > > its
> >> > > > >> > > > > > buffer
> >> > > > >> > > > > > >> > > size/count instead of opaquely requesting a
> >> commit
> >> > in
> >> > > > >> order to
> >> > > > >> > > > > > >> delegate
> >> > > > >> > > > > > >> > the
> >> > > > >> > > > > > >> > > decision making to the Task itself. This
> enables
> >> > > Tasks
> >> > > > >> to look
> >> > > > >> > > > at
> >> > > > >> > > > > > >> *all*
> >> > > > >> > > > > > >> > of
> >> > > > >> > > > > > >> > > their StateStores, and determine whether an
> >> early
> >> > > > commit
> >> > > > >> is
> >> > > > >> > > > > > necessary.
> >> > > > >> > > > > > >> > > Notably, it enables pre-Task thresholds,
> >> instead of
> >> > > > >> per-Store,
> >> > > > >> > > > > which
> >> > > > >> > > > > > >> > > prevents Tasks with many StateStores from
> using
> >> > much
> >> > > > more
> >> > > > >> > > memory
> >> > > > >> > > > > > than
> >> > > > >> > > > > > >> > Tasks
> >> > > > >> > > > > > >> > > with one StateStore. This makes sense, since
> >> > commits
> >> > > > are
> >> > > > >> done
> >> > > > >> > > > > > by-Task,
> >> > > > >> > > > > > >> > not
> >> > > > >> > > > > > >> > > by-Store.
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > > Prizes* for anyone who can come up with a
> better
> >> > name
> >> > > > >> for the
> >> > > > >> > > > new
> >> > > > >> > > > > > >> config
> >> > > > >> > > > > > >> > > properties!
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > > Thanks for pointing out the potential
> >> performance
> >> > > > issues
> >> > > > >> of
> >> > > > >> > > > WBWI.
> >> > > > >> > > > > > From
> >> > > > >> > > > > > >> > the
> >> > > > >> > > > > > >> > > benchmarks that user posted[1], it looks like
> >> WBWI
> >> > > > still
> >> > > > >> > > > performs
> >> > > > >> > > > > > >> > > considerably better than individual puts,
> which
> >> is
> >> > > the
> >> > > > >> > > existing
> >> > > > >> > > > > > >> design,
> >> > > > >> > > > > > >> > so
> >> > > > >> > > > > > >> > > I'd actually expect a performance boost from
> >> WBWI,
> >> > > just
> >> > > > >> not as
> >> > > > >> > > > > great
> >> > > > >> > > > > > >> as
> >> > > > >> > > > > > >> > > we'd get from a plain WriteBatch. This does
> >> suggest
> >> > > > that
> >> > > > >> a
> >> > > > >> > > good
> >> > > > >> > > > > > >> > > optimization would be to use a regular
> >> WriteBatch
> >> > for
> >> > > > >> > > > restoration
> >> > > > >> > > > > > (in
> >> > > > >> > > > > > >> > > RocksDBStore#restoreBatch), since we know that
> >> > those
> >> > > > >> records
> >> > > > >> > > > will
> >> > > > >> > > > > > >> never
> >> > > > >> > > > > > >> > be
> >> > > > >> > > > > > >> > > queried before they're committed.
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > > 1:
> >> > > > >> > > > > > >> >
> >> > > > >> > > > > >
> >> > > > >> > > >
> >> > > > >>
> >> > >
> >> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > > * Just kidding, no prizes, sadly.
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > > On Wed, 23 Nov 2022 at 12:28, Alexander
> >> Sorokoumov
> >> > > > >> > > > > > >> > > <as...@confluent.io.invalid> wrote:
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> > >> 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 to keep the current
> >> > > checkpointing
> >> > > > >> as a
> >> > > > >> > > > > > fallback
> >> > > > >> > > > > > >> > >> mechanism. We want to keep existing users'
> >> > workflows
> >> > > > >> intact
> >> > > > >> > > if
> >> > > > >> > > > we
> >> > > > >> > > > > > >> can. A
> >> > > > >> > > > > > >> > >> non-intrusive way would be to add a separate
> >> > > > StateStore
> >> > > > >> > > method,
> >> > > > >> > > > > > say,
> >> > > > >> > > > > > >> > >> StateStore#managesCheckpointing(), that
> >> controls
> >> > > > >> whether the
> >> > > > >> > > > > state
> >> > > > >> > > > > > >> store
> >> > > > >> > > > > > >> > >> implementation owns checkpointing.
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> > >> I think that a solution to the transactional
> >> > writes
> >> > > > >> should
> >> > > > >> > > > > address
> >> > > > >> > > > > > >> the
> >> > > > >> > > > > > >> > >> OOMEs. One possible way to address that is to
> >> wire
> >> > > > >> > > StateStore's
> >> > > > >> > > > > > >> commit
> >> > > > >> > > > > > >> > >> request by adding, say,
> StateStore#commitNeeded
> >> > that
> >> > > > is
> >> > > > >> > > checked
> >> > > > >> > > > > in
> >> > > > >> > > > > > >> > >> StreamTask#commitNeeded via the corresponding
> >> > > > >> > > > > > ProcessorStateManager.
> >> > > > >> > > > > > >> > With
> >> > > > >> > > > > > >> > >> that change, RocksDBStore will have to track
> >> the
> >> > > > current
> >> > > > >> > > > > > transaction
> >> > > > >> > > > > > >> > size
> >> > > > >> > > > > > >> > >> and request a commit when the size goes over
> a
> >> > > > >> (configurable)
> >> > > > >> > > > > > >> threshold.
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> > >> AFAIU WriteBatchWithIndex might perform
> >> > > significantly
> >> > > > >> slower
> >> > > > >> > > > than
> >> > > > >> > > > > > >> > non-txn
> >> > > > >> > > > > > >> > >> puts as the batch size grows [1]. We should
> >> have a
> >> > > > >> > > > configuration
> >> > > > >> > > > > to
> >> > > > >> > > > > > >> fall
> >> > > > >> > > > > > >> > >> back to the current behavior (and/or disable
> >> txn
> >> > > > stores
> >> > > > >> for
> >> > > > >> > > > ALOS)
> >> > > > >> > > > > > >> unless
> >> > > > >> > > > > > >> > >> the benchmarks show negligible overhead for
> >> longer
> >> > > > >> commits /
> >> > > > >> > > > > > >> > large-enough
> >> > > > >> > > > > > >> > >> batch sizes.
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> > >> If you prefer to keep the KIP smaller, I
> would
> >> > > rather
> >> > > > >> cut out
> >> > > > >> > > > > > >> > >> state-store-managed checkpointing rather than
> >> > proper
> >> > > > >> OOMe
> >> > > > >> > > > > handling
> >> > > > >> > > > > > >> and
> >> > > > >> > > > > > >> > >> being able to switch to non-txn behavior. The
> >> > > > >> checkpointing
> >> > > > >> > > 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 configuration parts as well as
> review
> >> and
> >> > > > test
> >> > > > >> the
> >> > > > >> > > > > patch.
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> > >> Best,
> >> > > > >> > > > > > >> > >> Alex
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> > >> 1.
> >> https://github.com/facebook/rocksdb/issues/608
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> > >> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford
> <
> >> > > > >> > > > > > nick.telford@gmail.com
> >> > > > >> > > > > > >> >
> >> > > > >> > > > > > >> > >> wrote:
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> > >> > Hi John,
> >> > > > >> > > > > > >> > >> >
> >> > > > >> > > > > > >> > >> > Thanks for the review and feedback!
> >> > > > >> > > > > > >> > >> >
> >> > > > >> > > > > > >> > >> > 1. Custom Stores: I've been mulling over
> this
> >> > > > problem
> >> > > > >> > > myself.
> >> > > > >> > > > > As
> >> > > > >> > > > > > it
> >> > > > >> > > > > > >> > >> stands,
> >> > > > >> > > > > > >> > >> > custom stores would essentially lose
> >> > checkpointing
> >> > > > >> with no
> >> > > > >> > > > > > >> indication
> >> > > > >> > > > > > >> > >> that
> >> > > > >> > > > > > >> > >> > they're expected to make changes, besides a
> >> line
> >> > > in
> >> > > > >> the
> >> > > > >> > > > release
> >> > > > >> > > > > > >> > notes. I
> >> > > > >> > > > > > >> > >> > agree that the best solution would be to
> >> > provide a
> >> > > > >> default
> >> > > > >> > > > that
> >> > > > >> > > > > > >> > >> checkpoints
> >> > > > >> > > > > > >> > >> > to a file. The one thing I would change is
> >> that
> >> > > the
> >> > > > >> > > > > checkpointing
> >> > > > >> > > > > > >> is
> >> > > > >> > > > > > >> > to
> >> > > > >> > > > > > >> > >> a
> >> > > > >> > > > > > >> > >> > store-local file, instead of a per-Task
> file.
> >> > This
> >> > > > >> way the
> >> > > > >> > > > > > >> StateStore
> >> > > > >> > > > > > >> > >> still
> >> > > > >> > > > > > >> > >> > technically owns its own checkpointing
> (via a
> >> > > > default
> >> > > > >> > > > > > >> implementation),
> >> > > > >> > > > > > >> > >> and
> >> > > > >> > > > > > >> > >> > the StateManager/Task execution engine
> >> doesn't
> >> > > need
> >> > > > >> to know
> >> > > > >> > > > > > >> anything
> >> > > > >> > > > > > >> > >> about
> >> > > > >> > > > > > >> > >> > checkpointing, which greatly simplifies
> some
> >> of
> >> > > the
> >> > > > >> logic.
> >> > > > >> > > > > > >> > >> >
> >> > > > >> > > > > > >> > >> > 2. OOME errors: The main reasons why I
> didn't
> >> > > > explore
> >> > > > >> a
> >> > > > >> > > > > solution
> >> > > > >> > > > > > to
> >> > > > >> > > > > > >> > >> this is
> >> > > > >> > > > > > >> > >> > a) to keep this KIP as simple as possible,
> >> and
> >> > b)
> >> > > > >> because
> >> > > > >> > > I'm
> >> > > > >> > > > > not
> >> > > > >> > > > > > >> > >> exactly
> >> > > > >> > > > > > >> > >> > how to signal that a Task should commit
> >> > > prematurely.
> >> > > > >> I'm
> >> > > > >> > > > > > confident
> >> > > > >> > > > > > >> > it's
> >> > > > >> > > > > > >> > >> > possible, and I think it's worth adding a
> >> > section
> >> > > on
> >> > > > >> > > handling
> >> > > > >> > > > > > this.
> >> > > > >> > > > > > >> > >> Besides
> >> > > > >> > > > > > >> > >> > my proposal to force an early commit once
> >> memory
> >> > > > usage
> >> > > > >> > > > reaches
> >> > > > >> > > > > a
> >> > > > >> > > > > > >> > >> threshold,
> >> > > > >> > > > > > >> > >> > is there any other approach that you might
> >> > suggest
> >> > > > for
> >> > > > >> > > > tackling
> >> > > > >> > > > > > >> this
> >> > > > >> > > > > > >> > >> > problem?
> >> > > > >> > > > > > >> > >> >
> >> > > > >> > > > > > >> > >> > 3. ALOS: I can add in an explicit
> paragraph,
> >> but
> >> > > my
> >> > > > >> > > > assumption
> >> > > > >> > > > > is
> >> > > > >> > > > > > >> that
> >> > > > >> > > > > > >> > >> > since transactional behaviour comes at
> >> little/no
> >> > > > >> cost, that
> >> > > > >> > > > it
> >> > > > >> > > > > > >> should
> >> > > > >> > > > > > >> > be
> >> > > > >> > > > > > >> > >> > available by default on all stores,
> >> irrespective
> >> > > of
> >> > > > >> the
> >> > > > >> > > > > > processing
> >> > > > >> > > > > > >> > mode.
> >> > > > >> > > > > > >> > >> > While ALOS doesn't use transactions, the
> Task
> >> > > itself
> >> > > > >> still
> >> > > > >> > > > > > >> "commits",
> >> > > > >> > > > > > >> > so
> >> > > > >> > > > > > >> > >> > the behaviour should be correct under ALOS
> >> too.
> >> > > I'm
> >> > > > >> not
> >> > > > >> > > > > convinced
> >> > > > >> > > > > > >> that
> >> > > > >> > > > > > >> > >> it's
> >> > > > >> > > > > > >> > >> > worth having both
> >> > transactional/non-transactional
> >> > > > >> stores
> >> > > > >> > > > > > >> available, as
> >> > > > >> > > > > > >> > >> it
> >> > > > >> > > > > > >> > >> > would considerably increase the complexity
> of
> >> > the
> >> > > > >> codebase,
> >> > > > >> > > > for
> >> > > > >> > > > > > >> very
> >> > > > >> > > > > > >> > >> little
> >> > > > >> > > > > > >> > >> > benefit.
> >> > > > >> > > > > > >> > >> >
> >> > > > >> > > > > > >> > >> > 4. Method deprecation: Are you referring to
> >> > > > >> > > > > > >> StateStore#getPosition()?
> >> > > > >> > > > > > >> > >> As I
> >> > > > >> > > > > > >> > >> > understand it, Position contains the
> >> position of
> >> > > the
> >> > > > >> > > *source*
> >> > > > >> > > > > > >> topics,
> >> > > > >> > > > > > >> > >> > whereas the commit offsets would be the
> >> > > *changelog*
> >> > > > >> > > offsets.
> >> > > > >> > > > So
> >> > > > >> > > > > > >> it's
> >> > > > >> > > > > > >> > >> still
> >> > > > >> > > > > > >> > >> > necessary to retain the Position data, as
> >> well
> >> > as
> >> > > > the
> >> > > > >> > > > changelog
> >> > > > >> > > > > > >> > offsets.
> >> > > > >> > > > > > >> > >> > What I meant in the KIP is that Position
> >> offsets
> >> > > are
> >> > > > >> > > > currently
> >> > > > >> > > > > > >> stored
> >> > > > >> > > > > > >> > >> in a
> >> > > > >> > > > > > >> > >> > file, and since we can atomically store
> >> metadata
> >> > > > >> along with
> >> > > > >> > > > the
> >> > > > >> > > > > > >> record
> >> > > > >> > > > > > >> > >> > batch we commit to RocksDB, we can move our
> >> > > Position
> >> > > > >> > > offsets
> >> > > > >> > > > in
> >> > > > >> > > > > > to
> >> > > > >> > > > > > >> > this
> >> > > > >> > > > > > >> > >> > metadata too, and gain the same
> transactional
> >> > > > >> guarantees
> >> > > > >> > > that
> >> > > > >> > > > > we
> >> > > > >> > > > > > >> will
> >> > > > >> > > > > > >> > >> for
> >> > > > >> > > > > > >> > >> > changelog offsets, ensuring that the
> Position
> >> > > > offsets
> >> > > > >> are
> >> > > > >> > > > > > >> consistent
> >> > > > >> > > > > > >> > >> with
> >> > > > >> > > > > > >> > >> > the records that are read from the
> database.
> >> > > > >> > > > > > >> > >> >
> >> > > > >> > > > > > >> > >> > Regards,
> >> > > > >> > > > > > >> > >> > Nick
> >> > > > >> > > > > > >> > >> >
> >> > > > >> > > > > > >> > >> > On Tue, 22 Nov 2022 at 16:25, John Roesler
> <
> >> > > > >> > > > > vvcephei@apache.org>
> >> > > > >> > > > > > >> > wrote:
> >> > > > >> > > > > > >> > >> >
> >> > > > >> > > > > > >> > >> > > Thanks for publishing this alternative,
> >> Nick!
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > The benchmark you mentioned in the
> KIP-844
> >> > > > >> discussion
> >> > > > >> > > seems
> >> > > > >> > > > > > like
> >> > > > >> > > > > > >> a
> >> > > > >> > > > > > >> > >> > > compelling reason to revisit the built-in
> >> > > > >> > > transactionality
> >> > > > >> > > > > > >> > mechanism.
> >> > > > >> > > > > > >> > >> I
> >> > > > >> > > > > > >> > >> > > also appreciate you analysis, showing
> that
> >> for
> >> > > > most
> >> > > > >> use
> >> > > > >> > > > > cases,
> >> > > > >> > > > > > >> the
> >> > > > >> > > > > > >> > >> write
> >> > > > >> > > > > > >> > >> > > batch approach should be just fine.
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > There are a couple of points that would
> >> hold
> >> > me
> >> > > > >> back from
> >> > > > >> > > > > > >> approving
> >> > > > >> > > > > > >> > >> this
> >> > > > >> > > > > > >> > >> > > KIP right now:
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > 1. Loss of coverage for custom stores.
> >> > > > >> > > > > > >> > >> > > The fact that you can plug in a
> >> (relatively)
> >> > > > simple
> >> > > > >> > > > > > >> implementation
> >> > > > >> > > > > > >> > of
> >> > > > >> > > > > > >> > >> the
> >> > > > >> > > > > > >> > >> > > XStateStore interfaces and automagically
> >> get a
> >> > > > >> > > distributed
> >> > > > >> > > > > > >> database
> >> > > > >> > > > > > >> > >> out
> >> > > > >> > > > > > >> > >> > of
> >> > > > >> > > > > > >> > >> > > it is a significant benefit of Kafka
> >> Streams.
> >> > > I'd
> >> > > > >> hate to
> >> > > > >> > > > > lose
> >> > > > >> > > > > > >> it,
> >> > > > >> > > > > > >> > so
> >> > > > >> > > > > > >> > >> it
> >> > > > >> > > > > > >> > >> > > would be better to spend some time and
> >> come up
> >> > > > with
> >> > > > >> a way
> >> > > > >> > > > to
> >> > > > >> > > > > > >> > preserve
> >> > > > >> > > > > > >> > >> > that
> >> > > > >> > > > > > >> > >> > > property. For example, can we provide a
> >> > default
> >> > > > >> > > > > implementation
> >> > > > >> > > > > > of
> >> > > > >> > > > > > >> > >> > > `commit(..)` that re-implements the
> >> existing
> >> > > > >> > > > checkpoint-file
> >> > > > >> > > > > > >> > >> approach? Or
> >> > > > >> > > > > > >> > >> > > perhaps add an `isTransactional()` flag
> to
> >> the
> >> > > > state
> >> > > > >> > > store
> >> > > > >> > > > > > >> interface
> >> > > > >> > > > > > >> > >> so
> >> > > > >> > > > > > >> > >> > > that the runtime can decide whether to
> >> > continue
> >> > > to
> >> > > > >> manage
> >> > > > >> > > > > > >> checkpoint
> >> > > > >> > > > > > >> > >> > files
> >> > > > >> > > > > > >> > >> > > vs delegating transactionality to the
> >> stores?
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > 2. Guarding against OOME
> >> > > > >> > > > > > >> > >> > > I appreciate your analysis, but I don't
> >> think
> >> > > it's
> >> > > > >> > > > sufficient
> >> > > > >> > > > > > to
> >> > > > >> > > > > > >> say
> >> > > > >> > > > > > >> > >> that
> >> > > > >> > > > > > >> > >> > > we will solve the memory problem later if
> >> it
> >> > > > becomes
> >> > > > >> > > > > necessary.
> >> > > > >> > > > > > >> The
> >> > > > >> > > > > > >> > >> > > experience leading to that situation
> would
> >> be
> >> > > > quite
> >> > > > >> bad:
> >> > > > >> > > > > > Imagine,
> >> > > > >> > > > > > >> > you
> >> > > > >> > > > > > >> > >> > > upgrade to AK 3.next, your tests pass, so
> >> you
> >> > > > >> deploy to
> >> > > > >> > > > > > >> production.
> >> > > > >> > > > > > >> > >> That
> >> > > > >> > > > > > >> > >> > > night, you get paged because your app is
> >> now
> >> > > > >> crashing
> >> > > > >> > > with
> >> > > > >> > > > > > >> OOMEs. As
> >> > > > >> > > > > > >> > >> with
> >> > > > >> > > > > > >> > >> > > all OOMEs, you'll have a really hard time
> >> > > finding
> >> > > > >> the
> >> > > > >> > > root
> >> > > > >> > > > > > cause,
> >> > > > >> > > > > > >> > and
> >> > > > >> > > > > > >> > >> > once
> >> > > > >> > > > > > >> > >> > > you do, you won't have a clear path to
> >> resolve
> >> > > the
> >> > > > >> issue.
> >> > > > >> > > > You
> >> > > > >> > > > > > >> could
> >> > > > >> > > > > > >> > >> only
> >> > > > >> > > > > > >> > >> > > tune down the commit interval and cache
> >> buffer
> >> > > > size
> >> > > > >> until
> >> > > > >> > > > you
> >> > > > >> > > > > > >> stop
> >> > > > >> > > > > > >> > >> > getting
> >> > > > >> > > > > > >> > >> > > crashes.
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > FYI, I know of multiple cases where
> people
> >> run
> >> > > EOS
> >> > > > >> with
> >> > > > >> > > > much
> >> > > > >> > > > > > >> larger
> >> > > > >> > > > > > >> > >> > commit
> >> > > > >> > > > > > >> > >> > > intervals to get better batching than the
> >> > > default,
> >> > > > >> so I
> >> > > > >> > > > don't
> >> > > > >> > > > > > >> think
> >> > > > >> > > > > > >> > >> this
> >> > > > >> > > > > > >> > >> > > pathological case would be as rare as you
> >> > > suspect.
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > Given that we already have the rudiments
> >> of an
> >> > > > idea
> >> > > > >> of
> >> > > > >> > > what
> >> > > > >> > > > > we
> >> > > > >> > > > > > >> could
> >> > > > >> > > > > > >> > >> do
> >> > > > >> > > > > > >> > >> > to
> >> > > > >> > > > > > >> > >> > > prevent this downside, we should take the
> >> time
> >> > > to
> >> > > > >> design
> >> > > > >> > > a
> >> > > > >> > > > > > >> solution.
> >> > > > >> > > > > > >> > >> We
> >> > > > >> > > > > > >> > >> > owe
> >> > > > >> > > > > > >> > >> > > it to our users to ensure that awesome
> new
> >> > > > features
> >> > > > >> don't
> >> > > > >> > > > > come
> >> > > > >> > > > > > >> with
> >> > > > >> > > > > > >> > >> > bitter
> >> > > > >> > > > > > >> > >> > > pills unless we can't avoid it.
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > 3. ALOS mode.
> >> > > > >> > > > > > >> > >> > > On the other hand, I didn't see an
> >> indication
> >> > of
> >> > > > how
> >> > > > >> > > stores
> >> > > > >> > > > > > will
> >> > > > >> > > > > > >> be
> >> > > > >> > > > > > >> > >> > > handled under ALOS (aka non-EOS) mode.
> >> > > > >> Theoretically, the
> >> > > > >> > > > > > >> > >> > transactionality
> >> > > > >> > > > > > >> > >> > > of the store and the processing mode are
> >> > > > >> orthogonal. A
> >> > > > >> > > > > > >> transactional
> >> > > > >> > > > > > >> > >> > store
> >> > > > >> > > > > > >> > >> > > would serve ALOS just as well as a
> >> > > > >> non-transactional one
> >> > > > >> > > > (if
> >> > > > >> > > > > > not
> >> > > > >> > > > > > >> > >> better).
> >> > > > >> > > > > > >> > >> > > Under ALOS, though, the default commit
> >> > interval
> >> > > is
> >> > > > >> five
> >> > > > >> > > > > > minutes,
> >> > > > >> > > > > > >> so
> >> > > > >> > > > > > >> > >> the
> >> > > > >> > > > > > >> > >> > > memory issue is far more pressing.
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > As I see it, we have several options to
> >> > resolve
> >> > > > this
> >> > > > >> > > point.
> >> > > > >> > > > > We
> >> > > > >> > > > > > >> could
> >> > > > >> > > > > > >> > >> > > demonstrate that transactional stores
> work
> >> > just
> >> > > > >> fine for
> >> > > > >> > > > ALOS
> >> > > > >> > > > > > >> and we
> >> > > > >> > > > > > >> > >> can
> >> > > > >> > > > > > >> > >> > > therefore just swap over unconditionally.
> >> We
> >> > > could
> >> > > > >> also
> >> > > > >> > > > > disable
> >> > > > >> > > > > > >> the
> >> > > > >> > > > > > >> > >> > > transactional mechanism under ALOS so
> that
> >> > > stores
> >> > > > >> operate
> >> > > > >> > > > > just
> >> > > > >> > > > > > >> the
> >> > > > >> > > > > > >> > >> same
> >> > > > >> > > > > > >> > >> > as
> >> > > > >> > > > > > >> > >> > > they do today when run in ALOS mode.
> >> Finally,
> >> > we
> >> > > > >> could do
> >> > > > >> > > > the
> >> > > > >> > > > > > >> same
> >> > > > >> > > > > > >> > as
> >> > > > >> > > > > > >> > >> in
> >> > > > >> > > > > > >> > >> > > KIP-844 and make transactional stores
> >> opt-in
> >> > > (it'd
> >> > > > >> be
> >> > > > >> > > > better
> >> > > > >> > > > > to
> >> > > > >> > > > > > >> > avoid
> >> > > > >> > > > > > >> > >> the
> >> > > > >> > > > > > >> > >> > > extra opt-in mechanism, but it's a good
> >> > > > >> > > > get-out-of-jail-free
> >> > > > >> > > > > > >> card).
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > 4. (minor point) Deprecation of methods
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > You mentioned that the new `commit`
> 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?
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > Thanks again for the KIP! It's really
> nice
> >> > that
> >> > > > you
> >> > > > >> and
> >> > > > >> > > > Alex
> >> > > > >> > > > > > will
> >> > > > >> > > > > > >> > get
> >> > > > >> > > > > > >> > >> the
> >> > > > >> > > > > > >> > >> > > chance to collaborate on both directions
> so
> >> > that
> >> > > > we
> >> > > > >> can
> >> > > > >> > > get
> >> > > > >> > > > > the
> >> > > > >> > > > > > >> best
> >> > > > >> > > > > > >> > >> > > outcome for Streams and its users.
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > -John
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> > > On 2022/11/21 15:02:15 Nick Telford
> wrote:
> >> > > > >> > > > > > >> > >> > > > 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
> >> > > > >> > > > > > >> > >> > > > <
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> >
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> >
> >> > > > >> > > > > > >>
> >> > > > >> > > > > >
> >> > > > >> > > > >
> >> > > > >> > > >
> >> > > > >> > >
> >> > > > >>
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >> > > > >> > > > > > >> > >> > > >,
> >> > > > >> > > > > > >> > >> > > > so that it can be discussed/reviewed
> >> > > separately
> >> > > > >> from
> >> > > > >> > > > > KIP-844.
> >> > > > >> > > > > > >> > >> > > >
> >> > > > >> > > > > > >> > >> > > > Alex: I'm especially interested in what
> >> you
> >> > > > think!
> >> > > > >> > > > > > >> > >> > > >
> >> > > > >> > > > > > >> > >> > > > I have a nearly complete implementation
> >> of
> >> > the
> >> > > > >> changes
> >> > > > >> > > > > > >> outlined in
> >> > > > >> > > > > > >> > >> this
> >> > > > >> > > > > > >> > >> > > > KIP, please let me know if you'd like
> me
> >> to
> >> > > push
> >> > > > >> them
> >> > > > >> > > for
> >> > > > >> > > > > > >> review
> >> > > > >> > > > > > >> > in
> >> > > > >> > > > > > >> > >> > > advance
> >> > > > >> > > > > > >> > >> > > > of a vote.
> >> > > > >> > > > > > >> > >> > > >
> >> > > > >> > > > > > >> > >> > > > Regards,
> >> > > > >> > > > > > >> > >> > > >
> >> > > > >> > > > > > >> > >> > > > Nick
> >> > > > >> > > > > > >> > >> > > >
> >> > > > >> > > > > > >> > >> > >
> >> > > > >> > > > > > >> > >> >
> >> > > > >> > > > > > >> > >>
> >> > > > >> > > > > > >> > >
> >> > > > >> > > > > > >> >
> >> > > > >> > > > > > >>
> >> > > > >> > > > > > >
> >> > > > >> > > > > >
> >> > > > >> > > > >
> >> > > > >> > > >
> >> > > > >> > >
> >> > > > >>
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi everyone,

I find myself (again) considering removing the offset management from
StateStores, and keeping the old checkpoint file system. The reason is that
the StreamPartitionAssignor directly reads checkpoint files in order to
determine which instance has the most up-to-date copy of the local state.
If we move offsets into the StateStore itself, then we will need to open,
initialize, read offsets and then close each StateStore (that is not
already assigned and open) for which we have *any* local state, on every
rebalance.

Generally, I don't think there are many "orphan" stores like this sitting
around on most instances, but even a few would introduce additional latency
to an already somewhat lengthy rebalance procedure.

I'm leaning towards Colt's (Slack) suggestion of just keeping things in the
checkpoint file(s) for now, and not worrying about the race. The downside
is that we wouldn't be able to remove the explicit RocksDB flush on-commit,
which likely hurts performance.

If anyone has any thoughts or ideas on this subject, I would appreciate it!

Regards,
Nick

On Wed, 19 Apr 2023 at 15:05, Nick Telford <ni...@gmail.com> wrote:

> Hi Colt,
>
> The issue is that if there's a crash between 2 and 3, then you still end
> up with inconsistent data in RocksDB. The only way to guarantee that your
> checkpoint offsets and locally stored data are consistent with each other
> are to atomically commit them, which can be achieved by having the offsets
> stored in RocksDB.
>
> The offsets column family is likely to be extremely small (one
> per-changelog partition + one per Topology input partition for regular
> stores, one per input partition for global stores). So the overhead will be
> minimal.
>
> A major benefit of doing this is that we can remove the explicit calls to
> db.flush(), which forcibly flushes memtables to disk on-commit. It turns
> out, RocksDB memtable flushes are largely dictated by Kafka Streams
> commits, *not* RocksDB configuration, which could be a major source of
> confusion. Atomic checkpointing makes it safe to remove these explicit
> flushes, because it no longer matters exactly when RocksDB flushes data to
> disk; since the data and corresponding checkpoint offsets will always be
> flushed together, the local store is always in a consistent state, and
> on-restart, it can always safely resume restoration from the on-disk
> offsets, restoring the small amount of data that hadn't been flushed when
> the app exited/crashed.
>
> Regards,
> Nick
>
> On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io> wrote:
>
>> Nick,
>>
>> Thanks for your reply. Ack to A) and B).
>>
>> For item C), I see what you're referring to. Your proposed solution will
>> work, so no need to change it. What I was suggesting was that it might be
>> possible to achieve this with only one column family. So long as:
>>
>>    - No uncommitted records (i.e. not committed to the changelog) are
>>    *committed* to the state store, AND
>>    - The Checkpoint offset (which refers to the changelog topic) is less
>>    than or equal to the last written changelog offset in rocksdb
>>
>> I don't see the need to do the full restoration from scratch. My
>> understanding was that prior to 844/892, full restorations were required
>> because there could be uncommitted records written to RocksDB; however,
>> given your use of RocksDB transactions, that can be avoided with the
>> pattern of 1) commit Kafka transaction, 2) commit RocksDB transaction, 3)
>> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <ni...@gmail.com>
>> wrote:
>>
>> > Hi Colt,
>> >
>> > A. I've done my best to de-couple the StateStore stuff from the rest of
>> the
>> > Streams engine. The fact that there will be only one ongoing (write)
>> > transaction at a time is not guaranteed by any API, and is just a
>> > consequence of the way Streams operates. To that end, I tried to ensure
>> the
>> > documentation and guarantees provided by the new APIs are independent of
>> > this incidental behaviour. In practice, you're right, this essentially
>> > refers to "interactive queries", which are technically "read
>> transactions",
>> > even if they don't actually use the transaction API to isolate
>> themselves.
>> >
>> > B. Yes, although not ideal. This is for backwards compatibility,
>> because:
>> >     1) Existing custom StateStore implementations will implement
>> flush(),
>> > and not commit(), but the Streams engine now calls commit(), so those
>> calls
>> > need to be forwarded to flush() for these legacy stores.
>> >     2) Existing StateStore *users*, i.e. outside of the Streams engine
>> > itself, may depend on explicitly calling flush(), so for these cases,
>> > flush() needs to be redirected to call commit().
>> > If anyone has a better way to guarantee compatibility without
>> introducing
>> > this potential recursion loop, I'm open to changes!
>> >
>> > C. This is described in the "Atomic Checkpointing" section. Offsets are
>> > stored in a separate RocksDB column family, which is guaranteed to be
>> > atomically flushed to disk with all other column families. The issue of
>> > checkpoints being written to disk after commit causing inconsistency if
>> it
>> > crashes in between is the reason why, under EOS, checkpoint files are
>> only
>> > written on clean shutdown. This is one of the major causes of "full
>> > restorations", so moving the offsets into a place where they can be
>> > guaranteed to be atomically written with the data they checkpoint
>> allows us
>> > to write the checkpoint offsets *on every commit*, not just on clean
>> > shutdown.
>> >
>> > Regards,
>> > Nick
>> >
>> > On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io> wrote:
>> >
>> > > Nick,
>> > >
>> > > Thank you for continuing this work. I have a few minor clarifying
>> > > questions.
>> > >
>> > > A) "Records written to any transaction are visible to all other
>> > > transactions immediately." I am confused here—I thought there could
>> only
>> > be
>> > > one transaction going on at a time for a given state store given the
>> > > threading model for processing records on a Task. Do you mean
>> Interactive
>> > > Queries by "other transactions"? (If so, then everything makes sense—I
>> > > thought that since IQ were read-only then they didn't count as
>> > > transactions).
>> > >
>> > > B) Is it intentional that the default implementations of the flush()
>> and
>> > > commit() methods in the StateStore class refer to each other in some
>> sort
>> > > of unbounded recursion?
>> > >
>> > > C) How will the getCommittedOffset() method work? At first I thought
>> the
>> > > way to do it would be using a special key in the RocksDB store to
>> store
>> > the
>> > > offset, and committing that with the transaction. But upon second
>> > thought,
>> > > since restoration from the changelog is an idempotent procedure, I
>> think
>> > it
>> > > would be fine to 1) commit the RocksDB transaction and then 2) write
>> the
>> > > offset to disk in a checkpoint file. If there is a crash between 1)
>> and
>> > 2),
>> > > I think the only downside is 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 <ni...@gmail.com>
>> > > wrote:
>> > >
>> > > > Hi everyone,
>> > > >
>> > > > I've updated the KIP to reflect the latest version of the design:
>> > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> > > >
>> > > > There are several changes in there that reflect feedback from this
>> > > thread,
>> > > > and there's a new section and a bunch of interface changes relating
>> to
>> > > > Atomic Checkpointing, which is the final piece of the puzzle to
>> making
>> > > > everything robust.
>> > > >
>> > > > Let me know what you think!
>> > > >
>> > > > Regards,
>> > > > Nick
>> > > >
>> > > > On Tue, 3 Jan 2023 at 11:33, Nick Telford <ni...@gmail.com>
>> > > wrote:
>> > > >
>> > > > > Hi Lucas,
>> > > > >
>> > > > > Thanks for looking over my KIP.
>> > > > >
>> > > > > A) The bound is per-instance, not per-Task. This was a typo in the
>> > KIP
>> > > > > that I've now corrected. It was originally per-Task, but I
>> changed it
>> > > to
>> > > > > per-instance for exactly the reason you highlighted.
>> > > > > B) It's worth noting that transactionality is only enabled under
>> EOS,
>> > > and
>> > > > > in the default mode of operation (ALOS), there should be no
>> change in
>> > > > > behavior at all. I think, under EOS, we can mitigate the impact on
>> > > users
>> > > > by
>> > > > > sufficiently low default values for the memory bound
>> configuration. I
>> > > > > understand your hesitation to include a significant change of
>> > > behaviour,
>> > > > > especially in a minor release, but I suspect that most users will
>> > > prefer
>> > > > > the memory impact (under EOS) to the existing behaviour of
>> frequent
>> > > state
>> > > > > restorations! If this is a problem, the changes can wait until the
>> > next
>> > > > > major release. I'll be running a patched version of streams in
>> > > production
>> > > > > with these changes as soon as they're ready, so it won't disrupt
>> me
>> > :-D
>> > > > > C) The main purpose of this sentence was just to note that some
>> > changes
>> > > > > will need to be made to the way Segments are handled in order to
>> > ensure
>> > > > > they also benefit from transactions. At the time I wrote it, I
>> hadn't
>> > > > > figured out the specific changes necessary, so it was deliberately
>> > > vague.
>> > > > > This is the one outstanding problem I'm currently working on, and
>> > I'll
>> > > > > update this section with more detail once I have figured out the
>> > exact
>> > > > > changes required.
>> > > > > D) newTransaction() provides the necessary isolation guarantees.
>> > While
>> > > > > the RocksDB implementation of transactions doesn't technically
>> *need*
>> > > > > read-only users to call newTransaction(), other implementations
>> > (e.g. a
>> > > > > hypothetical PostgresStore) may require it. Calling
>> newTransaction()
>> > > when
>> > > > > no transaction is necessary is essentially free, as it will just
>> > return
>> > > > > this.
>> > > > >
>> > > > > I didn't do any profiling of the KIP-844 PoC, but I think it
>> should
>> > be
>> > > > > fairly obvious where the performance problems stem from: writes
>> under
>> > > > > KIP-844 require 3 extra memory-copies: 1 to encode it with the
>> > > > > tombstone/record flag, 1 to decode it from the tombstone/record
>> flag,
>> > > > and 1
>> > > > > to copy the record from the "temporary" store to the "main" store,
>> > when
>> > > > the
>> > > > > transaction commits. The different approach taken by KIP-869
>> should
>> > > > perform
>> > > > > much better, as it avoids all these copies, and may actually
>> perform
>> > > > > slightly better than trunk, due to batched writes in RocksDB
>> > performing
>> > > > > better than non-batched writes.[1]
>> > > > >
>> > > > > Regards,
>> > > > > Nick
>> > > > >
>> > > > > 1:
>> > > >
>> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>> > > > >
>> > > > > On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
>> lbrutschy@confluent.io
>> > > > .invalid>
>> > > > > wrote:
>> > > > >
>> > > > >> Hi Nick,
>> > > > >>
>> > > > >> I'm just starting to read up on the whole discussion about
>> KIP-892
>> > and
>> > > > >> KIP-844. Thanks a lot for your work on this, I do think
>> > > > >> `WriteBatchWithIndex` may be the way to go here. I do have some
>> > > > >> questions about the latest draft.
>> > > > >>
>> > > > >>  A) If I understand correctly, you propose to put a bound on the
>> > > > >> (native) memory consumed by each task. However, I wonder if this
>> is
>> > > > >> sufficient if we have temporary imbalances in the cluster. For
>> > > > >> example, depending on the timing of rebalances during a cluster
>> > > > >> restart, it could happen that a single streams node is assigned a
>> > lot
>> > > > >> more tasks than expected. With your proposed change, this would
>> mean
>> > > > >> that the memory required by this one node could be a multiple of
>> > what
>> > > > >> is required during normal operation. I wonder if it wouldn't be
>> > safer
>> > > > >> to put a global bound on the memory use, across all tasks.
>> > > > >>  B) Generally, the memory concerns still give me the feeling that
>> > this
>> > > > >> should not be enabled by default for all users in a minor
>> release.
>> > > > >>  C) In section "Transaction Management": the sentence "A similar
>> > > > >> analogue will be created to automatically manage `Segment`
>> > > > >> transactions.". Maybe this is just me lacking some background,
>> 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 you to find out why the KIP-844 PoC is 20x slower
>> (e.g.
>> > > > >> by providing a flame graph), that would be quite interesting.
>> > > > >>
>> > > > >> Cheers,
>> > > > >> Lucas
>> > > > >>
>> > > > >> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
>> > nick.telford@gmail.com>
>> > > > >> wrote:
>> > > > >> >
>> > > > >> > Hi everyone,
>> > > > >> >
>> > > > >> > I've updated the KIP with a more detailed design, which
>> reflects
>> > the
>> > > > >> > implementation I've been working on:
>> > > > >> >
>> > > > >>
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> > > > >> >
>> > > > >> > This new design should address the outstanding points already
>> made
>> > > in
>> > > > >> the
>> > > > >> > thread.
>> > > > >> >
>> > > > >> > Please let me know if there are areas that are unclear or need
>> > more
>> > > > >> > clarification.
>> > > > >> >
>> > > > >> > I have a (nearly) working implementation. I'm confident that
>> the
>> > > > >> remaining
>> > > > >> > work (making Segments behave) will not impact the documented
>> > design.
>> > > > >> >
>> > > > >> > Regards,
>> > > > >> >
>> > > > >> > Nick
>> > > > >> >
>> > > > >> > On Tue, 6 Dec 2022 at 19:24, Colt McNealy <colt@littlehorse.io
>> >
>> > > > wrote:
>> > > > >> >
>> > > > >> > > Nick,
>> > > > >> > >
>> > > > >> > > Thank you for the reply; that makes sense. I was hoping that,
>> > > since
>> > > > >> reading
>> > > > >> > > uncommitted records from IQ in EOS isn't part of the
>> documented
>> > > API,
>> > > > >> maybe
>> > > > >> > > you *wouldn't* have to wait for the next major release to
>> make
>> > > that
>> > > > >> change;
>> > > > >> > > but given 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 <
>> > > > nick.telford@gmail.com>
>> > > > >> > > wrote:
>> > > > >> > >
>> > > > >> > > > Hi Colt,
>> > > > >> > > >
>> > > > >> > > > 10: Yes, I agree it's not ideal. I originally intended to
>> try
>> > to
>> > > > >> keep the
>> > > > >> > > > behaviour unchanged as much as possible, otherwise we'd
>> have
>> > to
>> > > > >> wait for
>> > > > >> > > a
>> > > > >> > > > major version release to land these changes.
>> > > > >> > > > 20: Good point, ALOS doesn't need the same level of
>> guarantee,
>> > > and
>> > > > >> the
>> > > > >> > > > typically longer commit intervals would be problematic when
>> > > > reading
>> > > > >> only
>> > > > >> > > > "committed" records.
>> > > > >> > > >
>> > > > >> > > > I've been away for 5 days recovering from minor surgery,
>> but I
>> > > > >> spent a
>> > > > >> > > > considerable amount of that time working through ideas for
>> > > > possible
>> > > > >> > > > solutions in my head. I think your suggestion of keeping
>> ALOS
>> > > > >> as-is, but
>> > > > >> > > > buffering writes for EOS is the right path forwards,
>> although
>> > I
>> > > > >> have a
>> > > > >> > > > solution that both expands on this, and provides for some
>> more
>> > > > >> formal
>> > > > >> > > > guarantees.
>> > > > >> > > >
>> > > > >> > > > Essentially, adding support to KeyValueStores for
>> > > "Transactions",
>> > > > >> with
>> > > > >> > > > clearly defined IsolationLevels. Using "Read Committed"
>> when
>> > > under
>> > > > >> EOS,
>> > > > >> > > and
>> > > > >> > > > "Read Uncommitted" under ALOS.
>> > > > >> > > >
>> > > > >> > > > The nice thing about this approach is that it gives us much
>> > more
>> > > > >> clearly
>> > > > >> > > > defined isolation behaviour that can be properly
>> documented to
>> > > > >> ensure
>> > > > >> > > users
>> > > > >> > > > know what to expect.
>> > > > >> > > >
>> > > > >> > > > I'm still working out the kinks in the design, and will
>> update
>> > > the
>> > > > >> KIP
>> > > > >> > > when
>> > > > >> > > > I have something. The main struggle is trying to implement
>> > this
>> > > > >> without
>> > > > >> > > > making any major changes to the existing interfaces or
>> > breaking
>> > > > >> existing
>> > > > >> > > > implementations, because currently everything expects to
>> > operate
>> > > > >> directly
>> > > > >> > > > on a StateStore, and not a Transaction of that store. I
>> think
>> > > I'm
>> > > > >> getting
>> > > > >> > > > close, although sadly I won't be able to progress much
>> until
>> > > next
>> > > > >> week
>> > > > >> > > due
>> > > > >> > > > to some work commitments.
>> > > > >> > > >
>> > > > >> > > > Regards,
>> > > > >> > > > Nick
>> > > > >> > > >
>> > > > >> > > > On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
>> > colt@littlehorse.io>
>> > > > >> wrote:
>> > > > >> > > >
>> > > > >> > > > > Nick,
>> > > > >> > > > >
>> > > > >> > > > > Thank you for the explanation, and also for the updated
>> > KIP. I
>> > > > am
>> > > > >> quite
>> > > > >> > > > > eager for this improvement to be released as it would
>> > greatly
>> > > > >> reduce
>> > > > >> > > the
>> > > > >> > > > > operational difficulties of EOS streams apps.
>> > > > >> > > > >
>> > > > >> > > > > Two questions:
>> > > > >> > > > >
>> > > > >> > > > > 10)
>> > > > >> > > > > >When reading records, we will use the
>> > > > >> > > > > WriteBatchWithIndex#getFromBatchAndDB
>> > > > >> > > > >  and WriteBatchWithIndex#newIteratorWithBase utilities in
>> > > order
>> > > > to
>> > > > >> > > ensure
>> > > > >> > > > > that uncommitted writes are available to query.
>> > > > >> > > > > Why do extra work to enable the reading of uncommitted
>> > writes
>> > > > >> during
>> > > > >> > > IQ?
>> > > > >> > > > > Code complexity aside, reading uncommitted writes is, in
>> my
>> > > > >> opinion, a
>> > > > >> > > > > minor flaw in EOS IQ; it would be very nice to have the
>> > > > guarantee
>> > > > >> that,
>> > > > >> > > > > with EOS, IQ only reads committed records. In order to
>> avoid
>> > > > dirty
>> > > > >> > > reads,
>> > > > >> > > > > one currently must query a standby replica (but this
>> still
>> > > > doesn't
>> > > > >> > > fully
>> > > > >> > > > > guarantee monotonic reads).
>> > > > >> > > > >
>> > > > >> > > > > 20) Is it also necessary to enable this optimization on
>> ALOS
>> > > > >> stores?
>> > > > >> > > The
>> > > > >> > > > > motivation of KIP-844 was mainly to reduce the need to
>> > restore
>> > > > >> state
>> > > > >> > > from
>> > > > >> > > > > scratch on unclean EOS shutdowns; with ALOS it was
>> > acceptable
>> > > to
>> > > > >> accept
>> > > > >> > > > > that there may have been uncommitted writes on disk. On a
>> > side
>> > > > >> note, if
>> > > > >> > > > you
>> > > > >> > > > > enable this type of store on ALOS processors, the
>> community
>> > > > would
>> > > > >> > > > > 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 <
>> > > > >> nick.telford@gmail.com>
>> > > > >> > > > > wrote:
>> > > > >> > > > >
>> > > > >> > > > > > Hi everyone,
>> > > > >> > > > > >
>> > > > >> > > > > > I've drastically reduced the scope of this KIP to no
>> > longer
>> > > > >> include
>> > > > >> > > the
>> > > > >> > > > > > StateStore management of checkpointing. This can be
>> added
>> > > as a
>> > > > >> KIP
>> > > > >> > > > later
>> > > > >> > > > > on
>> > > > >> > > > > > to further optimize the consistency and performance of
>> > state
>> > > > >> stores.
>> > > > >> > > > > >
>> > > > >> > > > > > I've also added a section discussing some of the
>> concerns
>> > > > 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 2022 at 16:36, Nick Telford <
>> > > > >> nick.telford@gmail.com>
>> > > > >> > > > > wrote:
>> > > > >> > > > > >
>> > > > >> > > > > > > Hi Colt,
>> > > > >> > > > > > >
>> > > > >> > > > > > > I didn't do any profiling, but the 844
>> implementation:
>> > > > >> > > > > > >
>> > > > >> > > > > > >    - Writes uncommitted records to a temporary
>> RocksDB
>> > > > >> instance
>> > > > >> > > > > > >       - Since tombstones need to be flagged, all
>> record
>> > > > >> values are
>> > > > >> > > > > > >       prefixed with a value/tombstone marker. This
>> > > > >> necessitates a
>> > > > >> > > > > memory
>> > > > >> > > > > > copy.
>> > > > >> > > > > > >    - On-commit, iterates all records in this
>> temporary
>> > > > >> instance and
>> > > > >> > > > > > >    writes them to the main RocksDB store.
>> > > > >> > > > > > >    - While iterating, the value/tombstone marker
>> needs
>> > to
>> > > be
>> > > > >> parsed
>> > > > >> > > > and
>> > > > >> > > > > > >    the real value extracted. This necessitates
>> another
>> > > > memory
>> > > > >> copy.
>> > > > >> > > > > > >
>> > > > >> > > > > > > My guess is that the cost of iterating the temporary
>> > > RocksDB
>> > > > >> store
>> > > > >> > > is
>> > > > >> > > > > the
>> > > > >> > > > > > > major factor, with the 2 extra memory copies
>> per-Record
>> > > > >> > > contributing
>> > > > >> > > > a
>> > > > >> > > > > > > significant amount too.
>> > > > >> > > > > > >
>> > > > >> > > > > > > Regards,
>> > > > >> > > > > > > Nick
>> > > > >> > > > > > >
>> > > > >> > > > > > > On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
>> > > > >> colt@littlehorse.io>
>> > > > >> > > > > wrote:
>> > > > >> > > > > > >
>> > > > >> > > > > > >> Hi all,
>> > > > >> > > > > > >>
>> > > > >> > > > > > >> Out of curiosity, why does the performance of the
>> store
>> > > > >> degrade so
>> > > > >> > > > > > >> significantly with the 844 implementation? I
>> wouldn't
>> > be
>> > > > too
>> > > > >> > > > surprised
>> > > > >> > > > > > by
>> > > > >> > > > > > >> a
>> > > > >> > > > > > >> 50-60% drop (caused by each record being written
>> > twice),
>> > > > but
>> > > > >> 96%
>> > > > >> > > is
>> > > > >> > > > > > >> extreme.
>> > > > >> > > > > > >>
>> > > > >> > > > > > >> The only thing I can think of which could create
>> such a
>> > > > >> bottleneck
>> > > > >> > > > > 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 Nick Telford <
>> > > > >> > > > nick.telford@gmail.com>
>> > > > >> > > > > > >> wrote:
>> > > > >> > > > > > >>
>> > > > >> > > > > > >> > Hi everyone,
>> > > > >> > > > > > >> >
>> > > > >> > > > > > >> > I've updated the KIP to resolve all the points
>> that
>> > > have
>> > > > >> been
>> > > > >> > > > raised
>> > > > >> > > > > > so
>> > > > >> > > > > > >> > far, with one exception: the ALOS default commit
>> > > interval
>> > > > >> of 5
>> > > > >> > > > > minutes
>> > > > >> > > > > > >> is
>> > > > >> > > > > > >> > likely to cause WriteBatchWithIndex memory to grow
>> > too
>> > > > >> large.
>> > > > >> > > > > > >> >
>> > > > >> > > > > > >> > There's a couple of different things I can think
>> of
>> > to
>> > > > >> solve
>> > > > >> > > this:
>> > > > >> > > > > > >> >
>> > > > >> > > > > > >> >    - We already have a memory/record limit in the
>> KIP
>> > > to
>> > > > >> prevent
>> > > > >> > > > OOM
>> > > > >> > > > > > >> >    errors. Should we choose a default value for
>> > these?
>> > > My
>> > > > >> > > concern
>> > > > >> > > > > here
>> > > > >> > > > > > >> is
>> > > > >> > > > > > >> > that
>> > > > >> > > > > > >> >    anything we choose might seem rather
>> arbitrary. We
>> > > > could
>> > > > >> > > change
>> > > > >> > > > > > >> >    its behaviour such that under ALOS, it only
>> > triggers
>> > > > the
>> > > > >> > > commit
>> > > > >> > > > > of
>> > > > >> > > > > > >> the
>> > > > >> > > > > > >> >    StateStore, but under EOS, it triggers a
>> commit of
>> > > the
>> > > > >> Kafka
>> > > > >> > > > > > >> > transaction.
>> > > > >> > > > > > >> >    - We could introduce a separate `
>> > > > checkpoint.interval.ms`
>> > > > >> to
>> > > > >> > > > > allow
>> > > > >> > > > > > >> ALOS
>> > > > >> > > > > > >> >    to commit the StateStores more frequently than
>> the
>> > > > >> general
>> > > > >> > > > > > >> >    commit.interval.ms? My concern here is that
>> the
>> > > > >> semantics of
>> > > > >> > > > > this
>> > > > >> > > > > > >> > config
>> > > > >> > > > > > >> >    would depend 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 <
>> > > > >> > > > nick.telford@gmail.com>
>> > > > >> > > > > > >> wrote:
>> > > > >> > > > > > >> >
>> > > > >> > > > > > >> > > Hi Alex,
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > > Thanks for the feedback.
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > > I've updated the discussion of OOM issues by
>> > > describing
>> > > > >> how
>> > > > >> > > > we'll
>> > > > >> > > > > > >> handle
>> > > > >> > > > > > >> > > it. Here's the new text:
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > > To mitigate this, we will automatically force a
>> > Task
>> > > > >> commit if
>> > > > >> > > > the
>> > > > >> > > > > > >> total
>> > > > >> > > > > > >> > >> uncommitted records returned by
>> > > > >> > > > > > >> > >> StateStore#approximateNumUncommittedEntries()
>> > > > exceeds a
>> > > > >> > > > > threshold,
>> > > > >> > > > > > >> > >> configured by
>> > > max.uncommitted.state.entries.per.task;
>> > > > >> or the
>> > > > >> > > > > total
>> > > > >> > > > > > >> > >> memory used for buffering uncommitted records
>> > > returned
>> > > > >> by
>> > > > >> > > > > > >> > >> StateStore#approximateNumUncommittedBytes()
>> > exceeds
>> > > > the
>> > > > >> > > > threshold
>> > > > >> > > > > > >> > >> configured by
>> > max.uncommitted.state.bytes.per.task.
>> > > > >> This will
>> > > > >> > > > > > roughly
>> > > > >> > > > > > >> > >> bound the memory required per-Task for
>> buffering
>> > > > >> uncommitted
>> > > > >> > > > > > records,
>> > > > >> > > > > > >> > >> irrespective of the commit.interval.ms, and
>> will
>> > > > >> effectively
>> > > > >> > > > > bound
>> > > > >> > > > > > >> the
>> > > > >> > > > > > >> > >> number of records that will need to be
>> restored in
>> > > the
>> > > > >> event
>> > > > >> > > > of a
>> > > > >> > > > > > >> > failure.
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > > These limits will be checked in
>> StreamTask#process
>> > > and
>> > > > a
>> > > > >> > > > premature
>> > > > >> > > > > > >> commit
>> > > > >> > > > > > >> > >> will be requested via Task#requestCommit().
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > > Note that these new methods provide default
>> > > > >> implementations
>> > > > >> > > that
>> > > > >> > > > > > >> ensure
>> > > > >> > > > > > >> > >> existing custom stores and non-transactional
>> > stores
>> > > > >> (e.g.
>> > > > >> > > > > > >> > >> InMemoryKeyValueStore) do not force any early
>> > > commits.
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > > I've chosen to have the StateStore expose
>> > > > approximations
>> > > > >> of
>> > > > >> > > its
>> > > > >> > > > > > buffer
>> > > > >> > > > > > >> > > size/count instead of opaquely requesting a
>> commit
>> > in
>> > > > >> order to
>> > > > >> > > > > > >> delegate
>> > > > >> > > > > > >> > the
>> > > > >> > > > > > >> > > decision making to the Task itself. This enables
>> > > Tasks
>> > > > >> to look
>> > > > >> > > > at
>> > > > >> > > > > > >> *all*
>> > > > >> > > > > > >> > of
>> > > > >> > > > > > >> > > their StateStores, and determine whether an
>> early
>> > > > commit
>> > > > >> is
>> > > > >> > > > > > necessary.
>> > > > >> > > > > > >> > > Notably, it enables pre-Task thresholds,
>> instead of
>> > > > >> per-Store,
>> > > > >> > > > > which
>> > > > >> > > > > > >> > > prevents Tasks with many StateStores from using
>> > much
>> > > > more
>> > > > >> > > memory
>> > > > >> > > > > > than
>> > > > >> > > > > > >> > Tasks
>> > > > >> > > > > > >> > > with one StateStore. This makes sense, since
>> > commits
>> > > > are
>> > > > >> done
>> > > > >> > > > > > by-Task,
>> > > > >> > > > > > >> > not
>> > > > >> > > > > > >> > > by-Store.
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > > Prizes* for anyone who can come up with a better
>> > name
>> > > > >> for the
>> > > > >> > > > new
>> > > > >> > > > > > >> config
>> > > > >> > > > > > >> > > properties!
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > > Thanks for pointing out the potential
>> performance
>> > > > issues
>> > > > >> of
>> > > > >> > > > WBWI.
>> > > > >> > > > > > From
>> > > > >> > > > > > >> > the
>> > > > >> > > > > > >> > > benchmarks that user posted[1], it looks like
>> WBWI
>> > > > still
>> > > > >> > > > performs
>> > > > >> > > > > > >> > > considerably better than individual puts, which
>> is
>> > > the
>> > > > >> > > existing
>> > > > >> > > > > > >> design,
>> > > > >> > > > > > >> > so
>> > > > >> > > > > > >> > > I'd actually expect a performance boost from
>> WBWI,
>> > > just
>> > > > >> not as
>> > > > >> > > > > great
>> > > > >> > > > > > >> as
>> > > > >> > > > > > >> > > we'd get from a plain WriteBatch. This does
>> suggest
>> > > > that
>> > > > >> a
>> > > > >> > > good
>> > > > >> > > > > > >> > > optimization would be to use a regular
>> WriteBatch
>> > for
>> > > > >> > > > restoration
>> > > > >> > > > > > (in
>> > > > >> > > > > > >> > > RocksDBStore#restoreBatch), since we know that
>> > those
>> > > > >> records
>> > > > >> > > > will
>> > > > >> > > > > > >> never
>> > > > >> > > > > > >> > be
>> > > > >> > > > > > >> > > queried before they're committed.
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > > 1:
>> > > > >> > > > > > >> >
>> > > > >> > > > > >
>> > > > >> > > >
>> > > > >>
>> > >
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > > * Just kidding, no prizes, sadly.
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > > On Wed, 23 Nov 2022 at 12:28, Alexander
>> Sorokoumov
>> > > > >> > > > > > >> > > <as...@confluent.io.invalid> wrote:
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> > >> 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 to keep the current
>> > > checkpointing
>> > > > >> as a
>> > > > >> > > > > > fallback
>> > > > >> > > > > > >> > >> mechanism. We want to keep existing users'
>> > workflows
>> > > > >> intact
>> > > > >> > > if
>> > > > >> > > > we
>> > > > >> > > > > > >> can. A
>> > > > >> > > > > > >> > >> non-intrusive way would be to add a separate
>> > > > StateStore
>> > > > >> > > method,
>> > > > >> > > > > > say,
>> > > > >> > > > > > >> > >> StateStore#managesCheckpointing(), that
>> controls
>> > > > >> whether the
>> > > > >> > > > > state
>> > > > >> > > > > > >> store
>> > > > >> > > > > > >> > >> implementation owns checkpointing.
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> > >> I think that a solution to the transactional
>> > writes
>> > > > >> should
>> > > > >> > > > > address
>> > > > >> > > > > > >> the
>> > > > >> > > > > > >> > >> OOMEs. One possible way to address that is to
>> wire
>> > > > >> > > StateStore's
>> > > > >> > > > > > >> commit
>> > > > >> > > > > > >> > >> request by adding, say, StateStore#commitNeeded
>> > that
>> > > > is
>> > > > >> > > checked
>> > > > >> > > > > in
>> > > > >> > > > > > >> > >> StreamTask#commitNeeded via the corresponding
>> > > > >> > > > > > ProcessorStateManager.
>> > > > >> > > > > > >> > With
>> > > > >> > > > > > >> > >> that change, RocksDBStore will have to track
>> the
>> > > > current
>> > > > >> > > > > > transaction
>> > > > >> > > > > > >> > size
>> > > > >> > > > > > >> > >> and request a commit when the size goes over a
>> > > > >> (configurable)
>> > > > >> > > > > > >> threshold.
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> > >> AFAIU WriteBatchWithIndex might perform
>> > > significantly
>> > > > >> slower
>> > > > >> > > > than
>> > > > >> > > > > > >> > non-txn
>> > > > >> > > > > > >> > >> puts as the batch size grows [1]. We should
>> have a
>> > > > >> > > > configuration
>> > > > >> > > > > to
>> > > > >> > > > > > >> fall
>> > > > >> > > > > > >> > >> back to the current behavior (and/or disable
>> txn
>> > > > stores
>> > > > >> for
>> > > > >> > > > ALOS)
>> > > > >> > > > > > >> unless
>> > > > >> > > > > > >> > >> the benchmarks show negligible overhead for
>> longer
>> > > > >> commits /
>> > > > >> > > > > > >> > large-enough
>> > > > >> > > > > > >> > >> batch sizes.
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> > >> If you prefer to keep the KIP smaller, I would
>> > > rather
>> > > > >> cut out
>> > > > >> > > > > > >> > >> state-store-managed checkpointing rather than
>> > proper
>> > > > >> OOMe
>> > > > >> > > > > handling
>> > > > >> > > > > > >> and
>> > > > >> > > > > > >> > >> being able to switch to non-txn behavior. The
>> > > > >> checkpointing
>> > > > >> > > 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 configuration parts as well as review
>> and
>> > > > test
>> > > > >> the
>> > > > >> > > > > patch.
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> > >> Best,
>> > > > >> > > > > > >> > >> Alex
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> > >> 1.
>> https://github.com/facebook/rocksdb/issues/608
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> > >> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
>> > > > >> > > > > > nick.telford@gmail.com
>> > > > >> > > > > > >> >
>> > > > >> > > > > > >> > >> wrote:
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> > >> > Hi John,
>> > > > >> > > > > > >> > >> >
>> > > > >> > > > > > >> > >> > Thanks for the review and feedback!
>> > > > >> > > > > > >> > >> >
>> > > > >> > > > > > >> > >> > 1. Custom Stores: I've been mulling over this
>> > > > problem
>> > > > >> > > myself.
>> > > > >> > > > > As
>> > > > >> > > > > > it
>> > > > >> > > > > > >> > >> stands,
>> > > > >> > > > > > >> > >> > custom stores would essentially lose
>> > checkpointing
>> > > > >> with no
>> > > > >> > > > > > >> indication
>> > > > >> > > > > > >> > >> that
>> > > > >> > > > > > >> > >> > they're expected to make changes, besides a
>> line
>> > > in
>> > > > >> the
>> > > > >> > > > release
>> > > > >> > > > > > >> > notes. I
>> > > > >> > > > > > >> > >> > agree that the best solution would be to
>> > provide a
>> > > > >> default
>> > > > >> > > > that
>> > > > >> > > > > > >> > >> checkpoints
>> > > > >> > > > > > >> > >> > to a file. The one thing I would change is
>> that
>> > > the
>> > > > >> > > > > checkpointing
>> > > > >> > > > > > >> is
>> > > > >> > > > > > >> > to
>> > > > >> > > > > > >> > >> a
>> > > > >> > > > > > >> > >> > store-local file, instead of a per-Task file.
>> > This
>> > > > >> way the
>> > > > >> > > > > > >> StateStore
>> > > > >> > > > > > >> > >> still
>> > > > >> > > > > > >> > >> > technically owns its own checkpointing (via a
>> > > > default
>> > > > >> > > > > > >> implementation),
>> > > > >> > > > > > >> > >> and
>> > > > >> > > > > > >> > >> > the StateManager/Task execution engine
>> doesn't
>> > > need
>> > > > >> to know
>> > > > >> > > > > > >> anything
>> > > > >> > > > > > >> > >> about
>> > > > >> > > > > > >> > >> > checkpointing, which greatly simplifies some
>> of
>> > > the
>> > > > >> logic.
>> > > > >> > > > > > >> > >> >
>> > > > >> > > > > > >> > >> > 2. OOME errors: The main reasons why I didn't
>> > > > explore
>> > > > >> a
>> > > > >> > > > > solution
>> > > > >> > > > > > to
>> > > > >> > > > > > >> > >> this is
>> > > > >> > > > > > >> > >> > a) to keep this KIP as simple as possible,
>> and
>> > b)
>> > > > >> because
>> > > > >> > > I'm
>> > > > >> > > > > not
>> > > > >> > > > > > >> > >> exactly
>> > > > >> > > > > > >> > >> > how to signal that a Task should commit
>> > > prematurely.
>> > > > >> I'm
>> > > > >> > > > > > confident
>> > > > >> > > > > > >> > it's
>> > > > >> > > > > > >> > >> > possible, and I think it's worth adding a
>> > section
>> > > on
>> > > > >> > > handling
>> > > > >> > > > > > this.
>> > > > >> > > > > > >> > >> Besides
>> > > > >> > > > > > >> > >> > my proposal to force an early commit once
>> memory
>> > > > usage
>> > > > >> > > > reaches
>> > > > >> > > > > a
>> > > > >> > > > > > >> > >> threshold,
>> > > > >> > > > > > >> > >> > is there any other approach that you might
>> > suggest
>> > > > for
>> > > > >> > > > tackling
>> > > > >> > > > > > >> this
>> > > > >> > > > > > >> > >> > problem?
>> > > > >> > > > > > >> > >> >
>> > > > >> > > > > > >> > >> > 3. ALOS: I can add in an explicit paragraph,
>> but
>> > > my
>> > > > >> > > > assumption
>> > > > >> > > > > is
>> > > > >> > > > > > >> that
>> > > > >> > > > > > >> > >> > since transactional behaviour comes at
>> little/no
>> > > > >> cost, that
>> > > > >> > > > it
>> > > > >> > > > > > >> should
>> > > > >> > > > > > >> > be
>> > > > >> > > > > > >> > >> > available by default on all stores,
>> irrespective
>> > > of
>> > > > >> the
>> > > > >> > > > > > processing
>> > > > >> > > > > > >> > mode.
>> > > > >> > > > > > >> > >> > While ALOS doesn't use transactions, the Task
>> > > itself
>> > > > >> still
>> > > > >> > > > > > >> "commits",
>> > > > >> > > > > > >> > so
>> > > > >> > > > > > >> > >> > the behaviour should be correct under ALOS
>> too.
>> > > I'm
>> > > > >> not
>> > > > >> > > > > convinced
>> > > > >> > > > > > >> that
>> > > > >> > > > > > >> > >> it's
>> > > > >> > > > > > >> > >> > worth having both
>> > transactional/non-transactional
>> > > > >> stores
>> > > > >> > > > > > >> available, as
>> > > > >> > > > > > >> > >> it
>> > > > >> > > > > > >> > >> > would considerably increase the complexity of
>> > the
>> > > > >> codebase,
>> > > > >> > > > for
>> > > > >> > > > > > >> very
>> > > > >> > > > > > >> > >> little
>> > > > >> > > > > > >> > >> > benefit.
>> > > > >> > > > > > >> > >> >
>> > > > >> > > > > > >> > >> > 4. Method deprecation: Are you referring to
>> > > > >> > > > > > >> StateStore#getPosition()?
>> > > > >> > > > > > >> > >> As I
>> > > > >> > > > > > >> > >> > understand it, Position contains the
>> position of
>> > > the
>> > > > >> > > *source*
>> > > > >> > > > > > >> topics,
>> > > > >> > > > > > >> > >> > whereas the commit offsets would be the
>> > > *changelog*
>> > > > >> > > offsets.
>> > > > >> > > > So
>> > > > >> > > > > > >> it's
>> > > > >> > > > > > >> > >> still
>> > > > >> > > > > > >> > >> > necessary to retain the Position data, as
>> well
>> > as
>> > > > the
>> > > > >> > > > changelog
>> > > > >> > > > > > >> > offsets.
>> > > > >> > > > > > >> > >> > What I meant in the KIP is that Position
>> offsets
>> > > are
>> > > > >> > > > currently
>> > > > >> > > > > > >> stored
>> > > > >> > > > > > >> > >> in a
>> > > > >> > > > > > >> > >> > file, and since we can atomically store
>> metadata
>> > > > >> along with
>> > > > >> > > > the
>> > > > >> > > > > > >> record
>> > > > >> > > > > > >> > >> > batch we commit to RocksDB, we can move our
>> > > Position
>> > > > >> > > offsets
>> > > > >> > > > in
>> > > > >> > > > > > to
>> > > > >> > > > > > >> > this
>> > > > >> > > > > > >> > >> > metadata too, and gain the same transactional
>> > > > >> guarantees
>> > > > >> > > that
>> > > > >> > > > > we
>> > > > >> > > > > > >> will
>> > > > >> > > > > > >> > >> for
>> > > > >> > > > > > >> > >> > changelog offsets, ensuring that the Position
>> > > > offsets
>> > > > >> are
>> > > > >> > > > > > >> consistent
>> > > > >> > > > > > >> > >> with
>> > > > >> > > > > > >> > >> > the records that are read from the database.
>> > > > >> > > > > > >> > >> >
>> > > > >> > > > > > >> > >> > Regards,
>> > > > >> > > > > > >> > >> > Nick
>> > > > >> > > > > > >> > >> >
>> > > > >> > > > > > >> > >> > On Tue, 22 Nov 2022 at 16:25, John Roesler <
>> > > > >> > > > > vvcephei@apache.org>
>> > > > >> > > > > > >> > wrote:
>> > > > >> > > > > > >> > >> >
>> > > > >> > > > > > >> > >> > > Thanks for publishing this alternative,
>> Nick!
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > The benchmark you mentioned in the KIP-844
>> > > > >> discussion
>> > > > >> > > seems
>> > > > >> > > > > > like
>> > > > >> > > > > > >> a
>> > > > >> > > > > > >> > >> > > compelling reason to revisit the built-in
>> > > > >> > > transactionality
>> > > > >> > > > > > >> > mechanism.
>> > > > >> > > > > > >> > >> I
>> > > > >> > > > > > >> > >> > > also appreciate you analysis, showing that
>> for
>> > > > most
>> > > > >> use
>> > > > >> > > > > cases,
>> > > > >> > > > > > >> the
>> > > > >> > > > > > >> > >> write
>> > > > >> > > > > > >> > >> > > batch approach should be just fine.
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > There are a couple of points that would
>> hold
>> > me
>> > > > >> back from
>> > > > >> > > > > > >> approving
>> > > > >> > > > > > >> > >> this
>> > > > >> > > > > > >> > >> > > KIP right now:
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > 1. Loss of coverage for custom stores.
>> > > > >> > > > > > >> > >> > > The fact that you can plug in a
>> (relatively)
>> > > > simple
>> > > > >> > > > > > >> implementation
>> > > > >> > > > > > >> > of
>> > > > >> > > > > > >> > >> the
>> > > > >> > > > > > >> > >> > > XStateStore interfaces and automagically
>> get a
>> > > > >> > > distributed
>> > > > >> > > > > > >> database
>> > > > >> > > > > > >> > >> out
>> > > > >> > > > > > >> > >> > of
>> > > > >> > > > > > >> > >> > > it is a significant benefit of Kafka
>> Streams.
>> > > I'd
>> > > > >> hate to
>> > > > >> > > > > lose
>> > > > >> > > > > > >> it,
>> > > > >> > > > > > >> > so
>> > > > >> > > > > > >> > >> it
>> > > > >> > > > > > >> > >> > > would be better to spend some time and
>> come up
>> > > > with
>> > > > >> a way
>> > > > >> > > > to
>> > > > >> > > > > > >> > preserve
>> > > > >> > > > > > >> > >> > that
>> > > > >> > > > > > >> > >> > > property. For example, can we provide a
>> > default
>> > > > >> > > > > implementation
>> > > > >> > > > > > of
>> > > > >> > > > > > >> > >> > > `commit(..)` that re-implements the
>> existing
>> > > > >> > > > checkpoint-file
>> > > > >> > > > > > >> > >> approach? Or
>> > > > >> > > > > > >> > >> > > perhaps add an `isTransactional()` flag to
>> the
>> > > > state
>> > > > >> > > store
>> > > > >> > > > > > >> interface
>> > > > >> > > > > > >> > >> so
>> > > > >> > > > > > >> > >> > > that the runtime can decide whether to
>> > continue
>> > > to
>> > > > >> manage
>> > > > >> > > > > > >> checkpoint
>> > > > >> > > > > > >> > >> > files
>> > > > >> > > > > > >> > >> > > vs delegating transactionality to the
>> stores?
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > 2. Guarding against OOME
>> > > > >> > > > > > >> > >> > > I appreciate your analysis, but I don't
>> think
>> > > it's
>> > > > >> > > > sufficient
>> > > > >> > > > > > to
>> > > > >> > > > > > >> say
>> > > > >> > > > > > >> > >> that
>> > > > >> > > > > > >> > >> > > we will solve the memory problem later if
>> it
>> > > > becomes
>> > > > >> > > > > necessary.
>> > > > >> > > > > > >> The
>> > > > >> > > > > > >> > >> > > experience leading to that situation would
>> be
>> > > > quite
>> > > > >> bad:
>> > > > >> > > > > > Imagine,
>> > > > >> > > > > > >> > you
>> > > > >> > > > > > >> > >> > > upgrade to AK 3.next, your tests pass, so
>> you
>> > > > >> deploy to
>> > > > >> > > > > > >> production.
>> > > > >> > > > > > >> > >> That
>> > > > >> > > > > > >> > >> > > night, you get paged because your app is
>> now
>> > > > >> crashing
>> > > > >> > > with
>> > > > >> > > > > > >> OOMEs. As
>> > > > >> > > > > > >> > >> with
>> > > > >> > > > > > >> > >> > > all OOMEs, you'll have a really hard time
>> > > finding
>> > > > >> the
>> > > > >> > > root
>> > > > >> > > > > > cause,
>> > > > >> > > > > > >> > and
>> > > > >> > > > > > >> > >> > once
>> > > > >> > > > > > >> > >> > > you do, you won't have a clear path to
>> resolve
>> > > the
>> > > > >> issue.
>> > > > >> > > > You
>> > > > >> > > > > > >> could
>> > > > >> > > > > > >> > >> only
>> > > > >> > > > > > >> > >> > > tune down the commit interval and cache
>> buffer
>> > > > size
>> > > > >> until
>> > > > >> > > > you
>> > > > >> > > > > > >> stop
>> > > > >> > > > > > >> > >> > getting
>> > > > >> > > > > > >> > >> > > crashes.
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > FYI, I know of multiple cases where people
>> run
>> > > EOS
>> > > > >> with
>> > > > >> > > > much
>> > > > >> > > > > > >> larger
>> > > > >> > > > > > >> > >> > commit
>> > > > >> > > > > > >> > >> > > intervals to get better batching than the
>> > > default,
>> > > > >> so I
>> > > > >> > > > don't
>> > > > >> > > > > > >> think
>> > > > >> > > > > > >> > >> this
>> > > > >> > > > > > >> > >> > > pathological case would be as rare as you
>> > > suspect.
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > Given that we already have the rudiments
>> of an
>> > > > idea
>> > > > >> of
>> > > > >> > > what
>> > > > >> > > > > we
>> > > > >> > > > > > >> could
>> > > > >> > > > > > >> > >> do
>> > > > >> > > > > > >> > >> > to
>> > > > >> > > > > > >> > >> > > prevent this downside, we should take the
>> time
>> > > to
>> > > > >> design
>> > > > >> > > a
>> > > > >> > > > > > >> solution.
>> > > > >> > > > > > >> > >> We
>> > > > >> > > > > > >> > >> > owe
>> > > > >> > > > > > >> > >> > > it to our users to ensure that awesome new
>> > > > features
>> > > > >> don't
>> > > > >> > > > > come
>> > > > >> > > > > > >> with
>> > > > >> > > > > > >> > >> > bitter
>> > > > >> > > > > > >> > >> > > pills unless we can't avoid it.
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > 3. ALOS mode.
>> > > > >> > > > > > >> > >> > > On the other hand, I didn't see an
>> indication
>> > of
>> > > > how
>> > > > >> > > stores
>> > > > >> > > > > > will
>> > > > >> > > > > > >> be
>> > > > >> > > > > > >> > >> > > handled under ALOS (aka non-EOS) mode.
>> > > > >> Theoretically, the
>> > > > >> > > > > > >> > >> > transactionality
>> > > > >> > > > > > >> > >> > > of the store and the processing mode are
>> > > > >> orthogonal. A
>> > > > >> > > > > > >> transactional
>> > > > >> > > > > > >> > >> > store
>> > > > >> > > > > > >> > >> > > would serve ALOS just as well as a
>> > > > >> non-transactional one
>> > > > >> > > > (if
>> > > > >> > > > > > not
>> > > > >> > > > > > >> > >> better).
>> > > > >> > > > > > >> > >> > > Under ALOS, though, the default commit
>> > interval
>> > > is
>> > > > >> five
>> > > > >> > > > > > minutes,
>> > > > >> > > > > > >> so
>> > > > >> > > > > > >> > >> the
>> > > > >> > > > > > >> > >> > > memory issue is far more pressing.
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > As I see it, we have several options to
>> > resolve
>> > > > this
>> > > > >> > > point.
>> > > > >> > > > > We
>> > > > >> > > > > > >> could
>> > > > >> > > > > > >> > >> > > demonstrate that transactional stores work
>> > just
>> > > > >> fine for
>> > > > >> > > > ALOS
>> > > > >> > > > > > >> and we
>> > > > >> > > > > > >> > >> can
>> > > > >> > > > > > >> > >> > > therefore just swap over unconditionally.
>> We
>> > > could
>> > > > >> also
>> > > > >> > > > > disable
>> > > > >> > > > > > >> the
>> > > > >> > > > > > >> > >> > > transactional mechanism under ALOS so that
>> > > stores
>> > > > >> operate
>> > > > >> > > > > just
>> > > > >> > > > > > >> the
>> > > > >> > > > > > >> > >> same
>> > > > >> > > > > > >> > >> > as
>> > > > >> > > > > > >> > >> > > they do today when run in ALOS mode.
>> Finally,
>> > we
>> > > > >> could do
>> > > > >> > > > the
>> > > > >> > > > > > >> same
>> > > > >> > > > > > >> > as
>> > > > >> > > > > > >> > >> in
>> > > > >> > > > > > >> > >> > > KIP-844 and make transactional stores
>> opt-in
>> > > (it'd
>> > > > >> be
>> > > > >> > > > better
>> > > > >> > > > > to
>> > > > >> > > > > > >> > avoid
>> > > > >> > > > > > >> > >> the
>> > > > >> > > > > > >> > >> > > extra opt-in mechanism, but it's a good
>> > > > >> > > > get-out-of-jail-free
>> > > > >> > > > > > >> card).
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > 4. (minor point) Deprecation of methods
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > You mentioned that the new `commit` 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?
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > Thanks again for the KIP! It's really nice
>> > that
>> > > > you
>> > > > >> and
>> > > > >> > > > Alex
>> > > > >> > > > > > will
>> > > > >> > > > > > >> > get
>> > > > >> > > > > > >> > >> the
>> > > > >> > > > > > >> > >> > > chance to collaborate on both directions so
>> > that
>> > > > we
>> > > > >> can
>> > > > >> > > get
>> > > > >> > > > > the
>> > > > >> > > > > > >> best
>> > > > >> > > > > > >> > >> > > outcome for Streams and its users.
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > -John
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> > > On 2022/11/21 15:02:15 Nick Telford wrote:
>> > > > >> > > > > > >> > >> > > > 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
>> > > > >> > > > > > >> > >> > > > <
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> >
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> >
>> > > > >> > > > > > >>
>> > > > >> > > > > >
>> > > > >> > > > >
>> > > > >> > > >
>> > > > >> > >
>> > > > >>
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> > > > >> > > > > > >> > >> > > >,
>> > > > >> > > > > > >> > >> > > > so that it can be discussed/reviewed
>> > > separately
>> > > > >> from
>> > > > >> > > > > KIP-844.
>> > > > >> > > > > > >> > >> > > >
>> > > > >> > > > > > >> > >> > > > Alex: I'm especially interested in what
>> you
>> > > > think!
>> > > > >> > > > > > >> > >> > > >
>> > > > >> > > > > > >> > >> > > > I have a nearly complete implementation
>> of
>> > the
>> > > > >> changes
>> > > > >> > > > > > >> outlined in
>> > > > >> > > > > > >> > >> this
>> > > > >> > > > > > >> > >> > > > KIP, please let me know if you'd like me
>> to
>> > > push
>> > > > >> them
>> > > > >> > > for
>> > > > >> > > > > > >> review
>> > > > >> > > > > > >> > in
>> > > > >> > > > > > >> > >> > > advance
>> > > > >> > > > > > >> > >> > > > of a vote.
>> > > > >> > > > > > >> > >> > > >
>> > > > >> > > > > > >> > >> > > > Regards,
>> > > > >> > > > > > >> > >> > > >
>> > > > >> > > > > > >> > >> > > > Nick
>> > > > >> > > > > > >> > >> > > >
>> > > > >> > > > > > >> > >> > >
>> > > > >> > > > > > >> > >> >
>> > > > >> > > > > > >> > >>
>> > > > >> > > > > > >> > >
>> > > > >> > > > > > >> >
>> > > > >> > > > > > >>
>> > > > >> > > > > > >
>> > > > >> > > > > >
>> > > > >> > > > >
>> > > > >> > > >
>> > > > >> > >
>> > > > >>
>> > > > >
>> > > >
>> > >
>> >
>>
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Colt,

The issue is that if there's a crash between 2 and 3, then you still end up
with inconsistent data in RocksDB. The only way to guarantee that your
checkpoint offsets and locally stored data are consistent with each other
are to atomically commit them, which can be achieved by having the offsets
stored in RocksDB.

The offsets column family is likely to be extremely small (one
per-changelog partition + one per Topology input partition for regular
stores, one per input partition for global stores). So the overhead will be
minimal.

A major benefit of doing this is that we can remove the explicit calls to
db.flush(), which forcibly flushes memtables to disk on-commit. It turns
out, RocksDB memtable flushes are largely dictated by Kafka Streams
commits, *not* RocksDB configuration, which could be a major source of
confusion. Atomic checkpointing makes it safe to remove these explicit
flushes, because it no longer matters exactly when RocksDB flushes data to
disk; since the data and corresponding checkpoint offsets will always be
flushed together, the local store is always in a consistent state, and
on-restart, it can always safely resume restoration from the on-disk
offsets, restoring the small amount of data that hadn't been flushed when
the app exited/crashed.

Regards,
Nick

On Wed, 19 Apr 2023 at 14:35, Colt McNealy <co...@littlehorse.io> wrote:

> Nick,
>
> Thanks for your reply. Ack to A) and B).
>
> For item C), I see what you're referring to. Your proposed solution will
> work, so no need to change it. What I was suggesting was that it might be
> possible to achieve this with only one column family. So long as:
>
>    - No uncommitted records (i.e. not committed to the changelog) are
>    *committed* to the state store, AND
>    - The Checkpoint offset (which refers to the changelog topic) is less
>    than or equal to the last written changelog offset in rocksdb
>
> I don't see the need to do the full restoration from scratch. My
> understanding was that prior to 844/892, full restorations were required
> because there could be uncommitted records written to RocksDB; however,
> given your use of RocksDB transactions, that can be avoided with the
> pattern of 1) commit Kafka transaction, 2) commit RocksDB transaction, 3)
> update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <ni...@gmail.com>
> wrote:
>
> > Hi Colt,
> >
> > A. I've done my best to de-couple the StateStore stuff from the rest of
> the
> > Streams engine. The fact that there will be only one ongoing (write)
> > transaction at a time is not guaranteed by any API, and is just a
> > consequence of the way Streams operates. To that end, I tried to ensure
> the
> > documentation and guarantees provided by the new APIs are independent of
> > this incidental behaviour. In practice, you're right, this essentially
> > refers to "interactive queries", which are technically "read
> transactions",
> > even if they don't actually use the transaction API to isolate
> themselves.
> >
> > B. Yes, although not ideal. This is for backwards compatibility, because:
> >     1) Existing custom StateStore implementations will implement flush(),
> > and not commit(), but the Streams engine now calls commit(), so those
> calls
> > need to be forwarded to flush() for these legacy stores.
> >     2) Existing StateStore *users*, i.e. outside of the Streams engine
> > itself, may depend on explicitly calling flush(), so for these cases,
> > flush() needs to be redirected to call commit().
> > If anyone has a better way to guarantee compatibility without introducing
> > this potential recursion loop, I'm open to changes!
> >
> > C. This is described in the "Atomic Checkpointing" section. Offsets are
> > stored in a separate RocksDB column family, which is guaranteed to be
> > atomically flushed to disk with all other column families. The issue of
> > checkpoints being written to disk after commit causing inconsistency if
> it
> > crashes in between is the reason why, under EOS, checkpoint files are
> only
> > written on clean shutdown. This is one of the major causes of "full
> > restorations", so moving the offsets into a place where they can be
> > guaranteed to be atomically written with the data they checkpoint allows
> us
> > to write the checkpoint offsets *on every commit*, not just on clean
> > shutdown.
> >
> > Regards,
> > Nick
> >
> > On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io> wrote:
> >
> > > Nick,
> > >
> > > Thank you for continuing this work. I have a few minor clarifying
> > > questions.
> > >
> > > A) "Records written to any transaction are visible to all other
> > > transactions immediately." I am confused here—I thought there could
> only
> > be
> > > one transaction going on at a time for a given state store given the
> > > threading model for processing records on a Task. Do you mean
> Interactive
> > > Queries by "other transactions"? (If so, then everything makes sense—I
> > > thought that since IQ were read-only then they didn't count as
> > > transactions).
> > >
> > > B) Is it intentional that the default implementations of the flush()
> and
> > > commit() methods in the StateStore class refer to each other in some
> sort
> > > of unbounded recursion?
> > >
> > > C) How will the getCommittedOffset() method work? At first I thought
> the
> > > way to do it would be using a special key in the RocksDB store to store
> > the
> > > offset, and committing that with the transaction. But upon second
> > thought,
> > > since restoration from the changelog is an idempotent procedure, I
> think
> > it
> > > would be fine to 1) commit the RocksDB transaction and then 2) write
> the
> > > offset to disk in a checkpoint file. If there is a crash between 1) and
> > 2),
> > > I think the only downside is 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 <ni...@gmail.com>
> > > wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > I've updated the KIP to reflect the latest version of the design:
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > >
> > > > There are several changes in there that reflect feedback from this
> > > thread,
> > > > and there's a new section and a bunch of interface changes relating
> to
> > > > Atomic Checkpointing, which is the final piece of the puzzle to
> making
> > > > everything robust.
> > > >
> > > > Let me know what you think!
> > > >
> > > > Regards,
> > > > Nick
> > > >
> > > > On Tue, 3 Jan 2023 at 11:33, Nick Telford <ni...@gmail.com>
> > > wrote:
> > > >
> > > > > Hi Lucas,
> > > > >
> > > > > Thanks for looking over my KIP.
> > > > >
> > > > > A) The bound is per-instance, not per-Task. This was a typo in the
> > KIP
> > > > > that I've now corrected. It was originally per-Task, but I changed
> it
> > > to
> > > > > per-instance for exactly the reason you highlighted.
> > > > > B) It's worth noting that transactionality is only enabled under
> EOS,
> > > and
> > > > > in the default mode of operation (ALOS), there should be no change
> in
> > > > > behavior at all. I think, under EOS, we can mitigate the impact on
> > > users
> > > > by
> > > > > sufficiently low default values for the memory bound
> configuration. I
> > > > > understand your hesitation to include a significant change of
> > > behaviour,
> > > > > especially in a minor release, but I suspect that most users will
> > > prefer
> > > > > the memory impact (under EOS) to the existing behaviour of frequent
> > > state
> > > > > restorations! If this is a problem, the changes can wait until the
> > next
> > > > > major release. I'll be running a patched version of streams in
> > > production
> > > > > with these changes as soon as they're ready, so it won't disrupt me
> > :-D
> > > > > C) The main purpose of this sentence was just to note that some
> > changes
> > > > > will need to be made to the way Segments are handled in order to
> > ensure
> > > > > they also benefit from transactions. At the time I wrote it, I
> hadn't
> > > > > figured out the specific changes necessary, so it was deliberately
> > > vague.
> > > > > This is the one outstanding problem I'm currently working on, and
> > I'll
> > > > > update this section with more detail once I have figured out the
> > exact
> > > > > changes required.
> > > > > D) newTransaction() provides the necessary isolation guarantees.
> > While
> > > > > the RocksDB implementation of transactions doesn't technically
> *need*
> > > > > read-only users to call newTransaction(), other implementations
> > (e.g. a
> > > > > hypothetical PostgresStore) may require it. Calling
> newTransaction()
> > > when
> > > > > no transaction is necessary is essentially free, as it will just
> > return
> > > > > this.
> > > > >
> > > > > I didn't do any profiling of the KIP-844 PoC, but I think it should
> > be
> > > > > fairly obvious where the performance problems stem from: writes
> under
> > > > > KIP-844 require 3 extra memory-copies: 1 to encode it with the
> > > > > tombstone/record flag, 1 to decode it from the tombstone/record
> flag,
> > > > and 1
> > > > > to copy the record from the "temporary" store to the "main" store,
> > when
> > > > the
> > > > > transaction commits. The different approach taken by KIP-869 should
> > > > perform
> > > > > much better, as it avoids all these copies, and may actually
> perform
> > > > > slightly better than trunk, due to batched writes in RocksDB
> > performing
> > > > > better than non-batched writes.[1]
> > > > >
> > > > > Regards,
> > > > > Nick
> > > > >
> > > > > 1:
> > > >
> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > >
> > > > > On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <
> lbrutschy@confluent.io
> > > > .invalid>
> > > > > wrote:
> > > > >
> > > > >> Hi Nick,
> > > > >>
> > > > >> I'm just starting to read up on the whole discussion about KIP-892
> > and
> > > > >> KIP-844. Thanks a lot for your work on this, I do think
> > > > >> `WriteBatchWithIndex` may be the way to go here. I do have some
> > > > >> questions about the latest draft.
> > > > >>
> > > > >>  A) If I understand correctly, you propose to put a bound on the
> > > > >> (native) memory consumed by each task. However, I wonder if this
> is
> > > > >> sufficient if we have temporary imbalances in the cluster. For
> > > > >> example, depending on the timing of rebalances during a cluster
> > > > >> restart, it could happen that a single streams node is assigned a
> > lot
> > > > >> more tasks than expected. With your proposed change, this would
> mean
> > > > >> that the memory required by this one node could be a multiple of
> > what
> > > > >> is required during normal operation. I wonder if it wouldn't be
> > safer
> > > > >> to put a global bound on the memory use, across all tasks.
> > > > >>  B) Generally, the memory concerns still give me the feeling that
> > this
> > > > >> should not be enabled by default for all users in a minor release.
> > > > >>  C) In section "Transaction Management": the sentence "A similar
> > > > >> analogue will be created to automatically manage `Segment`
> > > > >> transactions.". Maybe this is just me lacking some background,
> 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 you to find out why the KIP-844 PoC is 20x slower
> (e.g.
> > > > >> by providing a flame graph), that would be quite interesting.
> > > > >>
> > > > >> Cheers,
> > > > >> Lucas
> > > > >>
> > > > >> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
> > nick.telford@gmail.com>
> > > > >> wrote:
> > > > >> >
> > > > >> > Hi everyone,
> > > > >> >
> > > > >> > I've updated the KIP with a more detailed design, which reflects
> > the
> > > > >> > implementation I've been working on:
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > >> >
> > > > >> > This new design should address the outstanding points already
> made
> > > in
> > > > >> the
> > > > >> > thread.
> > > > >> >
> > > > >> > Please let me know if there are areas that are unclear or need
> > more
> > > > >> > clarification.
> > > > >> >
> > > > >> > I have a (nearly) working implementation. I'm confident that the
> > > > >> remaining
> > > > >> > work (making Segments behave) will not impact the documented
> > design.
> > > > >> >
> > > > >> > Regards,
> > > > >> >
> > > > >> > Nick
> > > > >> >
> > > > >> > On Tue, 6 Dec 2022 at 19:24, Colt McNealy <co...@littlehorse.io>
> > > > wrote:
> > > > >> >
> > > > >> > > Nick,
> > > > >> > >
> > > > >> > > Thank you for the reply; that makes sense. I was hoping that,
> > > since
> > > > >> reading
> > > > >> > > uncommitted records from IQ in EOS isn't part of the
> documented
> > > API,
> > > > >> maybe
> > > > >> > > you *wouldn't* have to wait for the next major release to make
> > > that
> > > > >> change;
> > > > >> > > but given 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 <
> > > > nick.telford@gmail.com>
> > > > >> > > wrote:
> > > > >> > >
> > > > >> > > > Hi Colt,
> > > > >> > > >
> > > > >> > > > 10: Yes, I agree it's not ideal. I originally intended to
> try
> > to
> > > > >> keep the
> > > > >> > > > behaviour unchanged as much as possible, otherwise we'd have
> > to
> > > > >> wait for
> > > > >> > > a
> > > > >> > > > major version release to land these changes.
> > > > >> > > > 20: Good point, ALOS doesn't need the same level of
> guarantee,
> > > and
> > > > >> the
> > > > >> > > > typically longer commit intervals would be problematic when
> > > > reading
> > > > >> only
> > > > >> > > > "committed" records.
> > > > >> > > >
> > > > >> > > > I've been away for 5 days recovering from minor surgery,
> but I
> > > > >> spent a
> > > > >> > > > considerable amount of that time working through ideas for
> > > > possible
> > > > >> > > > solutions in my head. I think your suggestion of keeping
> ALOS
> > > > >> as-is, but
> > > > >> > > > buffering writes for EOS is the right path forwards,
> although
> > I
> > > > >> have a
> > > > >> > > > solution that both expands on this, and provides for some
> more
> > > > >> formal
> > > > >> > > > guarantees.
> > > > >> > > >
> > > > >> > > > Essentially, adding support to KeyValueStores for
> > > "Transactions",
> > > > >> with
> > > > >> > > > clearly defined IsolationLevels. Using "Read Committed" when
> > > under
> > > > >> EOS,
> > > > >> > > and
> > > > >> > > > "Read Uncommitted" under ALOS.
> > > > >> > > >
> > > > >> > > > The nice thing about this approach is that it gives us much
> > more
> > > > >> clearly
> > > > >> > > > defined isolation behaviour that can be properly documented
> to
> > > > >> ensure
> > > > >> > > users
> > > > >> > > > know what to expect.
> > > > >> > > >
> > > > >> > > > I'm still working out the kinks in the design, and will
> update
> > > the
> > > > >> KIP
> > > > >> > > when
> > > > >> > > > I have something. The main struggle is trying to implement
> > this
> > > > >> without
> > > > >> > > > making any major changes to the existing interfaces or
> > breaking
> > > > >> existing
> > > > >> > > > implementations, because currently everything expects to
> > operate
> > > > >> directly
> > > > >> > > > on a StateStore, and not a Transaction of that store. I
> think
> > > I'm
> > > > >> getting
> > > > >> > > > close, although sadly I won't be able to progress much until
> > > next
> > > > >> week
> > > > >> > > due
> > > > >> > > > to some work commitments.
> > > > >> > > >
> > > > >> > > > Regards,
> > > > >> > > > Nick
> > > > >> > > >
> > > > >> > > > On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
> > colt@littlehorse.io>
> > > > >> wrote:
> > > > >> > > >
> > > > >> > > > > Nick,
> > > > >> > > > >
> > > > >> > > > > Thank you for the explanation, and also for the updated
> > KIP. I
> > > > am
> > > > >> quite
> > > > >> > > > > eager for this improvement to be released as it would
> > greatly
> > > > >> reduce
> > > > >> > > the
> > > > >> > > > > operational difficulties of EOS streams apps.
> > > > >> > > > >
> > > > >> > > > > Two questions:
> > > > >> > > > >
> > > > >> > > > > 10)
> > > > >> > > > > >When reading records, we will use the
> > > > >> > > > > WriteBatchWithIndex#getFromBatchAndDB
> > > > >> > > > >  and WriteBatchWithIndex#newIteratorWithBase utilities in
> > > order
> > > > to
> > > > >> > > ensure
> > > > >> > > > > that uncommitted writes are available to query.
> > > > >> > > > > Why do extra work to enable the reading of uncommitted
> > writes
> > > > >> during
> > > > >> > > IQ?
> > > > >> > > > > Code complexity aside, reading uncommitted writes is, in
> my
> > > > >> opinion, a
> > > > >> > > > > minor flaw in EOS IQ; it would be very nice to have the
> > > > guarantee
> > > > >> that,
> > > > >> > > > > with EOS, IQ only reads committed records. In order to
> avoid
> > > > dirty
> > > > >> > > reads,
> > > > >> > > > > one currently must query a standby replica (but this still
> > > > doesn't
> > > > >> > > fully
> > > > >> > > > > guarantee monotonic reads).
> > > > >> > > > >
> > > > >> > > > > 20) Is it also necessary to enable this optimization on
> ALOS
> > > > >> stores?
> > > > >> > > The
> > > > >> > > > > motivation of KIP-844 was mainly to reduce the need to
> > restore
> > > > >> state
> > > > >> > > from
> > > > >> > > > > scratch on unclean EOS shutdowns; with ALOS it was
> > acceptable
> > > to
> > > > >> accept
> > > > >> > > > > that there may have been uncommitted writes on disk. On a
> > side
> > > > >> note, if
> > > > >> > > > you
> > > > >> > > > > enable this type of store on ALOS processors, the
> community
> > > > would
> > > > >> > > > > 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 <
> > > > >> nick.telford@gmail.com>
> > > > >> > > > > wrote:
> > > > >> > > > >
> > > > >> > > > > > Hi everyone,
> > > > >> > > > > >
> > > > >> > > > > > I've drastically reduced the scope of this KIP to no
> > longer
> > > > >> include
> > > > >> > > the
> > > > >> > > > > > StateStore management of checkpointing. This can be
> added
> > > as a
> > > > >> KIP
> > > > >> > > > later
> > > > >> > > > > on
> > > > >> > > > > > to further optimize the consistency and performance of
> > state
> > > > >> stores.
> > > > >> > > > > >
> > > > >> > > > > > I've also added a section discussing some of the
> concerns
> > > > 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 2022 at 16:36, Nick Telford <
> > > > >> nick.telford@gmail.com>
> > > > >> > > > > wrote:
> > > > >> > > > > >
> > > > >> > > > > > > Hi Colt,
> > > > >> > > > > > >
> > > > >> > > > > > > I didn't do any profiling, but the 844 implementation:
> > > > >> > > > > > >
> > > > >> > > > > > >    - Writes uncommitted records to a temporary RocksDB
> > > > >> instance
> > > > >> > > > > > >       - Since tombstones need to be flagged, all
> record
> > > > >> values are
> > > > >> > > > > > >       prefixed with a value/tombstone marker. This
> > > > >> necessitates a
> > > > >> > > > > memory
> > > > >> > > > > > copy.
> > > > >> > > > > > >    - On-commit, iterates all records in this temporary
> > > > >> instance and
> > > > >> > > > > > >    writes them to the main RocksDB store.
> > > > >> > > > > > >    - While iterating, the value/tombstone marker needs
> > to
> > > be
> > > > >> parsed
> > > > >> > > > and
> > > > >> > > > > > >    the real value extracted. This necessitates another
> > > > memory
> > > > >> copy.
> > > > >> > > > > > >
> > > > >> > > > > > > My guess is that the cost of iterating the temporary
> > > RocksDB
> > > > >> store
> > > > >> > > is
> > > > >> > > > > the
> > > > >> > > > > > > major factor, with the 2 extra memory copies
> per-Record
> > > > >> > > contributing
> > > > >> > > > a
> > > > >> > > > > > > significant amount too.
> > > > >> > > > > > >
> > > > >> > > > > > > Regards,
> > > > >> > > > > > > Nick
> > > > >> > > > > > >
> > > > >> > > > > > > On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
> > > > >> colt@littlehorse.io>
> > > > >> > > > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > >> Hi all,
> > > > >> > > > > > >>
> > > > >> > > > > > >> Out of curiosity, why does the performance of the
> store
> > > > >> degrade so
> > > > >> > > > > > >> significantly with the 844 implementation? I wouldn't
> > be
> > > > too
> > > > >> > > > surprised
> > > > >> > > > > > by
> > > > >> > > > > > >> a
> > > > >> > > > > > >> 50-60% drop (caused by each record being written
> > twice),
> > > > but
> > > > >> 96%
> > > > >> > > is
> > > > >> > > > > > >> extreme.
> > > > >> > > > > > >>
> > > > >> > > > > > >> The only thing I can think of which could create
> such a
> > > > >> bottleneck
> > > > >> > > > > 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 Nick Telford <
> > > > >> > > > nick.telford@gmail.com>
> > > > >> > > > > > >> wrote:
> > > > >> > > > > > >>
> > > > >> > > > > > >> > Hi everyone,
> > > > >> > > > > > >> >
> > > > >> > > > > > >> > I've updated the KIP to resolve all the points that
> > > have
> > > > >> been
> > > > >> > > > raised
> > > > >> > > > > > so
> > > > >> > > > > > >> > far, with one exception: the ALOS default commit
> > > interval
> > > > >> of 5
> > > > >> > > > > minutes
> > > > >> > > > > > >> is
> > > > >> > > > > > >> > likely to cause WriteBatchWithIndex memory to grow
> > too
> > > > >> large.
> > > > >> > > > > > >> >
> > > > >> > > > > > >> > There's a couple of different things I can think of
> > to
> > > > >> solve
> > > > >> > > this:
> > > > >> > > > > > >> >
> > > > >> > > > > > >> >    - We already have a memory/record limit in the
> KIP
> > > to
> > > > >> prevent
> > > > >> > > > OOM
> > > > >> > > > > > >> >    errors. Should we choose a default value for
> > these?
> > > My
> > > > >> > > concern
> > > > >> > > > > here
> > > > >> > > > > > >> is
> > > > >> > > > > > >> > that
> > > > >> > > > > > >> >    anything we choose might seem rather arbitrary.
> We
> > > > could
> > > > >> > > change
> > > > >> > > > > > >> >    its behaviour such that under ALOS, it only
> > triggers
> > > > the
> > > > >> > > commit
> > > > >> > > > > of
> > > > >> > > > > > >> the
> > > > >> > > > > > >> >    StateStore, but under EOS, it triggers a commit
> of
> > > the
> > > > >> Kafka
> > > > >> > > > > > >> > transaction.
> > > > >> > > > > > >> >    - We could introduce a separate `
> > > > checkpoint.interval.ms`
> > > > >> to
> > > > >> > > > > allow
> > > > >> > > > > > >> ALOS
> > > > >> > > > > > >> >    to commit the StateStores more frequently than
> the
> > > > >> general
> > > > >> > > > > > >> >    commit.interval.ms? My concern here is that the
> > > > >> semantics of
> > > > >> > > > > this
> > > > >> > > > > > >> > config
> > > > >> > > > > > >> >    would depend 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 <
> > > > >> > > > nick.telford@gmail.com>
> > > > >> > > > > > >> wrote:
> > > > >> > > > > > >> >
> > > > >> > > > > > >> > > Hi Alex,
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > Thanks for the feedback.
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > I've updated the discussion of OOM issues by
> > > describing
> > > > >> how
> > > > >> > > > we'll
> > > > >> > > > > > >> handle
> > > > >> > > > > > >> > > it. Here's the new text:
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > To mitigate this, we will automatically force a
> > Task
> > > > >> commit if
> > > > >> > > > the
> > > > >> > > > > > >> total
> > > > >> > > > > > >> > >> uncommitted records returned by
> > > > >> > > > > > >> > >> StateStore#approximateNumUncommittedEntries()
> > > > exceeds a
> > > > >> > > > > threshold,
> > > > >> > > > > > >> > >> configured by
> > > max.uncommitted.state.entries.per.task;
> > > > >> or the
> > > > >> > > > > total
> > > > >> > > > > > >> > >> memory used for buffering uncommitted records
> > > returned
> > > > >> by
> > > > >> > > > > > >> > >> StateStore#approximateNumUncommittedBytes()
> > exceeds
> > > > the
> > > > >> > > > threshold
> > > > >> > > > > > >> > >> configured by
> > max.uncommitted.state.bytes.per.task.
> > > > >> This will
> > > > >> > > > > > roughly
> > > > >> > > > > > >> > >> bound the memory required per-Task for buffering
> > > > >> uncommitted
> > > > >> > > > > > records,
> > > > >> > > > > > >> > >> irrespective of the commit.interval.ms, and
> will
> > > > >> effectively
> > > > >> > > > > bound
> > > > >> > > > > > >> the
> > > > >> > > > > > >> > >> number of records that will need to be restored
> in
> > > the
> > > > >> event
> > > > >> > > > of a
> > > > >> > > > > > >> > failure.
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > These limits will be checked in
> StreamTask#process
> > > and
> > > > a
> > > > >> > > > premature
> > > > >> > > > > > >> commit
> > > > >> > > > > > >> > >> will be requested via Task#requestCommit().
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > Note that these new methods provide default
> > > > >> implementations
> > > > >> > > that
> > > > >> > > > > > >> ensure
> > > > >> > > > > > >> > >> existing custom stores and non-transactional
> > stores
> > > > >> (e.g.
> > > > >> > > > > > >> > >> InMemoryKeyValueStore) do not force any early
> > > commits.
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > I've chosen to have the StateStore expose
> > > > approximations
> > > > >> of
> > > > >> > > its
> > > > >> > > > > > buffer
> > > > >> > > > > > >> > > size/count instead of opaquely requesting a
> commit
> > in
> > > > >> order to
> > > > >> > > > > > >> delegate
> > > > >> > > > > > >> > the
> > > > >> > > > > > >> > > decision making to the Task itself. This enables
> > > Tasks
> > > > >> to look
> > > > >> > > > at
> > > > >> > > > > > >> *all*
> > > > >> > > > > > >> > of
> > > > >> > > > > > >> > > their StateStores, and determine whether an early
> > > > commit
> > > > >> is
> > > > >> > > > > > necessary.
> > > > >> > > > > > >> > > Notably, it enables pre-Task thresholds, instead
> of
> > > > >> per-Store,
> > > > >> > > > > which
> > > > >> > > > > > >> > > prevents Tasks with many StateStores from using
> > much
> > > > more
> > > > >> > > memory
> > > > >> > > > > > than
> > > > >> > > > > > >> > Tasks
> > > > >> > > > > > >> > > with one StateStore. This makes sense, since
> > commits
> > > > are
> > > > >> done
> > > > >> > > > > > by-Task,
> > > > >> > > > > > >> > not
> > > > >> > > > > > >> > > by-Store.
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > Prizes* for anyone who can come up with a better
> > name
> > > > >> for the
> > > > >> > > > new
> > > > >> > > > > > >> config
> > > > >> > > > > > >> > > properties!
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > Thanks for pointing out the potential performance
> > > > issues
> > > > >> of
> > > > >> > > > WBWI.
> > > > >> > > > > > From
> > > > >> > > > > > >> > the
> > > > >> > > > > > >> > > benchmarks that user posted[1], it looks like
> WBWI
> > > > still
> > > > >> > > > performs
> > > > >> > > > > > >> > > considerably better than individual puts, which
> is
> > > the
> > > > >> > > existing
> > > > >> > > > > > >> design,
> > > > >> > > > > > >> > so
> > > > >> > > > > > >> > > I'd actually expect a performance boost from
> WBWI,
> > > just
> > > > >> not as
> > > > >> > > > > great
> > > > >> > > > > > >> as
> > > > >> > > > > > >> > > we'd get from a plain WriteBatch. This does
> suggest
> > > > that
> > > > >> a
> > > > >> > > good
> > > > >> > > > > > >> > > optimization would be to use a regular WriteBatch
> > for
> > > > >> > > > restoration
> > > > >> > > > > > (in
> > > > >> > > > > > >> > > RocksDBStore#restoreBatch), since we know that
> > those
> > > > >> records
> > > > >> > > > will
> > > > >> > > > > > >> never
> > > > >> > > > > > >> > be
> > > > >> > > > > > >> > > queried before they're committed.
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > 1:
> > > > >> > > > > > >> >
> > > > >> > > > > >
> > > > >> > > >
> > > > >>
> > >
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > * Just kidding, no prizes, sadly.
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > > On Wed, 23 Nov 2022 at 12:28, Alexander
> Sorokoumov
> > > > >> > > > > > >> > > <as...@confluent.io.invalid> wrote:
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> > >> 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 to keep the current
> > > checkpointing
> > > > >> as a
> > > > >> > > > > > fallback
> > > > >> > > > > > >> > >> mechanism. We want to keep existing users'
> > workflows
> > > > >> intact
> > > > >> > > if
> > > > >> > > > we
> > > > >> > > > > > >> can. A
> > > > >> > > > > > >> > >> non-intrusive way would be to add a separate
> > > > StateStore
> > > > >> > > method,
> > > > >> > > > > > say,
> > > > >> > > > > > >> > >> StateStore#managesCheckpointing(), that controls
> > > > >> whether the
> > > > >> > > > > state
> > > > >> > > > > > >> store
> > > > >> > > > > > >> > >> implementation owns checkpointing.
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> > >> I think that a solution to the transactional
> > writes
> > > > >> should
> > > > >> > > > > address
> > > > >> > > > > > >> the
> > > > >> > > > > > >> > >> OOMEs. One possible way to address that is to
> wire
> > > > >> > > StateStore's
> > > > >> > > > > > >> commit
> > > > >> > > > > > >> > >> request by adding, say, StateStore#commitNeeded
> > that
> > > > is
> > > > >> > > checked
> > > > >> > > > > in
> > > > >> > > > > > >> > >> StreamTask#commitNeeded via the corresponding
> > > > >> > > > > > ProcessorStateManager.
> > > > >> > > > > > >> > With
> > > > >> > > > > > >> > >> that change, RocksDBStore will have to track the
> > > > current
> > > > >> > > > > > transaction
> > > > >> > > > > > >> > size
> > > > >> > > > > > >> > >> and request a commit when the size goes over a
> > > > >> (configurable)
> > > > >> > > > > > >> threshold.
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> > >> AFAIU WriteBatchWithIndex might perform
> > > significantly
> > > > >> slower
> > > > >> > > > than
> > > > >> > > > > > >> > non-txn
> > > > >> > > > > > >> > >> puts as the batch size grows [1]. We should
> have a
> > > > >> > > > configuration
> > > > >> > > > > to
> > > > >> > > > > > >> fall
> > > > >> > > > > > >> > >> back to the current behavior (and/or disable txn
> > > > stores
> > > > >> for
> > > > >> > > > ALOS)
> > > > >> > > > > > >> unless
> > > > >> > > > > > >> > >> the benchmarks show negligible overhead for
> longer
> > > > >> commits /
> > > > >> > > > > > >> > large-enough
> > > > >> > > > > > >> > >> batch sizes.
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> > >> If you prefer to keep the KIP smaller, I would
> > > rather
> > > > >> cut out
> > > > >> > > > > > >> > >> state-store-managed checkpointing rather than
> > proper
> > > > >> OOMe
> > > > >> > > > > handling
> > > > >> > > > > > >> and
> > > > >> > > > > > >> > >> being able to switch to non-txn behavior. The
> > > > >> checkpointing
> > > > >> > > 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 configuration parts as well as review
> and
> > > > test
> > > > >> the
> > > > >> > > > > patch.
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> > >> Best,
> > > > >> > > > > > >> > >> Alex
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> > >> 1.
> https://github.com/facebook/rocksdb/issues/608
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> > >> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
> > > > >> > > > > > nick.telford@gmail.com
> > > > >> > > > > > >> >
> > > > >> > > > > > >> > >> wrote:
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> > >> > Hi John,
> > > > >> > > > > > >> > >> >
> > > > >> > > > > > >> > >> > Thanks for the review and feedback!
> > > > >> > > > > > >> > >> >
> > > > >> > > > > > >> > >> > 1. Custom Stores: I've been mulling over this
> > > > problem
> > > > >> > > myself.
> > > > >> > > > > As
> > > > >> > > > > > it
> > > > >> > > > > > >> > >> stands,
> > > > >> > > > > > >> > >> > custom stores would essentially lose
> > checkpointing
> > > > >> with no
> > > > >> > > > > > >> indication
> > > > >> > > > > > >> > >> that
> > > > >> > > > > > >> > >> > they're expected to make changes, besides a
> line
> > > in
> > > > >> the
> > > > >> > > > release
> > > > >> > > > > > >> > notes. I
> > > > >> > > > > > >> > >> > agree that the best solution would be to
> > provide a
> > > > >> default
> > > > >> > > > that
> > > > >> > > > > > >> > >> checkpoints
> > > > >> > > > > > >> > >> > to a file. The one thing I would change is
> that
> > > the
> > > > >> > > > > checkpointing
> > > > >> > > > > > >> is
> > > > >> > > > > > >> > to
> > > > >> > > > > > >> > >> a
> > > > >> > > > > > >> > >> > store-local file, instead of a per-Task file.
> > This
> > > > >> way the
> > > > >> > > > > > >> StateStore
> > > > >> > > > > > >> > >> still
> > > > >> > > > > > >> > >> > technically owns its own checkpointing (via a
> > > > default
> > > > >> > > > > > >> implementation),
> > > > >> > > > > > >> > >> and
> > > > >> > > > > > >> > >> > the StateManager/Task execution engine doesn't
> > > need
> > > > >> to know
> > > > >> > > > > > >> anything
> > > > >> > > > > > >> > >> about
> > > > >> > > > > > >> > >> > checkpointing, which greatly simplifies some
> of
> > > the
> > > > >> logic.
> > > > >> > > > > > >> > >> >
> > > > >> > > > > > >> > >> > 2. OOME errors: The main reasons why I didn't
> > > > explore
> > > > >> a
> > > > >> > > > > solution
> > > > >> > > > > > to
> > > > >> > > > > > >> > >> this is
> > > > >> > > > > > >> > >> > a) to keep this KIP as simple as possible, and
> > b)
> > > > >> because
> > > > >> > > I'm
> > > > >> > > > > not
> > > > >> > > > > > >> > >> exactly
> > > > >> > > > > > >> > >> > how to signal that a Task should commit
> > > prematurely.
> > > > >> I'm
> > > > >> > > > > > confident
> > > > >> > > > > > >> > it's
> > > > >> > > > > > >> > >> > possible, and I think it's worth adding a
> > section
> > > on
> > > > >> > > handling
> > > > >> > > > > > this.
> > > > >> > > > > > >> > >> Besides
> > > > >> > > > > > >> > >> > my proposal to force an early commit once
> memory
> > > > usage
> > > > >> > > > reaches
> > > > >> > > > > a
> > > > >> > > > > > >> > >> threshold,
> > > > >> > > > > > >> > >> > is there any other approach that you might
> > suggest
> > > > for
> > > > >> > > > tackling
> > > > >> > > > > > >> this
> > > > >> > > > > > >> > >> > problem?
> > > > >> > > > > > >> > >> >
> > > > >> > > > > > >> > >> > 3. ALOS: I can add in an explicit paragraph,
> but
> > > my
> > > > >> > > > assumption
> > > > >> > > > > is
> > > > >> > > > > > >> that
> > > > >> > > > > > >> > >> > since transactional behaviour comes at
> little/no
> > > > >> cost, that
> > > > >> > > > it
> > > > >> > > > > > >> should
> > > > >> > > > > > >> > be
> > > > >> > > > > > >> > >> > available by default on all stores,
> irrespective
> > > of
> > > > >> the
> > > > >> > > > > > processing
> > > > >> > > > > > >> > mode.
> > > > >> > > > > > >> > >> > While ALOS doesn't use transactions, the Task
> > > itself
> > > > >> still
> > > > >> > > > > > >> "commits",
> > > > >> > > > > > >> > so
> > > > >> > > > > > >> > >> > the behaviour should be correct under ALOS
> too.
> > > I'm
> > > > >> not
> > > > >> > > > > convinced
> > > > >> > > > > > >> that
> > > > >> > > > > > >> > >> it's
> > > > >> > > > > > >> > >> > worth having both
> > transactional/non-transactional
> > > > >> stores
> > > > >> > > > > > >> available, as
> > > > >> > > > > > >> > >> it
> > > > >> > > > > > >> > >> > would considerably increase the complexity of
> > the
> > > > >> codebase,
> > > > >> > > > for
> > > > >> > > > > > >> very
> > > > >> > > > > > >> > >> little
> > > > >> > > > > > >> > >> > benefit.
> > > > >> > > > > > >> > >> >
> > > > >> > > > > > >> > >> > 4. Method deprecation: Are you referring to
> > > > >> > > > > > >> StateStore#getPosition()?
> > > > >> > > > > > >> > >> As I
> > > > >> > > > > > >> > >> > understand it, Position contains the position
> of
> > > the
> > > > >> > > *source*
> > > > >> > > > > > >> topics,
> > > > >> > > > > > >> > >> > whereas the commit offsets would be the
> > > *changelog*
> > > > >> > > offsets.
> > > > >> > > > So
> > > > >> > > > > > >> it's
> > > > >> > > > > > >> > >> still
> > > > >> > > > > > >> > >> > necessary to retain the Position data, as well
> > as
> > > > the
> > > > >> > > > changelog
> > > > >> > > > > > >> > offsets.
> > > > >> > > > > > >> > >> > What I meant in the KIP is that Position
> offsets
> > > are
> > > > >> > > > currently
> > > > >> > > > > > >> stored
> > > > >> > > > > > >> > >> in a
> > > > >> > > > > > >> > >> > file, and since we can atomically store
> metadata
> > > > >> along with
> > > > >> > > > the
> > > > >> > > > > > >> record
> > > > >> > > > > > >> > >> > batch we commit to RocksDB, we can move our
> > > Position
> > > > >> > > offsets
> > > > >> > > > in
> > > > >> > > > > > to
> > > > >> > > > > > >> > this
> > > > >> > > > > > >> > >> > metadata too, and gain the same transactional
> > > > >> guarantees
> > > > >> > > that
> > > > >> > > > > we
> > > > >> > > > > > >> will
> > > > >> > > > > > >> > >> for
> > > > >> > > > > > >> > >> > changelog offsets, ensuring that the Position
> > > > offsets
> > > > >> are
> > > > >> > > > > > >> consistent
> > > > >> > > > > > >> > >> with
> > > > >> > > > > > >> > >> > the records that are read from the database.
> > > > >> > > > > > >> > >> >
> > > > >> > > > > > >> > >> > Regards,
> > > > >> > > > > > >> > >> > Nick
> > > > >> > > > > > >> > >> >
> > > > >> > > > > > >> > >> > On Tue, 22 Nov 2022 at 16:25, John Roesler <
> > > > >> > > > > vvcephei@apache.org>
> > > > >> > > > > > >> > wrote:
> > > > >> > > > > > >> > >> >
> > > > >> > > > > > >> > >> > > Thanks for publishing this alternative,
> Nick!
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > The benchmark you mentioned in the KIP-844
> > > > >> discussion
> > > > >> > > seems
> > > > >> > > > > > like
> > > > >> > > > > > >> a
> > > > >> > > > > > >> > >> > > compelling reason to revisit the built-in
> > > > >> > > transactionality
> > > > >> > > > > > >> > mechanism.
> > > > >> > > > > > >> > >> I
> > > > >> > > > > > >> > >> > > also appreciate you analysis, showing that
> for
> > > > most
> > > > >> use
> > > > >> > > > > cases,
> > > > >> > > > > > >> the
> > > > >> > > > > > >> > >> write
> > > > >> > > > > > >> > >> > > batch approach should be just fine.
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > There are a couple of points that would hold
> > me
> > > > >> back from
> > > > >> > > > > > >> approving
> > > > >> > > > > > >> > >> this
> > > > >> > > > > > >> > >> > > KIP right now:
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > 1. Loss of coverage for custom stores.
> > > > >> > > > > > >> > >> > > The fact that you can plug in a (relatively)
> > > > simple
> > > > >> > > > > > >> implementation
> > > > >> > > > > > >> > of
> > > > >> > > > > > >> > >> the
> > > > >> > > > > > >> > >> > > XStateStore interfaces and automagically
> get a
> > > > >> > > distributed
> > > > >> > > > > > >> database
> > > > >> > > > > > >> > >> out
> > > > >> > > > > > >> > >> > of
> > > > >> > > > > > >> > >> > > it is a significant benefit of Kafka
> Streams.
> > > I'd
> > > > >> hate to
> > > > >> > > > > lose
> > > > >> > > > > > >> it,
> > > > >> > > > > > >> > so
> > > > >> > > > > > >> > >> it
> > > > >> > > > > > >> > >> > > would be better to spend some time and come
> up
> > > > with
> > > > >> a way
> > > > >> > > > to
> > > > >> > > > > > >> > preserve
> > > > >> > > > > > >> > >> > that
> > > > >> > > > > > >> > >> > > property. For example, can we provide a
> > default
> > > > >> > > > > implementation
> > > > >> > > > > > of
> > > > >> > > > > > >> > >> > > `commit(..)` that re-implements the existing
> > > > >> > > > checkpoint-file
> > > > >> > > > > > >> > >> approach? Or
> > > > >> > > > > > >> > >> > > perhaps add an `isTransactional()` flag to
> the
> > > > state
> > > > >> > > store
> > > > >> > > > > > >> interface
> > > > >> > > > > > >> > >> so
> > > > >> > > > > > >> > >> > > that the runtime can decide whether to
> > continue
> > > to
> > > > >> manage
> > > > >> > > > > > >> checkpoint
> > > > >> > > > > > >> > >> > files
> > > > >> > > > > > >> > >> > > vs delegating transactionality to the
> stores?
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > 2. Guarding against OOME
> > > > >> > > > > > >> > >> > > I appreciate your analysis, but I don't
> think
> > > it's
> > > > >> > > > sufficient
> > > > >> > > > > > to
> > > > >> > > > > > >> say
> > > > >> > > > > > >> > >> that
> > > > >> > > > > > >> > >> > > we will solve the memory problem later if it
> > > > becomes
> > > > >> > > > > necessary.
> > > > >> > > > > > >> The
> > > > >> > > > > > >> > >> > > experience leading to that situation would
> be
> > > > quite
> > > > >> bad:
> > > > >> > > > > > Imagine,
> > > > >> > > > > > >> > you
> > > > >> > > > > > >> > >> > > upgrade to AK 3.next, your tests pass, so
> you
> > > > >> deploy to
> > > > >> > > > > > >> production.
> > > > >> > > > > > >> > >> That
> > > > >> > > > > > >> > >> > > night, you get paged because your app is now
> > > > >> crashing
> > > > >> > > with
> > > > >> > > > > > >> OOMEs. As
> > > > >> > > > > > >> > >> with
> > > > >> > > > > > >> > >> > > all OOMEs, you'll have a really hard time
> > > finding
> > > > >> the
> > > > >> > > root
> > > > >> > > > > > cause,
> > > > >> > > > > > >> > and
> > > > >> > > > > > >> > >> > once
> > > > >> > > > > > >> > >> > > you do, you won't have a clear path to
> resolve
> > > the
> > > > >> issue.
> > > > >> > > > You
> > > > >> > > > > > >> could
> > > > >> > > > > > >> > >> only
> > > > >> > > > > > >> > >> > > tune down the commit interval and cache
> buffer
> > > > size
> > > > >> until
> > > > >> > > > you
> > > > >> > > > > > >> stop
> > > > >> > > > > > >> > >> > getting
> > > > >> > > > > > >> > >> > > crashes.
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > FYI, I know of multiple cases where people
> run
> > > EOS
> > > > >> with
> > > > >> > > > much
> > > > >> > > > > > >> larger
> > > > >> > > > > > >> > >> > commit
> > > > >> > > > > > >> > >> > > intervals to get better batching than the
> > > default,
> > > > >> so I
> > > > >> > > > don't
> > > > >> > > > > > >> think
> > > > >> > > > > > >> > >> this
> > > > >> > > > > > >> > >> > > pathological case would be as rare as you
> > > suspect.
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > Given that we already have the rudiments of
> an
> > > > idea
> > > > >> of
> > > > >> > > what
> > > > >> > > > > we
> > > > >> > > > > > >> could
> > > > >> > > > > > >> > >> do
> > > > >> > > > > > >> > >> > to
> > > > >> > > > > > >> > >> > > prevent this downside, we should take the
> time
> > > to
> > > > >> design
> > > > >> > > a
> > > > >> > > > > > >> solution.
> > > > >> > > > > > >> > >> We
> > > > >> > > > > > >> > >> > owe
> > > > >> > > > > > >> > >> > > it to our users to ensure that awesome new
> > > > features
> > > > >> don't
> > > > >> > > > > come
> > > > >> > > > > > >> with
> > > > >> > > > > > >> > >> > bitter
> > > > >> > > > > > >> > >> > > pills unless we can't avoid it.
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > 3. ALOS mode.
> > > > >> > > > > > >> > >> > > On the other hand, I didn't see an
> indication
> > of
> > > > how
> > > > >> > > stores
> > > > >> > > > > > will
> > > > >> > > > > > >> be
> > > > >> > > > > > >> > >> > > handled under ALOS (aka non-EOS) mode.
> > > > >> Theoretically, the
> > > > >> > > > > > >> > >> > transactionality
> > > > >> > > > > > >> > >> > > of the store and the processing mode are
> > > > >> orthogonal. A
> > > > >> > > > > > >> transactional
> > > > >> > > > > > >> > >> > store
> > > > >> > > > > > >> > >> > > would serve ALOS just as well as a
> > > > >> non-transactional one
> > > > >> > > > (if
> > > > >> > > > > > not
> > > > >> > > > > > >> > >> better).
> > > > >> > > > > > >> > >> > > Under ALOS, though, the default commit
> > interval
> > > is
> > > > >> five
> > > > >> > > > > > minutes,
> > > > >> > > > > > >> so
> > > > >> > > > > > >> > >> the
> > > > >> > > > > > >> > >> > > memory issue is far more pressing.
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > As I see it, we have several options to
> > resolve
> > > > this
> > > > >> > > point.
> > > > >> > > > > We
> > > > >> > > > > > >> could
> > > > >> > > > > > >> > >> > > demonstrate that transactional stores work
> > just
> > > > >> fine for
> > > > >> > > > ALOS
> > > > >> > > > > > >> and we
> > > > >> > > > > > >> > >> can
> > > > >> > > > > > >> > >> > > therefore just swap over unconditionally. We
> > > could
> > > > >> also
> > > > >> > > > > disable
> > > > >> > > > > > >> the
> > > > >> > > > > > >> > >> > > transactional mechanism under ALOS so that
> > > stores
> > > > >> operate
> > > > >> > > > > just
> > > > >> > > > > > >> the
> > > > >> > > > > > >> > >> same
> > > > >> > > > > > >> > >> > as
> > > > >> > > > > > >> > >> > > they do today when run in ALOS mode.
> Finally,
> > we
> > > > >> could do
> > > > >> > > > the
> > > > >> > > > > > >> same
> > > > >> > > > > > >> > as
> > > > >> > > > > > >> > >> in
> > > > >> > > > > > >> > >> > > KIP-844 and make transactional stores opt-in
> > > (it'd
> > > > >> be
> > > > >> > > > better
> > > > >> > > > > to
> > > > >> > > > > > >> > avoid
> > > > >> > > > > > >> > >> the
> > > > >> > > > > > >> > >> > > extra opt-in mechanism, but it's a good
> > > > >> > > > get-out-of-jail-free
> > > > >> > > > > > >> card).
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > 4. (minor point) Deprecation of methods
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > You mentioned that the new `commit` 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?
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > Thanks again for the KIP! It's really nice
> > that
> > > > you
> > > > >> and
> > > > >> > > > Alex
> > > > >> > > > > > will
> > > > >> > > > > > >> > get
> > > > >> > > > > > >> > >> the
> > > > >> > > > > > >> > >> > > chance to collaborate on both directions so
> > that
> > > > we
> > > > >> can
> > > > >> > > get
> > > > >> > > > > the
> > > > >> > > > > > >> best
> > > > >> > > > > > >> > >> > > outcome for Streams and its users.
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > -John
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> > > On 2022/11/21 15:02:15 Nick Telford wrote:
> > > > >> > > > > > >> > >> > > > 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
> > > > >> > > > > > >> > >> > > > <
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> >
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> >
> > > > >> > > > > > >>
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > >> > > > > > >> > >> > > >,
> > > > >> > > > > > >> > >> > > > so that it can be discussed/reviewed
> > > separately
> > > > >> from
> > > > >> > > > > KIP-844.
> > > > >> > > > > > >> > >> > > >
> > > > >> > > > > > >> > >> > > > Alex: I'm especially interested in what
> you
> > > > think!
> > > > >> > > > > > >> > >> > > >
> > > > >> > > > > > >> > >> > > > I have a nearly complete implementation of
> > the
> > > > >> changes
> > > > >> > > > > > >> outlined in
> > > > >> > > > > > >> > >> this
> > > > >> > > > > > >> > >> > > > KIP, please let me know if you'd like me
> to
> > > push
> > > > >> them
> > > > >> > > for
> > > > >> > > > > > >> review
> > > > >> > > > > > >> > in
> > > > >> > > > > > >> > >> > > advance
> > > > >> > > > > > >> > >> > > > of a vote.
> > > > >> > > > > > >> > >> > > >
> > > > >> > > > > > >> > >> > > > Regards,
> > > > >> > > > > > >> > >> > > >
> > > > >> > > > > > >> > >> > > > Nick
> > > > >> > > > > > >> > >> > > >
> > > > >> > > > > > >> > >> > >
> > > > >> > > > > > >> > >> >
> > > > >> > > > > > >> > >>
> > > > >> > > > > > >> > >
> > > > >> > > > > > >> >
> > > > >> > > > > > >>
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >>
> > > > >
> > > >
> > >
> >
>

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

Posted by Colt McNealy <co...@littlehorse.io>.
Nick,

Thanks for your reply. Ack to A) and B).

For item C), I see what you're referring to. Your proposed solution will
work, so no need to change it. What I was suggesting was that it might be
possible to achieve this with only one column family. So long as:

   - No uncommitted records (i.e. not committed to the changelog) are
   *committed* to the state store, AND
   - The Checkpoint offset (which refers to the changelog topic) is less
   than or equal to the last written changelog offset in rocksdb

I don't see the need to do the full restoration from scratch. My
understanding was that prior to 844/892, full restorations were required
because there could be uncommitted records written to RocksDB; however,
given your use of RocksDB transactions, that can be avoided with the
pattern of 1) commit Kafka transaction, 2) commit RocksDB transaction, 3)
update offset in checkpoint 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, 2023 at 5:53 AM Nick Telford <ni...@gmail.com> wrote:

> Hi Colt,
>
> A. I've done my best to de-couple the StateStore stuff from the rest of the
> Streams engine. The fact that there will be only one ongoing (write)
> transaction at a time is not guaranteed by any API, and is just a
> consequence of the way Streams operates. To that end, I tried to ensure the
> documentation and guarantees provided by the new APIs are independent of
> this incidental behaviour. In practice, you're right, this essentially
> refers to "interactive queries", which are technically "read transactions",
> even if they don't actually use the transaction API to isolate themselves.
>
> B. Yes, although not ideal. This is for backwards compatibility, because:
>     1) Existing custom StateStore implementations will implement flush(),
> and not commit(), but the Streams engine now calls commit(), so those calls
> need to be forwarded to flush() for these legacy stores.
>     2) Existing StateStore *users*, i.e. outside of the Streams engine
> itself, may depend on explicitly calling flush(), so for these cases,
> flush() needs to be redirected to call commit().
> If anyone has a better way to guarantee compatibility without introducing
> this potential recursion loop, I'm open to changes!
>
> C. This is described in the "Atomic Checkpointing" section. Offsets are
> stored in a separate RocksDB column family, which is guaranteed to be
> atomically flushed to disk with all other column families. The issue of
> checkpoints being written to disk after commit causing inconsistency if it
> crashes in between is the reason why, under EOS, checkpoint files are only
> written on clean shutdown. This is one of the major causes of "full
> restorations", so moving the offsets into a place where they can be
> guaranteed to be atomically written with the data they checkpoint allows us
> to write the checkpoint offsets *on every commit*, not just on clean
> shutdown.
>
> Regards,
> Nick
>
> On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io> wrote:
>
> > Nick,
> >
> > Thank you for continuing this work. I have a few minor clarifying
> > questions.
> >
> > A) "Records written to any transaction are visible to all other
> > transactions immediately." I am confused here—I thought there could only
> be
> > one transaction going on at a time for a given state store given the
> > threading model for processing records on a Task. Do you mean Interactive
> > Queries by "other transactions"? (If so, then everything makes sense—I
> > thought that since IQ were read-only then they didn't count as
> > transactions).
> >
> > B) Is it intentional that the default implementations of the flush() and
> > commit() methods in the StateStore class refer to each other in some sort
> > of unbounded recursion?
> >
> > C) How will the getCommittedOffset() method work? At first I thought the
> > way to do it would be using a special key in the RocksDB store to store
> the
> > offset, and committing that with the transaction. But upon second
> thought,
> > since restoration from the changelog is an idempotent procedure, I think
> it
> > would be fine to 1) commit the RocksDB transaction and then 2) write the
> > offset to disk in a checkpoint file. If there is a crash between 1) and
> 2),
> > I think the only downside is 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 <ni...@gmail.com>
> > wrote:
> >
> > > Hi everyone,
> > >
> > > I've updated the KIP to reflect the latest version of the design:
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > >
> > > There are several changes in there that reflect feedback from this
> > thread,
> > > and there's a new section and a bunch of interface changes relating to
> > > Atomic Checkpointing, which is the final piece of the puzzle to making
> > > everything robust.
> > >
> > > Let me know what you think!
> > >
> > > Regards,
> > > Nick
> > >
> > > On Tue, 3 Jan 2023 at 11:33, Nick Telford <ni...@gmail.com>
> > wrote:
> > >
> > > > Hi Lucas,
> > > >
> > > > Thanks for looking over my KIP.
> > > >
> > > > A) The bound is per-instance, not per-Task. This was a typo in the
> KIP
> > > > that I've now corrected. It was originally per-Task, but I changed it
> > to
> > > > per-instance for exactly the reason you highlighted.
> > > > B) It's worth noting that transactionality is only enabled under EOS,
> > and
> > > > in the default mode of operation (ALOS), there should be no change in
> > > > behavior at all. I think, under EOS, we can mitigate the impact on
> > users
> > > by
> > > > sufficiently low default values for the memory bound configuration. I
> > > > understand your hesitation to include a significant change of
> > behaviour,
> > > > especially in a minor release, but I suspect that most users will
> > prefer
> > > > the memory impact (under EOS) to the existing behaviour of frequent
> > state
> > > > restorations! If this is a problem, the changes can wait until the
> next
> > > > major release. I'll be running a patched version of streams in
> > production
> > > > with these changes as soon as they're ready, so it won't disrupt me
> :-D
> > > > C) The main purpose of this sentence was just to note that some
> changes
> > > > will need to be made to the way Segments are handled in order to
> ensure
> > > > they also benefit from transactions. At the time I wrote it, I hadn't
> > > > figured out the specific changes necessary, so it was deliberately
> > vague.
> > > > This is the one outstanding problem I'm currently working on, and
> I'll
> > > > update this section with more detail once I have figured out the
> exact
> > > > changes required.
> > > > D) newTransaction() provides the necessary isolation guarantees.
> While
> > > > the RocksDB implementation of transactions doesn't technically *need*
> > > > read-only users to call newTransaction(), other implementations
> (e.g. a
> > > > hypothetical PostgresStore) may require it. Calling newTransaction()
> > when
> > > > no transaction is necessary is essentially free, as it will just
> return
> > > > this.
> > > >
> > > > I didn't do any profiling of the KIP-844 PoC, but I think it should
> be
> > > > fairly obvious where the performance problems stem from: writes under
> > > > KIP-844 require 3 extra memory-copies: 1 to encode it with the
> > > > tombstone/record flag, 1 to decode it from the tombstone/record flag,
> > > and 1
> > > > to copy the record from the "temporary" store to the "main" store,
> when
> > > the
> > > > transaction commits. The different approach taken by KIP-869 should
> > > perform
> > > > much better, as it avoids all these copies, and may actually perform
> > > > slightly better than trunk, due to batched writes in RocksDB
> performing
> > > > better than non-batched writes.[1]
> > > >
> > > > Regards,
> > > > Nick
> > > >
> > > > 1:
> > >
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > >
> > > > On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <lbrutschy@confluent.io
> > > .invalid>
> > > > wrote:
> > > >
> > > >> Hi Nick,
> > > >>
> > > >> I'm just starting to read up on the whole discussion about KIP-892
> and
> > > >> KIP-844. Thanks a lot for your work on this, I do think
> > > >> `WriteBatchWithIndex` may be the way to go here. I do have some
> > > >> questions about the latest draft.
> > > >>
> > > >>  A) If I understand correctly, you propose to put a bound on the
> > > >> (native) memory consumed by each task. However, I wonder if this is
> > > >> sufficient if we have temporary imbalances in the cluster. For
> > > >> example, depending on the timing of rebalances during a cluster
> > > >> restart, it could happen that a single streams node is assigned a
> lot
> > > >> more tasks than expected. With your proposed change, this would mean
> > > >> that the memory required by this one node could be a multiple of
> what
> > > >> is required during normal operation. I wonder if it wouldn't be
> safer
> > > >> to put a global bound on the memory use, across all tasks.
> > > >>  B) Generally, the memory concerns still give me the feeling that
> this
> > > >> should not be enabled by default for all users in a minor release.
> > > >>  C) In section "Transaction Management": the sentence "A similar
> > > >> analogue will be created to automatically manage `Segment`
> > > >> transactions.". Maybe this is just me lacking some background, 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 you to find out why the KIP-844 PoC is 20x slower (e.g.
> > > >> by providing a flame graph), that would be quite interesting.
> > > >>
> > > >> Cheers,
> > > >> Lucas
> > > >>
> > > >> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <
> nick.telford@gmail.com>
> > > >> wrote:
> > > >> >
> > > >> > Hi everyone,
> > > >> >
> > > >> > I've updated the KIP with a more detailed design, which reflects
> the
> > > >> > implementation I've been working on:
> > > >> >
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > >> >
> > > >> > This new design should address the outstanding points already made
> > in
> > > >> the
> > > >> > thread.
> > > >> >
> > > >> > Please let me know if there are areas that are unclear or need
> more
> > > >> > clarification.
> > > >> >
> > > >> > I have a (nearly) working implementation. I'm confident that the
> > > >> remaining
> > > >> > work (making Segments behave) will not impact the documented
> design.
> > > >> >
> > > >> > Regards,
> > > >> >
> > > >> > Nick
> > > >> >
> > > >> > On Tue, 6 Dec 2022 at 19:24, Colt McNealy <co...@littlehorse.io>
> > > wrote:
> > > >> >
> > > >> > > Nick,
> > > >> > >
> > > >> > > Thank you for the reply; that makes sense. I was hoping that,
> > since
> > > >> reading
> > > >> > > uncommitted records from IQ in EOS isn't part of the documented
> > API,
> > > >> maybe
> > > >> > > you *wouldn't* have to wait for the next major release to make
> > that
> > > >> change;
> > > >> > > but given 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 <
> > > nick.telford@gmail.com>
> > > >> > > wrote:
> > > >> > >
> > > >> > > > Hi Colt,
> > > >> > > >
> > > >> > > > 10: Yes, I agree it's not ideal. I originally intended to try
> to
> > > >> keep the
> > > >> > > > behaviour unchanged as much as possible, otherwise we'd have
> to
> > > >> wait for
> > > >> > > a
> > > >> > > > major version release to land these changes.
> > > >> > > > 20: Good point, ALOS doesn't need the same level of guarantee,
> > and
> > > >> the
> > > >> > > > typically longer commit intervals would be problematic when
> > > reading
> > > >> only
> > > >> > > > "committed" records.
> > > >> > > >
> > > >> > > > I've been away for 5 days recovering from minor surgery, but I
> > > >> spent a
> > > >> > > > considerable amount of that time working through ideas for
> > > possible
> > > >> > > > solutions in my head. I think your suggestion of keeping ALOS
> > > >> as-is, but
> > > >> > > > buffering writes for EOS is the right path forwards, although
> I
> > > >> have a
> > > >> > > > solution that both expands on this, and provides for some more
> > > >> formal
> > > >> > > > guarantees.
> > > >> > > >
> > > >> > > > Essentially, adding support to KeyValueStores for
> > "Transactions",
> > > >> with
> > > >> > > > clearly defined IsolationLevels. Using "Read Committed" when
> > under
> > > >> EOS,
> > > >> > > and
> > > >> > > > "Read Uncommitted" under ALOS.
> > > >> > > >
> > > >> > > > The nice thing about this approach is that it gives us much
> more
> > > >> clearly
> > > >> > > > defined isolation behaviour that can be properly documented to
> > > >> ensure
> > > >> > > users
> > > >> > > > know what to expect.
> > > >> > > >
> > > >> > > > I'm still working out the kinks in the design, and will update
> > the
> > > >> KIP
> > > >> > > when
> > > >> > > > I have something. The main struggle is trying to implement
> this
> > > >> without
> > > >> > > > making any major changes to the existing interfaces or
> breaking
> > > >> existing
> > > >> > > > implementations, because currently everything expects to
> operate
> > > >> directly
> > > >> > > > on a StateStore, and not a Transaction of that store. I think
> > I'm
> > > >> getting
> > > >> > > > close, although sadly I won't be able to progress much until
> > next
> > > >> week
> > > >> > > due
> > > >> > > > to some work commitments.
> > > >> > > >
> > > >> > > > Regards,
> > > >> > > > Nick
> > > >> > > >
> > > >> > > > On Thu, 1 Dec 2022 at 00:01, Colt McNealy <
> colt@littlehorse.io>
> > > >> wrote:
> > > >> > > >
> > > >> > > > > Nick,
> > > >> > > > >
> > > >> > > > > Thank you for the explanation, and also for the updated
> KIP. I
> > > am
> > > >> quite
> > > >> > > > > eager for this improvement to be released as it would
> greatly
> > > >> reduce
> > > >> > > the
> > > >> > > > > operational difficulties of EOS streams apps.
> > > >> > > > >
> > > >> > > > > Two questions:
> > > >> > > > >
> > > >> > > > > 10)
> > > >> > > > > >When reading records, we will use the
> > > >> > > > > WriteBatchWithIndex#getFromBatchAndDB
> > > >> > > > >  and WriteBatchWithIndex#newIteratorWithBase utilities in
> > order
> > > to
> > > >> > > ensure
> > > >> > > > > that uncommitted writes are available to query.
> > > >> > > > > Why do extra work to enable the reading of uncommitted
> writes
> > > >> during
> > > >> > > IQ?
> > > >> > > > > Code complexity aside, reading uncommitted writes is, in my
> > > >> opinion, a
> > > >> > > > > minor flaw in EOS IQ; it would be very nice to have the
> > > guarantee
> > > >> that,
> > > >> > > > > with EOS, IQ only reads committed records. In order to avoid
> > > dirty
> > > >> > > reads,
> > > >> > > > > one currently must query a standby replica (but this still
> > > doesn't
> > > >> > > fully
> > > >> > > > > guarantee monotonic reads).
> > > >> > > > >
> > > >> > > > > 20) Is it also necessary to enable this optimization on ALOS
> > > >> stores?
> > > >> > > The
> > > >> > > > > motivation of KIP-844 was mainly to reduce the need to
> restore
> > > >> state
> > > >> > > from
> > > >> > > > > scratch on unclean EOS shutdowns; with ALOS it was
> acceptable
> > to
> > > >> accept
> > > >> > > > > that there may have been uncommitted writes on disk. On a
> side
> > > >> note, if
> > > >> > > > you
> > > >> > > > > enable this type of store on ALOS processors, the community
> > > would
> > > >> > > > > 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 <
> > > >> nick.telford@gmail.com>
> > > >> > > > > wrote:
> > > >> > > > >
> > > >> > > > > > Hi everyone,
> > > >> > > > > >
> > > >> > > > > > I've drastically reduced the scope of this KIP to no
> longer
> > > >> include
> > > >> > > the
> > > >> > > > > > StateStore management of checkpointing. This can be added
> > as a
> > > >> KIP
> > > >> > > > later
> > > >> > > > > on
> > > >> > > > > > to further optimize the consistency and performance of
> state
> > > >> stores.
> > > >> > > > > >
> > > >> > > > > > I've also added a section discussing some of the concerns
> > > 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 2022 at 16:36, Nick Telford <
> > > >> nick.telford@gmail.com>
> > > >> > > > > wrote:
> > > >> > > > > >
> > > >> > > > > > > Hi Colt,
> > > >> > > > > > >
> > > >> > > > > > > I didn't do any profiling, but the 844 implementation:
> > > >> > > > > > >
> > > >> > > > > > >    - Writes uncommitted records to a temporary RocksDB
> > > >> instance
> > > >> > > > > > >       - Since tombstones need to be flagged, all record
> > > >> values are
> > > >> > > > > > >       prefixed with a value/tombstone marker. This
> > > >> necessitates a
> > > >> > > > > memory
> > > >> > > > > > copy.
> > > >> > > > > > >    - On-commit, iterates all records in this temporary
> > > >> instance and
> > > >> > > > > > >    writes them to the main RocksDB store.
> > > >> > > > > > >    - While iterating, the value/tombstone marker needs
> to
> > be
> > > >> parsed
> > > >> > > > and
> > > >> > > > > > >    the real value extracted. This necessitates another
> > > memory
> > > >> copy.
> > > >> > > > > > >
> > > >> > > > > > > My guess is that the cost of iterating the temporary
> > RocksDB
> > > >> store
> > > >> > > is
> > > >> > > > > the
> > > >> > > > > > > major factor, with the 2 extra memory copies per-Record
> > > >> > > contributing
> > > >> > > > a
> > > >> > > > > > > significant amount too.
> > > >> > > > > > >
> > > >> > > > > > > Regards,
> > > >> > > > > > > Nick
> > > >> > > > > > >
> > > >> > > > > > > On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
> > > >> colt@littlehorse.io>
> > > >> > > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > >> Hi all,
> > > >> > > > > > >>
> > > >> > > > > > >> Out of curiosity, why does the performance of the store
> > > >> degrade so
> > > >> > > > > > >> significantly with the 844 implementation? I wouldn't
> be
> > > too
> > > >> > > > surprised
> > > >> > > > > > by
> > > >> > > > > > >> a
> > > >> > > > > > >> 50-60% drop (caused by each record being written
> twice),
> > > but
> > > >> 96%
> > > >> > > is
> > > >> > > > > > >> extreme.
> > > >> > > > > > >>
> > > >> > > > > > >> The only thing I can think of which could create such a
> > > >> bottleneck
> > > >> > > > > 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 Nick Telford <
> > > >> > > > nick.telford@gmail.com>
> > > >> > > > > > >> wrote:
> > > >> > > > > > >>
> > > >> > > > > > >> > Hi everyone,
> > > >> > > > > > >> >
> > > >> > > > > > >> > I've updated the KIP to resolve all the points that
> > have
> > > >> been
> > > >> > > > raised
> > > >> > > > > > so
> > > >> > > > > > >> > far, with one exception: the ALOS default commit
> > interval
> > > >> of 5
> > > >> > > > > minutes
> > > >> > > > > > >> is
> > > >> > > > > > >> > likely to cause WriteBatchWithIndex memory to grow
> too
> > > >> large.
> > > >> > > > > > >> >
> > > >> > > > > > >> > There's a couple of different things I can think of
> to
> > > >> solve
> > > >> > > this:
> > > >> > > > > > >> >
> > > >> > > > > > >> >    - We already have a memory/record limit in the KIP
> > to
> > > >> prevent
> > > >> > > > OOM
> > > >> > > > > > >> >    errors. Should we choose a default value for
> these?
> > My
> > > >> > > concern
> > > >> > > > > here
> > > >> > > > > > >> is
> > > >> > > > > > >> > that
> > > >> > > > > > >> >    anything we choose might seem rather arbitrary. We
> > > could
> > > >> > > change
> > > >> > > > > > >> >    its behaviour such that under ALOS, it only
> triggers
> > > the
> > > >> > > commit
> > > >> > > > > of
> > > >> > > > > > >> the
> > > >> > > > > > >> >    StateStore, but under EOS, it triggers a commit of
> > the
> > > >> Kafka
> > > >> > > > > > >> > transaction.
> > > >> > > > > > >> >    - We could introduce a separate `
> > > checkpoint.interval.ms`
> > > >> to
> > > >> > > > > allow
> > > >> > > > > > >> ALOS
> > > >> > > > > > >> >    to commit the StateStores more frequently than the
> > > >> general
> > > >> > > > > > >> >    commit.interval.ms? My concern here is that the
> > > >> semantics of
> > > >> > > > > this
> > > >> > > > > > >> > config
> > > >> > > > > > >> >    would depend 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 <
> > > >> > > > nick.telford@gmail.com>
> > > >> > > > > > >> wrote:
> > > >> > > > > > >> >
> > > >> > > > > > >> > > Hi Alex,
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > Thanks for the feedback.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > I've updated the discussion of OOM issues by
> > describing
> > > >> how
> > > >> > > > we'll
> > > >> > > > > > >> handle
> > > >> > > > > > >> > > it. Here's the new text:
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > To mitigate this, we will automatically force a
> Task
> > > >> commit if
> > > >> > > > the
> > > >> > > > > > >> total
> > > >> > > > > > >> > >> uncommitted records returned by
> > > >> > > > > > >> > >> StateStore#approximateNumUncommittedEntries()
> > > exceeds a
> > > >> > > > > threshold,
> > > >> > > > > > >> > >> configured by
> > max.uncommitted.state.entries.per.task;
> > > >> or the
> > > >> > > > > total
> > > >> > > > > > >> > >> memory used for buffering uncommitted records
> > returned
> > > >> by
> > > >> > > > > > >> > >> StateStore#approximateNumUncommittedBytes()
> exceeds
> > > the
> > > >> > > > threshold
> > > >> > > > > > >> > >> configured by
> max.uncommitted.state.bytes.per.task.
> > > >> This will
> > > >> > > > > > roughly
> > > >> > > > > > >> > >> bound the memory required per-Task for buffering
> > > >> uncommitted
> > > >> > > > > > records,
> > > >> > > > > > >> > >> irrespective of the commit.interval.ms, and will
> > > >> effectively
> > > >> > > > > bound
> > > >> > > > > > >> the
> > > >> > > > > > >> > >> number of records that will need to be restored in
> > the
> > > >> event
> > > >> > > > of a
> > > >> > > > > > >> > failure.
> > > >> > > > > > >> > >>
> > > >> > > > > > >> > >
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > These limits will be checked in StreamTask#process
> > and
> > > a
> > > >> > > > premature
> > > >> > > > > > >> commit
> > > >> > > > > > >> > >> will be requested via Task#requestCommit().
> > > >> > > > > > >> > >>
> > > >> > > > > > >> > >
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > Note that these new methods provide default
> > > >> implementations
> > > >> > > that
> > > >> > > > > > >> ensure
> > > >> > > > > > >> > >> existing custom stores and non-transactional
> stores
> > > >> (e.g.
> > > >> > > > > > >> > >> InMemoryKeyValueStore) do not force any early
> > commits.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > I've chosen to have the StateStore expose
> > > approximations
> > > >> of
> > > >> > > its
> > > >> > > > > > buffer
> > > >> > > > > > >> > > size/count instead of opaquely requesting a commit
> in
> > > >> order to
> > > >> > > > > > >> delegate
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > decision making to the Task itself. This enables
> > Tasks
> > > >> to look
> > > >> > > > at
> > > >> > > > > > >> *all*
> > > >> > > > > > >> > of
> > > >> > > > > > >> > > their StateStores, and determine whether an early
> > > commit
> > > >> is
> > > >> > > > > > necessary.
> > > >> > > > > > >> > > Notably, it enables pre-Task thresholds, instead of
> > > >> per-Store,
> > > >> > > > > which
> > > >> > > > > > >> > > prevents Tasks with many StateStores from using
> much
> > > more
> > > >> > > memory
> > > >> > > > > > than
> > > >> > > > > > >> > Tasks
> > > >> > > > > > >> > > with one StateStore. This makes sense, since
> commits
> > > are
> > > >> done
> > > >> > > > > > by-Task,
> > > >> > > > > > >> > not
> > > >> > > > > > >> > > by-Store.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > Prizes* for anyone who can come up with a better
> name
> > > >> for the
> > > >> > > > new
> > > >> > > > > > >> config
> > > >> > > > > > >> > > properties!
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > Thanks for pointing out the potential performance
> > > issues
> > > >> of
> > > >> > > > WBWI.
> > > >> > > > > > From
> > > >> > > > > > >> > the
> > > >> > > > > > >> > > benchmarks that user posted[1], it looks like WBWI
> > > still
> > > >> > > > performs
> > > >> > > > > > >> > > considerably better than individual puts, which is
> > the
> > > >> > > existing
> > > >> > > > > > >> design,
> > > >> > > > > > >> > so
> > > >> > > > > > >> > > I'd actually expect a performance boost from WBWI,
> > just
> > > >> not as
> > > >> > > > > great
> > > >> > > > > > >> as
> > > >> > > > > > >> > > we'd get from a plain WriteBatch. This does suggest
> > > that
> > > >> a
> > > >> > > good
> > > >> > > > > > >> > > optimization would be to use a regular WriteBatch
> for
> > > >> > > > restoration
> > > >> > > > > > (in
> > > >> > > > > > >> > > RocksDBStore#restoreBatch), since we know that
> those
> > > >> records
> > > >> > > > will
> > > >> > > > > > >> never
> > > >> > > > > > >> > be
> > > >> > > > > > >> > > queried before they're committed.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > 1:
> > > >> > > > > > >> >
> > > >> > > > > >
> > > >> > > >
> > > >>
> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > * Just kidding, no prizes, sadly.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > On Wed, 23 Nov 2022 at 12:28, Alexander Sorokoumov
> > > >> > > > > > >> > > <as...@confluent.io.invalid> wrote:
> > > >> > > > > > >> > >
> > > >> > > > > > >> > >> 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 to keep the current
> > checkpointing
> > > >> as a
> > > >> > > > > > fallback
> > > >> > > > > > >> > >> mechanism. We want to keep existing users'
> workflows
> > > >> intact
> > > >> > > if
> > > >> > > > we
> > > >> > > > > > >> can. A
> > > >> > > > > > >> > >> non-intrusive way would be to add a separate
> > > StateStore
> > > >> > > method,
> > > >> > > > > > say,
> > > >> > > > > > >> > >> StateStore#managesCheckpointing(), that controls
> > > >> whether the
> > > >> > > > > state
> > > >> > > > > > >> store
> > > >> > > > > > >> > >> implementation owns checkpointing.
> > > >> > > > > > >> > >>
> > > >> > > > > > >> > >> I think that a solution to the transactional
> writes
> > > >> should
> > > >> > > > > address
> > > >> > > > > > >> the
> > > >> > > > > > >> > >> OOMEs. One possible way to address that is to wire
> > > >> > > StateStore's
> > > >> > > > > > >> commit
> > > >> > > > > > >> > >> request by adding, say, StateStore#commitNeeded
> that
> > > is
> > > >> > > checked
> > > >> > > > > in
> > > >> > > > > > >> > >> StreamTask#commitNeeded via the corresponding
> > > >> > > > > > ProcessorStateManager.
> > > >> > > > > > >> > With
> > > >> > > > > > >> > >> that change, RocksDBStore will have to track the
> > > current
> > > >> > > > > > transaction
> > > >> > > > > > >> > size
> > > >> > > > > > >> > >> and request a commit when the size goes over a
> > > >> (configurable)
> > > >> > > > > > >> threshold.
> > > >> > > > > > >> > >>
> > > >> > > > > > >> > >> AFAIU WriteBatchWithIndex might perform
> > significantly
> > > >> slower
> > > >> > > > than
> > > >> > > > > > >> > non-txn
> > > >> > > > > > >> > >> puts as the batch size grows [1]. We should have a
> > > >> > > > configuration
> > > >> > > > > to
> > > >> > > > > > >> fall
> > > >> > > > > > >> > >> back to the current behavior (and/or disable txn
> > > stores
> > > >> for
> > > >> > > > ALOS)
> > > >> > > > > > >> unless
> > > >> > > > > > >> > >> the benchmarks show negligible overhead for longer
> > > >> commits /
> > > >> > > > > > >> > large-enough
> > > >> > > > > > >> > >> batch sizes.
> > > >> > > > > > >> > >>
> > > >> > > > > > >> > >> If you prefer to keep the KIP smaller, I would
> > rather
> > > >> cut out
> > > >> > > > > > >> > >> state-store-managed checkpointing rather than
> proper
> > > >> OOMe
> > > >> > > > > handling
> > > >> > > > > > >> and
> > > >> > > > > > >> > >> being able to switch to non-txn behavior. The
> > > >> checkpointing
> > > >> > > 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 configuration parts as well as review and
> > > test
> > > >> the
> > > >> > > > > patch.
> > > >> > > > > > >> > >>
> > > >> > > > > > >> > >> Best,
> > > >> > > > > > >> > >> Alex
> > > >> > > > > > >> > >>
> > > >> > > > > > >> > >>
> > > >> > > > > > >> > >> 1. https://github.com/facebook/rocksdb/issues/608
> > > >> > > > > > >> > >>
> > > >> > > > > > >> > >> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
> > > >> > > > > > nick.telford@gmail.com
> > > >> > > > > > >> >
> > > >> > > > > > >> > >> wrote:
> > > >> > > > > > >> > >>
> > > >> > > > > > >> > >> > Hi John,
> > > >> > > > > > >> > >> >
> > > >> > > > > > >> > >> > Thanks for the review and feedback!
> > > >> > > > > > >> > >> >
> > > >> > > > > > >> > >> > 1. Custom Stores: I've been mulling over this
> > > problem
> > > >> > > myself.
> > > >> > > > > As
> > > >> > > > > > it
> > > >> > > > > > >> > >> stands,
> > > >> > > > > > >> > >> > custom stores would essentially lose
> checkpointing
> > > >> with no
> > > >> > > > > > >> indication
> > > >> > > > > > >> > >> that
> > > >> > > > > > >> > >> > they're expected to make changes, besides a line
> > in
> > > >> the
> > > >> > > > release
> > > >> > > > > > >> > notes. I
> > > >> > > > > > >> > >> > agree that the best solution would be to
> provide a
> > > >> default
> > > >> > > > that
> > > >> > > > > > >> > >> checkpoints
> > > >> > > > > > >> > >> > to a file. The one thing I would change is that
> > the
> > > >> > > > > checkpointing
> > > >> > > > > > >> is
> > > >> > > > > > >> > to
> > > >> > > > > > >> > >> a
> > > >> > > > > > >> > >> > store-local file, instead of a per-Task file.
> This
> > > >> way the
> > > >> > > > > > >> StateStore
> > > >> > > > > > >> > >> still
> > > >> > > > > > >> > >> > technically owns its own checkpointing (via a
> > > default
> > > >> > > > > > >> implementation),
> > > >> > > > > > >> > >> and
> > > >> > > > > > >> > >> > the StateManager/Task execution engine doesn't
> > need
> > > >> to know
> > > >> > > > > > >> anything
> > > >> > > > > > >> > >> about
> > > >> > > > > > >> > >> > checkpointing, which greatly simplifies some of
> > the
> > > >> logic.
> > > >> > > > > > >> > >> >
> > > >> > > > > > >> > >> > 2. OOME errors: The main reasons why I didn't
> > > explore
> > > >> a
> > > >> > > > > solution
> > > >> > > > > > to
> > > >> > > > > > >> > >> this is
> > > >> > > > > > >> > >> > a) to keep this KIP as simple as possible, and
> b)
> > > >> because
> > > >> > > I'm
> > > >> > > > > not
> > > >> > > > > > >> > >> exactly
> > > >> > > > > > >> > >> > how to signal that a Task should commit
> > prematurely.
> > > >> I'm
> > > >> > > > > > confident
> > > >> > > > > > >> > it's
> > > >> > > > > > >> > >> > possible, and I think it's worth adding a
> section
> > on
> > > >> > > handling
> > > >> > > > > > this.
> > > >> > > > > > >> > >> Besides
> > > >> > > > > > >> > >> > my proposal to force an early commit once memory
> > > usage
> > > >> > > > reaches
> > > >> > > > > a
> > > >> > > > > > >> > >> threshold,
> > > >> > > > > > >> > >> > is there any other approach that you might
> suggest
> > > for
> > > >> > > > tackling
> > > >> > > > > > >> this
> > > >> > > > > > >> > >> > problem?
> > > >> > > > > > >> > >> >
> > > >> > > > > > >> > >> > 3. ALOS: I can add in an explicit paragraph, but
> > my
> > > >> > > > assumption
> > > >> > > > > is
> > > >> > > > > > >> that
> > > >> > > > > > >> > >> > since transactional behaviour comes at little/no
> > > >> cost, that
> > > >> > > > it
> > > >> > > > > > >> should
> > > >> > > > > > >> > be
> > > >> > > > > > >> > >> > available by default on all stores, irrespective
> > of
> > > >> the
> > > >> > > > > > processing
> > > >> > > > > > >> > mode.
> > > >> > > > > > >> > >> > While ALOS doesn't use transactions, the Task
> > itself
> > > >> still
> > > >> > > > > > >> "commits",
> > > >> > > > > > >> > so
> > > >> > > > > > >> > >> > the behaviour should be correct under ALOS too.
> > I'm
> > > >> not
> > > >> > > > > convinced
> > > >> > > > > > >> that
> > > >> > > > > > >> > >> it's
> > > >> > > > > > >> > >> > worth having both
> transactional/non-transactional
> > > >> stores
> > > >> > > > > > >> available, as
> > > >> > > > > > >> > >> it
> > > >> > > > > > >> > >> > would considerably increase the complexity of
> the
> > > >> codebase,
> > > >> > > > for
> > > >> > > > > > >> very
> > > >> > > > > > >> > >> little
> > > >> > > > > > >> > >> > benefit.
> > > >> > > > > > >> > >> >
> > > >> > > > > > >> > >> > 4. Method deprecation: Are you referring to
> > > >> > > > > > >> StateStore#getPosition()?
> > > >> > > > > > >> > >> As I
> > > >> > > > > > >> > >> > understand it, Position contains the position of
> > the
> > > >> > > *source*
> > > >> > > > > > >> topics,
> > > >> > > > > > >> > >> > whereas the commit offsets would be the
> > *changelog*
> > > >> > > offsets.
> > > >> > > > So
> > > >> > > > > > >> it's
> > > >> > > > > > >> > >> still
> > > >> > > > > > >> > >> > necessary to retain the Position data, as well
> as
> > > the
> > > >> > > > changelog
> > > >> > > > > > >> > offsets.
> > > >> > > > > > >> > >> > What I meant in the KIP is that Position offsets
> > are
> > > >> > > > currently
> > > >> > > > > > >> stored
> > > >> > > > > > >> > >> in a
> > > >> > > > > > >> > >> > file, and since we can atomically store metadata
> > > >> along with
> > > >> > > > the
> > > >> > > > > > >> record
> > > >> > > > > > >> > >> > batch we commit to RocksDB, we can move our
> > Position
> > > >> > > offsets
> > > >> > > > in
> > > >> > > > > > to
> > > >> > > > > > >> > this
> > > >> > > > > > >> > >> > metadata too, and gain the same transactional
> > > >> guarantees
> > > >> > > that
> > > >> > > > > we
> > > >> > > > > > >> will
> > > >> > > > > > >> > >> for
> > > >> > > > > > >> > >> > changelog offsets, ensuring that the Position
> > > offsets
> > > >> are
> > > >> > > > > > >> consistent
> > > >> > > > > > >> > >> with
> > > >> > > > > > >> > >> > the records that are read from the database.
> > > >> > > > > > >> > >> >
> > > >> > > > > > >> > >> > Regards,
> > > >> > > > > > >> > >> > Nick
> > > >> > > > > > >> > >> >
> > > >> > > > > > >> > >> > On Tue, 22 Nov 2022 at 16:25, John Roesler <
> > > >> > > > > vvcephei@apache.org>
> > > >> > > > > > >> > wrote:
> > > >> > > > > > >> > >> >
> > > >> > > > > > >> > >> > > Thanks for publishing this alternative, Nick!
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > The benchmark you mentioned in the KIP-844
> > > >> discussion
> > > >> > > seems
> > > >> > > > > > like
> > > >> > > > > > >> a
> > > >> > > > > > >> > >> > > compelling reason to revisit the built-in
> > > >> > > transactionality
> > > >> > > > > > >> > mechanism.
> > > >> > > > > > >> > >> I
> > > >> > > > > > >> > >> > > also appreciate you analysis, showing that for
> > > most
> > > >> use
> > > >> > > > > cases,
> > > >> > > > > > >> the
> > > >> > > > > > >> > >> write
> > > >> > > > > > >> > >> > > batch approach should be just fine.
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > There are a couple of points that would hold
> me
> > > >> back from
> > > >> > > > > > >> approving
> > > >> > > > > > >> > >> this
> > > >> > > > > > >> > >> > > KIP right now:
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > 1. Loss of coverage for custom stores.
> > > >> > > > > > >> > >> > > The fact that you can plug in a (relatively)
> > > simple
> > > >> > > > > > >> implementation
> > > >> > > > > > >> > of
> > > >> > > > > > >> > >> the
> > > >> > > > > > >> > >> > > XStateStore interfaces and automagically get a
> > > >> > > distributed
> > > >> > > > > > >> database
> > > >> > > > > > >> > >> out
> > > >> > > > > > >> > >> > of
> > > >> > > > > > >> > >> > > it is a significant benefit of Kafka Streams.
> > I'd
> > > >> hate to
> > > >> > > > > lose
> > > >> > > > > > >> it,
> > > >> > > > > > >> > so
> > > >> > > > > > >> > >> it
> > > >> > > > > > >> > >> > > would be better to spend some time and come up
> > > with
> > > >> a way
> > > >> > > > to
> > > >> > > > > > >> > preserve
> > > >> > > > > > >> > >> > that
> > > >> > > > > > >> > >> > > property. For example, can we provide a
> default
> > > >> > > > > implementation
> > > >> > > > > > of
> > > >> > > > > > >> > >> > > `commit(..)` that re-implements the existing
> > > >> > > > checkpoint-file
> > > >> > > > > > >> > >> approach? Or
> > > >> > > > > > >> > >> > > perhaps add an `isTransactional()` flag to the
> > > state
> > > >> > > store
> > > >> > > > > > >> interface
> > > >> > > > > > >> > >> so
> > > >> > > > > > >> > >> > > that the runtime can decide whether to
> continue
> > to
> > > >> manage
> > > >> > > > > > >> checkpoint
> > > >> > > > > > >> > >> > files
> > > >> > > > > > >> > >> > > vs delegating transactionality to the stores?
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > 2. Guarding against OOME
> > > >> > > > > > >> > >> > > I appreciate your analysis, but I don't think
> > it's
> > > >> > > > sufficient
> > > >> > > > > > to
> > > >> > > > > > >> say
> > > >> > > > > > >> > >> that
> > > >> > > > > > >> > >> > > we will solve the memory problem later if it
> > > becomes
> > > >> > > > > necessary.
> > > >> > > > > > >> The
> > > >> > > > > > >> > >> > > experience leading to that situation would be
> > > quite
> > > >> bad:
> > > >> > > > > > Imagine,
> > > >> > > > > > >> > you
> > > >> > > > > > >> > >> > > upgrade to AK 3.next, your tests pass, so you
> > > >> deploy to
> > > >> > > > > > >> production.
> > > >> > > > > > >> > >> That
> > > >> > > > > > >> > >> > > night, you get paged because your app is now
> > > >> crashing
> > > >> > > with
> > > >> > > > > > >> OOMEs. As
> > > >> > > > > > >> > >> with
> > > >> > > > > > >> > >> > > all OOMEs, you'll have a really hard time
> > finding
> > > >> the
> > > >> > > root
> > > >> > > > > > cause,
> > > >> > > > > > >> > and
> > > >> > > > > > >> > >> > once
> > > >> > > > > > >> > >> > > you do, you won't have a clear path to resolve
> > the
> > > >> issue.
> > > >> > > > You
> > > >> > > > > > >> could
> > > >> > > > > > >> > >> only
> > > >> > > > > > >> > >> > > tune down the commit interval and cache buffer
> > > size
> > > >> until
> > > >> > > > you
> > > >> > > > > > >> stop
> > > >> > > > > > >> > >> > getting
> > > >> > > > > > >> > >> > > crashes.
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > FYI, I know of multiple cases where people run
> > EOS
> > > >> with
> > > >> > > > much
> > > >> > > > > > >> larger
> > > >> > > > > > >> > >> > commit
> > > >> > > > > > >> > >> > > intervals to get better batching than the
> > default,
> > > >> so I
> > > >> > > > don't
> > > >> > > > > > >> think
> > > >> > > > > > >> > >> this
> > > >> > > > > > >> > >> > > pathological case would be as rare as you
> > suspect.
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > Given that we already have the rudiments of an
> > > idea
> > > >> of
> > > >> > > what
> > > >> > > > > we
> > > >> > > > > > >> could
> > > >> > > > > > >> > >> do
> > > >> > > > > > >> > >> > to
> > > >> > > > > > >> > >> > > prevent this downside, we should take the time
> > to
> > > >> design
> > > >> > > a
> > > >> > > > > > >> solution.
> > > >> > > > > > >> > >> We
> > > >> > > > > > >> > >> > owe
> > > >> > > > > > >> > >> > > it to our users to ensure that awesome new
> > > features
> > > >> don't
> > > >> > > > > come
> > > >> > > > > > >> with
> > > >> > > > > > >> > >> > bitter
> > > >> > > > > > >> > >> > > pills unless we can't avoid it.
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > 3. ALOS mode.
> > > >> > > > > > >> > >> > > On the other hand, I didn't see an indication
> of
> > > how
> > > >> > > stores
> > > >> > > > > > will
> > > >> > > > > > >> be
> > > >> > > > > > >> > >> > > handled under ALOS (aka non-EOS) mode.
> > > >> Theoretically, the
> > > >> > > > > > >> > >> > transactionality
> > > >> > > > > > >> > >> > > of the store and the processing mode are
> > > >> orthogonal. A
> > > >> > > > > > >> transactional
> > > >> > > > > > >> > >> > store
> > > >> > > > > > >> > >> > > would serve ALOS just as well as a
> > > >> non-transactional one
> > > >> > > > (if
> > > >> > > > > > not
> > > >> > > > > > >> > >> better).
> > > >> > > > > > >> > >> > > Under ALOS, though, the default commit
> interval
> > is
> > > >> five
> > > >> > > > > > minutes,
> > > >> > > > > > >> so
> > > >> > > > > > >> > >> the
> > > >> > > > > > >> > >> > > memory issue is far more pressing.
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > As I see it, we have several options to
> resolve
> > > this
> > > >> > > point.
> > > >> > > > > We
> > > >> > > > > > >> could
> > > >> > > > > > >> > >> > > demonstrate that transactional stores work
> just
> > > >> fine for
> > > >> > > > ALOS
> > > >> > > > > > >> and we
> > > >> > > > > > >> > >> can
> > > >> > > > > > >> > >> > > therefore just swap over unconditionally. We
> > could
> > > >> also
> > > >> > > > > disable
> > > >> > > > > > >> the
> > > >> > > > > > >> > >> > > transactional mechanism under ALOS so that
> > stores
> > > >> operate
> > > >> > > > > just
> > > >> > > > > > >> the
> > > >> > > > > > >> > >> same
> > > >> > > > > > >> > >> > as
> > > >> > > > > > >> > >> > > they do today when run in ALOS mode. Finally,
> we
> > > >> could do
> > > >> > > > the
> > > >> > > > > > >> same
> > > >> > > > > > >> > as
> > > >> > > > > > >> > >> in
> > > >> > > > > > >> > >> > > KIP-844 and make transactional stores opt-in
> > (it'd
> > > >> be
> > > >> > > > better
> > > >> > > > > to
> > > >> > > > > > >> > avoid
> > > >> > > > > > >> > >> the
> > > >> > > > > > >> > >> > > extra opt-in mechanism, but it's a good
> > > >> > > > get-out-of-jail-free
> > > >> > > > > > >> card).
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > 4. (minor point) Deprecation of methods
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > You mentioned that the new `commit` 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?
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > Thanks again for the KIP! It's really nice
> that
> > > you
> > > >> and
> > > >> > > > Alex
> > > >> > > > > > will
> > > >> > > > > > >> > get
> > > >> > > > > > >> > >> the
> > > >> > > > > > >> > >> > > chance to collaborate on both directions so
> that
> > > we
> > > >> can
> > > >> > > get
> > > >> > > > > the
> > > >> > > > > > >> best
> > > >> > > > > > >> > >> > > outcome for Streams and its users.
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > -John
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> > > On 2022/11/21 15:02:15 Nick Telford wrote:
> > > >> > > > > > >> > >> > > > 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
> > > >> > > > > > >> > >> > > > <
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> >
> > > >> > > > > > >> > >>
> > > >> > > > > > >> >
> > > >> > > > > > >>
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > >> > > > > > >> > >> > > >,
> > > >> > > > > > >> > >> > > > so that it can be discussed/reviewed
> > separately
> > > >> from
> > > >> > > > > KIP-844.
> > > >> > > > > > >> > >> > > >
> > > >> > > > > > >> > >> > > > Alex: I'm especially interested in what you
> > > think!
> > > >> > > > > > >> > >> > > >
> > > >> > > > > > >> > >> > > > I have a nearly complete implementation of
> the
> > > >> changes
> > > >> > > > > > >> outlined in
> > > >> > > > > > >> > >> this
> > > >> > > > > > >> > >> > > > KIP, please let me know if you'd like me to
> > push
> > > >> them
> > > >> > > for
> > > >> > > > > > >> review
> > > >> > > > > > >> > in
> > > >> > > > > > >> > >> > > advance
> > > >> > > > > > >> > >> > > > of a vote.
> > > >> > > > > > >> > >> > > >
> > > >> > > > > > >> > >> > > > Regards,
> > > >> > > > > > >> > >> > > >
> > > >> > > > > > >> > >> > > > Nick
> > > >> > > > > > >> > >> > > >
> > > >> > > > > > >> > >> > >
> > > >> > > > > > >> > >> >
> > > >> > > > > > >> > >>
> > > >> > > > > > >> > >
> > > >> > > > > > >> >
> > > >> > > > > > >>
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >>
> > > >
> > >
> >
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Colt,

A. I've done my best to de-couple the StateStore stuff from the rest of the
Streams engine. The fact that there will be only one ongoing (write)
transaction at a time is not guaranteed by any API, and is just a
consequence of the way Streams operates. To that end, I tried to ensure the
documentation and guarantees provided by the new APIs are independent of
this incidental behaviour. In practice, you're right, this essentially
refers to "interactive queries", which are technically "read transactions",
even if they don't actually use the transaction API to isolate themselves.

B. Yes, although not ideal. This is for backwards compatibility, because:
    1) Existing custom StateStore implementations will implement flush(),
and not commit(), but the Streams engine now calls commit(), so those calls
need to be forwarded to flush() for these legacy stores.
    2) Existing StateStore *users*, i.e. outside of the Streams engine
itself, may depend on explicitly calling flush(), so for these cases,
flush() needs to be redirected to call commit().
If anyone has a better way to guarantee compatibility without introducing
this potential recursion loop, I'm open to changes!

C. This is described in the "Atomic Checkpointing" section. Offsets are
stored in a separate RocksDB column family, which is guaranteed to be
atomically flushed to disk with all other column families. The issue of
checkpoints being written to disk after commit causing inconsistency if it
crashes in between is the reason why, under EOS, checkpoint files are only
written on clean shutdown. This is one of the major causes of "full
restorations", so moving the offsets into a place where they can be
guaranteed to be atomically written with the data they checkpoint allows us
to write the checkpoint offsets *on every commit*, not just on clean
shutdown.

Regards,
Nick

On Tue, 18 Apr 2023 at 15:39, Colt McNealy <co...@littlehorse.io> wrote:

> Nick,
>
> Thank you for continuing this work. I have a few minor clarifying
> questions.
>
> A) "Records written to any transaction are visible to all other
> transactions immediately." I am confused here—I thought there could only be
> one transaction going on at a time for a given state store given the
> threading model for processing records on a Task. Do you mean Interactive
> Queries by "other transactions"? (If so, then everything makes sense—I
> thought that since IQ were read-only then they didn't count as
> transactions).
>
> B) Is it intentional that the default implementations of the flush() and
> commit() methods in the StateStore class refer to each other in some sort
> of unbounded recursion?
>
> C) How will the getCommittedOffset() method work? At first I thought the
> way to do it would be using a special key in the RocksDB store to store the
> offset, and committing that with the transaction. But upon second thought,
> since restoration from the changelog is an idempotent procedure, I think it
> would be fine to 1) commit the RocksDB transaction and then 2) write the
> offset to disk in a checkpoint file. If there is a crash between 1) and 2),
> I think the only downside is 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 <ni...@gmail.com>
> wrote:
>
> > Hi everyone,
> >
> > I've updated the KIP to reflect the latest version of the design:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >
> > There are several changes in there that reflect feedback from this
> thread,
> > and there's a new section and a bunch of interface changes relating to
> > Atomic Checkpointing, which is the final piece of the puzzle to making
> > everything robust.
> >
> > Let me know what you think!
> >
> > Regards,
> > Nick
> >
> > On Tue, 3 Jan 2023 at 11:33, Nick Telford <ni...@gmail.com>
> wrote:
> >
> > > Hi Lucas,
> > >
> > > Thanks for looking over my KIP.
> > >
> > > A) The bound is per-instance, not per-Task. This was a typo in the KIP
> > > that I've now corrected. It was originally per-Task, but I changed it
> to
> > > per-instance for exactly the reason you highlighted.
> > > B) It's worth noting that transactionality is only enabled under EOS,
> and
> > > in the default mode of operation (ALOS), there should be no change in
> > > behavior at all. I think, under EOS, we can mitigate the impact on
> users
> > by
> > > sufficiently low default values for the memory bound configuration. I
> > > understand your hesitation to include a significant change of
> behaviour,
> > > especially in a minor release, but I suspect that most users will
> prefer
> > > the memory impact (under EOS) to the existing behaviour of frequent
> state
> > > restorations! If this is a problem, the changes can wait until the next
> > > major release. I'll be running a patched version of streams in
> production
> > > with these changes as soon as they're ready, so it won't disrupt me :-D
> > > C) The main purpose of this sentence was just to note that some changes
> > > will need to be made to the way Segments are handled in order to ensure
> > > they also benefit from transactions. At the time I wrote it, I hadn't
> > > figured out the specific changes necessary, so it was deliberately
> vague.
> > > This is the one outstanding problem I'm currently working on, and I'll
> > > update this section with more detail once I have figured out the exact
> > > changes required.
> > > D) newTransaction() provides the necessary isolation guarantees. While
> > > the RocksDB implementation of transactions doesn't technically *need*
> > > read-only users to call newTransaction(), other implementations (e.g. a
> > > hypothetical PostgresStore) may require it. Calling newTransaction()
> when
> > > no transaction is necessary is essentially free, as it will just return
> > > this.
> > >
> > > I didn't do any profiling of the KIP-844 PoC, but I think it should be
> > > fairly obvious where the performance problems stem from: writes under
> > > KIP-844 require 3 extra memory-copies: 1 to encode it with the
> > > tombstone/record flag, 1 to decode it from the tombstone/record flag,
> > and 1
> > > to copy the record from the "temporary" store to the "main" store, when
> > the
> > > transaction commits. The different approach taken by KIP-869 should
> > perform
> > > much better, as it avoids all these copies, and may actually perform
> > > slightly better than trunk, due to batched writes in RocksDB performing
> > > better than non-batched writes.[1]
> > >
> > > Regards,
> > > Nick
> > >
> > > 1:
> > https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > >
> > > On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <lbrutschy@confluent.io
> > .invalid>
> > > wrote:
> > >
> > >> Hi Nick,
> > >>
> > >> I'm just starting to read up on the whole discussion about KIP-892 and
> > >> KIP-844. Thanks a lot for your work on this, I do think
> > >> `WriteBatchWithIndex` may be the way to go here. I do have some
> > >> questions about the latest draft.
> > >>
> > >>  A) If I understand correctly, you propose to put a bound on the
> > >> (native) memory consumed by each task. However, I wonder if this is
> > >> sufficient if we have temporary imbalances in the cluster. For
> > >> example, depending on the timing of rebalances during a cluster
> > >> restart, it could happen that a single streams node is assigned a lot
> > >> more tasks than expected. With your proposed change, this would mean
> > >> that the memory required by this one node could be a multiple of what
> > >> is required during normal operation. I wonder if it wouldn't be safer
> > >> to put a global bound on the memory use, across all tasks.
> > >>  B) Generally, the memory concerns still give me the feeling that this
> > >> should not be enabled by default for all users in a minor release.
> > >>  C) In section "Transaction Management": the sentence "A similar
> > >> analogue will be created to automatically manage `Segment`
> > >> transactions.". Maybe this is just me lacking some background, 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 you to find out why the KIP-844 PoC is 20x slower (e.g.
> > >> by providing a flame graph), that would be quite interesting.
> > >>
> > >> Cheers,
> > >> Lucas
> > >>
> > >> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <ni...@gmail.com>
> > >> wrote:
> > >> >
> > >> > Hi everyone,
> > >> >
> > >> > I've updated the KIP with a more detailed design, which reflects the
> > >> > implementation I've been working on:
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > >> >
> > >> > This new design should address the outstanding points already made
> in
> > >> the
> > >> > thread.
> > >> >
> > >> > Please let me know if there are areas that are unclear or need more
> > >> > clarification.
> > >> >
> > >> > I have a (nearly) working implementation. I'm confident that the
> > >> remaining
> > >> > work (making Segments behave) will not impact the documented design.
> > >> >
> > >> > Regards,
> > >> >
> > >> > Nick
> > >> >
> > >> > On Tue, 6 Dec 2022 at 19:24, Colt McNealy <co...@littlehorse.io>
> > wrote:
> > >> >
> > >> > > Nick,
> > >> > >
> > >> > > Thank you for the reply; that makes sense. I was hoping that,
> since
> > >> reading
> > >> > > uncommitted records from IQ in EOS isn't part of the documented
> API,
> > >> maybe
> > >> > > you *wouldn't* have to wait for the next major release to make
> that
> > >> change;
> > >> > > but given 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 <
> > nick.telford@gmail.com>
> > >> > > wrote:
> > >> > >
> > >> > > > Hi Colt,
> > >> > > >
> > >> > > > 10: Yes, I agree it's not ideal. I originally intended to try to
> > >> keep the
> > >> > > > behaviour unchanged as much as possible, otherwise we'd have to
> > >> wait for
> > >> > > a
> > >> > > > major version release to land these changes.
> > >> > > > 20: Good point, ALOS doesn't need the same level of guarantee,
> and
> > >> the
> > >> > > > typically longer commit intervals would be problematic when
> > reading
> > >> only
> > >> > > > "committed" records.
> > >> > > >
> > >> > > > I've been away for 5 days recovering from minor surgery, but I
> > >> spent a
> > >> > > > considerable amount of that time working through ideas for
> > possible
> > >> > > > solutions in my head. I think your suggestion of keeping ALOS
> > >> as-is, but
> > >> > > > buffering writes for EOS is the right path forwards, although I
> > >> have a
> > >> > > > solution that both expands on this, and provides for some more
> > >> formal
> > >> > > > guarantees.
> > >> > > >
> > >> > > > Essentially, adding support to KeyValueStores for
> "Transactions",
> > >> with
> > >> > > > clearly defined IsolationLevels. Using "Read Committed" when
> under
> > >> EOS,
> > >> > > and
> > >> > > > "Read Uncommitted" under ALOS.
> > >> > > >
> > >> > > > The nice thing about this approach is that it gives us much more
> > >> clearly
> > >> > > > defined isolation behaviour that can be properly documented to
> > >> ensure
> > >> > > users
> > >> > > > know what to expect.
> > >> > > >
> > >> > > > I'm still working out the kinks in the design, and will update
> the
> > >> KIP
> > >> > > when
> > >> > > > I have something. The main struggle is trying to implement this
> > >> without
> > >> > > > making any major changes to the existing interfaces or breaking
> > >> existing
> > >> > > > implementations, because currently everything expects to operate
> > >> directly
> > >> > > > on a StateStore, and not a Transaction of that store. I think
> I'm
> > >> getting
> > >> > > > close, although sadly I won't be able to progress much until
> next
> > >> week
> > >> > > due
> > >> > > > to some work commitments.
> > >> > > >
> > >> > > > Regards,
> > >> > > > Nick
> > >> > > >
> > >> > > > On Thu, 1 Dec 2022 at 00:01, Colt McNealy <co...@littlehorse.io>
> > >> wrote:
> > >> > > >
> > >> > > > > Nick,
> > >> > > > >
> > >> > > > > Thank you for the explanation, and also for the updated KIP. I
> > am
> > >> quite
> > >> > > > > eager for this improvement to be released as it would greatly
> > >> reduce
> > >> > > the
> > >> > > > > operational difficulties of EOS streams apps.
> > >> > > > >
> > >> > > > > Two questions:
> > >> > > > >
> > >> > > > > 10)
> > >> > > > > >When reading records, we will use the
> > >> > > > > WriteBatchWithIndex#getFromBatchAndDB
> > >> > > > >  and WriteBatchWithIndex#newIteratorWithBase utilities in
> order
> > to
> > >> > > ensure
> > >> > > > > that uncommitted writes are available to query.
> > >> > > > > Why do extra work to enable the reading of uncommitted writes
> > >> during
> > >> > > IQ?
> > >> > > > > Code complexity aside, reading uncommitted writes is, in my
> > >> opinion, a
> > >> > > > > minor flaw in EOS IQ; it would be very nice to have the
> > guarantee
> > >> that,
> > >> > > > > with EOS, IQ only reads committed records. In order to avoid
> > dirty
> > >> > > reads,
> > >> > > > > one currently must query a standby replica (but this still
> > doesn't
> > >> > > fully
> > >> > > > > guarantee monotonic reads).
> > >> > > > >
> > >> > > > > 20) Is it also necessary to enable this optimization on ALOS
> > >> stores?
> > >> > > The
> > >> > > > > motivation of KIP-844 was mainly to reduce the need to restore
> > >> state
> > >> > > from
> > >> > > > > scratch on unclean EOS shutdowns; with ALOS it was acceptable
> to
> > >> accept
> > >> > > > > that there may have been uncommitted writes on disk. On a side
> > >> note, if
> > >> > > > you
> > >> > > > > enable this type of store on ALOS processors, the community
> > would
> > >> > > > > 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 <
> > >> nick.telford@gmail.com>
> > >> > > > > wrote:
> > >> > > > >
> > >> > > > > > Hi everyone,
> > >> > > > > >
> > >> > > > > > I've drastically reduced the scope of this KIP to no longer
> > >> include
> > >> > > the
> > >> > > > > > StateStore management of checkpointing. This can be added
> as a
> > >> KIP
> > >> > > > later
> > >> > > > > on
> > >> > > > > > to further optimize the consistency and performance of state
> > >> stores.
> > >> > > > > >
> > >> > > > > > I've also added a section discussing some of the concerns
> > 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 2022 at 16:36, Nick Telford <
> > >> nick.telford@gmail.com>
> > >> > > > > wrote:
> > >> > > > > >
> > >> > > > > > > Hi Colt,
> > >> > > > > > >
> > >> > > > > > > I didn't do any profiling, but the 844 implementation:
> > >> > > > > > >
> > >> > > > > > >    - Writes uncommitted records to a temporary RocksDB
> > >> instance
> > >> > > > > > >       - Since tombstones need to be flagged, all record
> > >> values are
> > >> > > > > > >       prefixed with a value/tombstone marker. This
> > >> necessitates a
> > >> > > > > memory
> > >> > > > > > copy.
> > >> > > > > > >    - On-commit, iterates all records in this temporary
> > >> instance and
> > >> > > > > > >    writes them to the main RocksDB store.
> > >> > > > > > >    - While iterating, the value/tombstone marker needs to
> be
> > >> parsed
> > >> > > > and
> > >> > > > > > >    the real value extracted. This necessitates another
> > memory
> > >> copy.
> > >> > > > > > >
> > >> > > > > > > My guess is that the cost of iterating the temporary
> RocksDB
> > >> store
> > >> > > is
> > >> > > > > the
> > >> > > > > > > major factor, with the 2 extra memory copies per-Record
> > >> > > contributing
> > >> > > > a
> > >> > > > > > > significant amount too.
> > >> > > > > > >
> > >> > > > > > > Regards,
> > >> > > > > > > Nick
> > >> > > > > > >
> > >> > > > > > > On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
> > >> colt@littlehorse.io>
> > >> > > > > wrote:
> > >> > > > > > >
> > >> > > > > > >> Hi all,
> > >> > > > > > >>
> > >> > > > > > >> Out of curiosity, why does the performance of the store
> > >> degrade so
> > >> > > > > > >> significantly with the 844 implementation? I wouldn't be
> > too
> > >> > > > surprised
> > >> > > > > > by
> > >> > > > > > >> a
> > >> > > > > > >> 50-60% drop (caused by each record being written twice),
> > but
> > >> 96%
> > >> > > is
> > >> > > > > > >> extreme.
> > >> > > > > > >>
> > >> > > > > > >> The only thing I can think of which could create such a
> > >> bottleneck
> > >> > > > > 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 Nick Telford <
> > >> > > > nick.telford@gmail.com>
> > >> > > > > > >> wrote:
> > >> > > > > > >>
> > >> > > > > > >> > Hi everyone,
> > >> > > > > > >> >
> > >> > > > > > >> > I've updated the KIP to resolve all the points that
> have
> > >> been
> > >> > > > raised
> > >> > > > > > so
> > >> > > > > > >> > far, with one exception: the ALOS default commit
> interval
> > >> of 5
> > >> > > > > minutes
> > >> > > > > > >> is
> > >> > > > > > >> > likely to cause WriteBatchWithIndex memory to grow too
> > >> large.
> > >> > > > > > >> >
> > >> > > > > > >> > There's a couple of different things I can think of to
> > >> solve
> > >> > > this:
> > >> > > > > > >> >
> > >> > > > > > >> >    - We already have a memory/record limit in the KIP
> to
> > >> prevent
> > >> > > > OOM
> > >> > > > > > >> >    errors. Should we choose a default value for these?
> My
> > >> > > concern
> > >> > > > > here
> > >> > > > > > >> is
> > >> > > > > > >> > that
> > >> > > > > > >> >    anything we choose might seem rather arbitrary. We
> > could
> > >> > > change
> > >> > > > > > >> >    its behaviour such that under ALOS, it only triggers
> > the
> > >> > > commit
> > >> > > > > of
> > >> > > > > > >> the
> > >> > > > > > >> >    StateStore, but under EOS, it triggers a commit of
> the
> > >> Kafka
> > >> > > > > > >> > transaction.
> > >> > > > > > >> >    - We could introduce a separate `
> > checkpoint.interval.ms`
> > >> to
> > >> > > > > allow
> > >> > > > > > >> ALOS
> > >> > > > > > >> >    to commit the StateStores more frequently than the
> > >> general
> > >> > > > > > >> >    commit.interval.ms? My concern here is that the
> > >> semantics of
> > >> > > > > this
> > >> > > > > > >> > config
> > >> > > > > > >> >    would depend 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 <
> > >> > > > nick.telford@gmail.com>
> > >> > > > > > >> wrote:
> > >> > > > > > >> >
> > >> > > > > > >> > > Hi Alex,
> > >> > > > > > >> > >
> > >> > > > > > >> > > Thanks for the feedback.
> > >> > > > > > >> > >
> > >> > > > > > >> > > I've updated the discussion of OOM issues by
> describing
> > >> how
> > >> > > > we'll
> > >> > > > > > >> handle
> > >> > > > > > >> > > it. Here's the new text:
> > >> > > > > > >> > >
> > >> > > > > > >> > > To mitigate this, we will automatically force a Task
> > >> commit if
> > >> > > > the
> > >> > > > > > >> total
> > >> > > > > > >> > >> uncommitted records returned by
> > >> > > > > > >> > >> StateStore#approximateNumUncommittedEntries()
> > exceeds a
> > >> > > > > threshold,
> > >> > > > > > >> > >> configured by
> max.uncommitted.state.entries.per.task;
> > >> or the
> > >> > > > > total
> > >> > > > > > >> > >> memory used for buffering uncommitted records
> returned
> > >> by
> > >> > > > > > >> > >> StateStore#approximateNumUncommittedBytes() exceeds
> > the
> > >> > > > threshold
> > >> > > > > > >> > >> configured by max.uncommitted.state.bytes.per.task.
> > >> This will
> > >> > > > > > roughly
> > >> > > > > > >> > >> bound the memory required per-Task for buffering
> > >> uncommitted
> > >> > > > > > records,
> > >> > > > > > >> > >> irrespective of the commit.interval.ms, and will
> > >> effectively
> > >> > > > > bound
> > >> > > > > > >> the
> > >> > > > > > >> > >> number of records that will need to be restored in
> the
> > >> event
> > >> > > > of a
> > >> > > > > > >> > failure.
> > >> > > > > > >> > >>
> > >> > > > > > >> > >
> > >> > > > > > >> > >
> > >> > > > > > >> > > These limits will be checked in StreamTask#process
> and
> > a
> > >> > > > premature
> > >> > > > > > >> commit
> > >> > > > > > >> > >> will be requested via Task#requestCommit().
> > >> > > > > > >> > >>
> > >> > > > > > >> > >
> > >> > > > > > >> > >
> > >> > > > > > >> > > Note that these new methods provide default
> > >> implementations
> > >> > > that
> > >> > > > > > >> ensure
> > >> > > > > > >> > >> existing custom stores and non-transactional stores
> > >> (e.g.
> > >> > > > > > >> > >> InMemoryKeyValueStore) do not force any early
> commits.
> > >> > > > > > >> > >
> > >> > > > > > >> > >
> > >> > > > > > >> > > I've chosen to have the StateStore expose
> > approximations
> > >> of
> > >> > > its
> > >> > > > > > buffer
> > >> > > > > > >> > > size/count instead of opaquely requesting a commit in
> > >> order to
> > >> > > > > > >> delegate
> > >> > > > > > >> > the
> > >> > > > > > >> > > decision making to the Task itself. This enables
> Tasks
> > >> to look
> > >> > > > at
> > >> > > > > > >> *all*
> > >> > > > > > >> > of
> > >> > > > > > >> > > their StateStores, and determine whether an early
> > commit
> > >> is
> > >> > > > > > necessary.
> > >> > > > > > >> > > Notably, it enables pre-Task thresholds, instead of
> > >> per-Store,
> > >> > > > > which
> > >> > > > > > >> > > prevents Tasks with many StateStores from using much
> > more
> > >> > > memory
> > >> > > > > > than
> > >> > > > > > >> > Tasks
> > >> > > > > > >> > > with one StateStore. This makes sense, since commits
> > are
> > >> done
> > >> > > > > > by-Task,
> > >> > > > > > >> > not
> > >> > > > > > >> > > by-Store.
> > >> > > > > > >> > >
> > >> > > > > > >> > > Prizes* for anyone who can come up with a better name
> > >> for the
> > >> > > > new
> > >> > > > > > >> config
> > >> > > > > > >> > > properties!
> > >> > > > > > >> > >
> > >> > > > > > >> > > Thanks for pointing out the potential performance
> > issues
> > >> of
> > >> > > > WBWI.
> > >> > > > > > From
> > >> > > > > > >> > the
> > >> > > > > > >> > > benchmarks that user posted[1], it looks like WBWI
> > still
> > >> > > > performs
> > >> > > > > > >> > > considerably better than individual puts, which is
> the
> > >> > > existing
> > >> > > > > > >> design,
> > >> > > > > > >> > so
> > >> > > > > > >> > > I'd actually expect a performance boost from WBWI,
> just
> > >> not as
> > >> > > > > great
> > >> > > > > > >> as
> > >> > > > > > >> > > we'd get from a plain WriteBatch. This does suggest
> > that
> > >> a
> > >> > > good
> > >> > > > > > >> > > optimization would be to use a regular WriteBatch for
> > >> > > > restoration
> > >> > > > > > (in
> > >> > > > > > >> > > RocksDBStore#restoreBatch), since we know that those
> > >> records
> > >> > > > will
> > >> > > > > > >> never
> > >> > > > > > >> > be
> > >> > > > > > >> > > queried before they're committed.
> > >> > > > > > >> > >
> > >> > > > > > >> > > 1:
> > >> > > > > > >> >
> > >> > > > > >
> > >> > > >
> > >>
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > >> > > > > > >> > >
> > >> > > > > > >> > > * Just kidding, no prizes, sadly.
> > >> > > > > > >> > >
> > >> > > > > > >> > > On Wed, 23 Nov 2022 at 12:28, Alexander Sorokoumov
> > >> > > > > > >> > > <as...@confluent.io.invalid> wrote:
> > >> > > > > > >> > >
> > >> > > > > > >> > >> 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 to keep the current
> checkpointing
> > >> as a
> > >> > > > > > fallback
> > >> > > > > > >> > >> mechanism. We want to keep existing users' workflows
> > >> intact
> > >> > > if
> > >> > > > we
> > >> > > > > > >> can. A
> > >> > > > > > >> > >> non-intrusive way would be to add a separate
> > StateStore
> > >> > > method,
> > >> > > > > > say,
> > >> > > > > > >> > >> StateStore#managesCheckpointing(), that controls
> > >> whether the
> > >> > > > > state
> > >> > > > > > >> store
> > >> > > > > > >> > >> implementation owns checkpointing.
> > >> > > > > > >> > >>
> > >> > > > > > >> > >> I think that a solution to the transactional writes
> > >> should
> > >> > > > > address
> > >> > > > > > >> the
> > >> > > > > > >> > >> OOMEs. One possible way to address that is to wire
> > >> > > StateStore's
> > >> > > > > > >> commit
> > >> > > > > > >> > >> request by adding, say, StateStore#commitNeeded that
> > is
> > >> > > checked
> > >> > > > > in
> > >> > > > > > >> > >> StreamTask#commitNeeded via the corresponding
> > >> > > > > > ProcessorStateManager.
> > >> > > > > > >> > With
> > >> > > > > > >> > >> that change, RocksDBStore will have to track the
> > current
> > >> > > > > > transaction
> > >> > > > > > >> > size
> > >> > > > > > >> > >> and request a commit when the size goes over a
> > >> (configurable)
> > >> > > > > > >> threshold.
> > >> > > > > > >> > >>
> > >> > > > > > >> > >> AFAIU WriteBatchWithIndex might perform
> significantly
> > >> slower
> > >> > > > than
> > >> > > > > > >> > non-txn
> > >> > > > > > >> > >> puts as the batch size grows [1]. We should have a
> > >> > > > configuration
> > >> > > > > to
> > >> > > > > > >> fall
> > >> > > > > > >> > >> back to the current behavior (and/or disable txn
> > stores
> > >> for
> > >> > > > ALOS)
> > >> > > > > > >> unless
> > >> > > > > > >> > >> the benchmarks show negligible overhead for longer
> > >> commits /
> > >> > > > > > >> > large-enough
> > >> > > > > > >> > >> batch sizes.
> > >> > > > > > >> > >>
> > >> > > > > > >> > >> If you prefer to keep the KIP smaller, I would
> rather
> > >> cut out
> > >> > > > > > >> > >> state-store-managed checkpointing rather than proper
> > >> OOMe
> > >> > > > > handling
> > >> > > > > > >> and
> > >> > > > > > >> > >> being able to switch to non-txn behavior. The
> > >> checkpointing
> > >> > > 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 configuration parts as well as review and
> > test
> > >> the
> > >> > > > > patch.
> > >> > > > > > >> > >>
> > >> > > > > > >> > >> Best,
> > >> > > > > > >> > >> Alex
> > >> > > > > > >> > >>
> > >> > > > > > >> > >>
> > >> > > > > > >> > >> 1. https://github.com/facebook/rocksdb/issues/608
> > >> > > > > > >> > >>
> > >> > > > > > >> > >> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
> > >> > > > > > nick.telford@gmail.com
> > >> > > > > > >> >
> > >> > > > > > >> > >> wrote:
> > >> > > > > > >> > >>
> > >> > > > > > >> > >> > Hi John,
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > Thanks for the review and feedback!
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > 1. Custom Stores: I've been mulling over this
> > problem
> > >> > > myself.
> > >> > > > > As
> > >> > > > > > it
> > >> > > > > > >> > >> stands,
> > >> > > > > > >> > >> > custom stores would essentially lose checkpointing
> > >> with no
> > >> > > > > > >> indication
> > >> > > > > > >> > >> that
> > >> > > > > > >> > >> > they're expected to make changes, besides a line
> in
> > >> the
> > >> > > > release
> > >> > > > > > >> > notes. I
> > >> > > > > > >> > >> > agree that the best solution would be to provide a
> > >> default
> > >> > > > that
> > >> > > > > > >> > >> checkpoints
> > >> > > > > > >> > >> > to a file. The one thing I would change is that
> the
> > >> > > > > checkpointing
> > >> > > > > > >> is
> > >> > > > > > >> > to
> > >> > > > > > >> > >> a
> > >> > > > > > >> > >> > store-local file, instead of a per-Task file. This
> > >> way the
> > >> > > > > > >> StateStore
> > >> > > > > > >> > >> still
> > >> > > > > > >> > >> > technically owns its own checkpointing (via a
> > default
> > >> > > > > > >> implementation),
> > >> > > > > > >> > >> and
> > >> > > > > > >> > >> > the StateManager/Task execution engine doesn't
> need
> > >> to know
> > >> > > > > > >> anything
> > >> > > > > > >> > >> about
> > >> > > > > > >> > >> > checkpointing, which greatly simplifies some of
> the
> > >> logic.
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > 2. OOME errors: The main reasons why I didn't
> > explore
> > >> a
> > >> > > > > solution
> > >> > > > > > to
> > >> > > > > > >> > >> this is
> > >> > > > > > >> > >> > a) to keep this KIP as simple as possible, and b)
> > >> because
> > >> > > I'm
> > >> > > > > not
> > >> > > > > > >> > >> exactly
> > >> > > > > > >> > >> > how to signal that a Task should commit
> prematurely.
> > >> I'm
> > >> > > > > > confident
> > >> > > > > > >> > it's
> > >> > > > > > >> > >> > possible, and I think it's worth adding a section
> on
> > >> > > handling
> > >> > > > > > this.
> > >> > > > > > >> > >> Besides
> > >> > > > > > >> > >> > my proposal to force an early commit once memory
> > usage
> > >> > > > reaches
> > >> > > > > a
> > >> > > > > > >> > >> threshold,
> > >> > > > > > >> > >> > is there any other approach that you might suggest
> > for
> > >> > > > tackling
> > >> > > > > > >> this
> > >> > > > > > >> > >> > problem?
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > 3. ALOS: I can add in an explicit paragraph, but
> my
> > >> > > > assumption
> > >> > > > > is
> > >> > > > > > >> that
> > >> > > > > > >> > >> > since transactional behaviour comes at little/no
> > >> cost, that
> > >> > > > it
> > >> > > > > > >> should
> > >> > > > > > >> > be
> > >> > > > > > >> > >> > available by default on all stores, irrespective
> of
> > >> the
> > >> > > > > > processing
> > >> > > > > > >> > mode.
> > >> > > > > > >> > >> > While ALOS doesn't use transactions, the Task
> itself
> > >> still
> > >> > > > > > >> "commits",
> > >> > > > > > >> > so
> > >> > > > > > >> > >> > the behaviour should be correct under ALOS too.
> I'm
> > >> not
> > >> > > > > convinced
> > >> > > > > > >> that
> > >> > > > > > >> > >> it's
> > >> > > > > > >> > >> > worth having both transactional/non-transactional
> > >> stores
> > >> > > > > > >> available, as
> > >> > > > > > >> > >> it
> > >> > > > > > >> > >> > would considerably increase the complexity of the
> > >> codebase,
> > >> > > > for
> > >> > > > > > >> very
> > >> > > > > > >> > >> little
> > >> > > > > > >> > >> > benefit.
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > 4. Method deprecation: Are you referring to
> > >> > > > > > >> StateStore#getPosition()?
> > >> > > > > > >> > >> As I
> > >> > > > > > >> > >> > understand it, Position contains the position of
> the
> > >> > > *source*
> > >> > > > > > >> topics,
> > >> > > > > > >> > >> > whereas the commit offsets would be the
> *changelog*
> > >> > > offsets.
> > >> > > > So
> > >> > > > > > >> it's
> > >> > > > > > >> > >> still
> > >> > > > > > >> > >> > necessary to retain the Position data, as well as
> > the
> > >> > > > changelog
> > >> > > > > > >> > offsets.
> > >> > > > > > >> > >> > What I meant in the KIP is that Position offsets
> are
> > >> > > > currently
> > >> > > > > > >> stored
> > >> > > > > > >> > >> in a
> > >> > > > > > >> > >> > file, and since we can atomically store metadata
> > >> along with
> > >> > > > the
> > >> > > > > > >> record
> > >> > > > > > >> > >> > batch we commit to RocksDB, we can move our
> Position
> > >> > > offsets
> > >> > > > in
> > >> > > > > > to
> > >> > > > > > >> > this
> > >> > > > > > >> > >> > metadata too, and gain the same transactional
> > >> guarantees
> > >> > > that
> > >> > > > > we
> > >> > > > > > >> will
> > >> > > > > > >> > >> for
> > >> > > > > > >> > >> > changelog offsets, ensuring that the Position
> > offsets
> > >> are
> > >> > > > > > >> consistent
> > >> > > > > > >> > >> with
> > >> > > > > > >> > >> > the records that are read from the database.
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > Regards,
> > >> > > > > > >> > >> > Nick
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > On Tue, 22 Nov 2022 at 16:25, John Roesler <
> > >> > > > > vvcephei@apache.org>
> > >> > > > > > >> > wrote:
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > > Thanks for publishing this alternative, Nick!
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > The benchmark you mentioned in the KIP-844
> > >> discussion
> > >> > > seems
> > >> > > > > > like
> > >> > > > > > >> a
> > >> > > > > > >> > >> > > compelling reason to revisit the built-in
> > >> > > transactionality
> > >> > > > > > >> > mechanism.
> > >> > > > > > >> > >> I
> > >> > > > > > >> > >> > > also appreciate you analysis, showing that for
> > most
> > >> use
> > >> > > > > cases,
> > >> > > > > > >> the
> > >> > > > > > >> > >> write
> > >> > > > > > >> > >> > > batch approach should be just fine.
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > There are a couple of points that would hold me
> > >> back from
> > >> > > > > > >> approving
> > >> > > > > > >> > >> this
> > >> > > > > > >> > >> > > KIP right now:
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > 1. Loss of coverage for custom stores.
> > >> > > > > > >> > >> > > The fact that you can plug in a (relatively)
> > simple
> > >> > > > > > >> implementation
> > >> > > > > > >> > of
> > >> > > > > > >> > >> the
> > >> > > > > > >> > >> > > XStateStore interfaces and automagically get a
> > >> > > distributed
> > >> > > > > > >> database
> > >> > > > > > >> > >> out
> > >> > > > > > >> > >> > of
> > >> > > > > > >> > >> > > it is a significant benefit of Kafka Streams.
> I'd
> > >> hate to
> > >> > > > > lose
> > >> > > > > > >> it,
> > >> > > > > > >> > so
> > >> > > > > > >> > >> it
> > >> > > > > > >> > >> > > would be better to spend some time and come up
> > with
> > >> a way
> > >> > > > to
> > >> > > > > > >> > preserve
> > >> > > > > > >> > >> > that
> > >> > > > > > >> > >> > > property. For example, can we provide a default
> > >> > > > > implementation
> > >> > > > > > of
> > >> > > > > > >> > >> > > `commit(..)` that re-implements the existing
> > >> > > > checkpoint-file
> > >> > > > > > >> > >> approach? Or
> > >> > > > > > >> > >> > > perhaps add an `isTransactional()` flag to the
> > state
> > >> > > store
> > >> > > > > > >> interface
> > >> > > > > > >> > >> so
> > >> > > > > > >> > >> > > that the runtime can decide whether to continue
> to
> > >> manage
> > >> > > > > > >> checkpoint
> > >> > > > > > >> > >> > files
> > >> > > > > > >> > >> > > vs delegating transactionality to the stores?
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > 2. Guarding against OOME
> > >> > > > > > >> > >> > > I appreciate your analysis, but I don't think
> it's
> > >> > > > sufficient
> > >> > > > > > to
> > >> > > > > > >> say
> > >> > > > > > >> > >> that
> > >> > > > > > >> > >> > > we will solve the memory problem later if it
> > becomes
> > >> > > > > necessary.
> > >> > > > > > >> The
> > >> > > > > > >> > >> > > experience leading to that situation would be
> > quite
> > >> bad:
> > >> > > > > > Imagine,
> > >> > > > > > >> > you
> > >> > > > > > >> > >> > > upgrade to AK 3.next, your tests pass, so you
> > >> deploy to
> > >> > > > > > >> production.
> > >> > > > > > >> > >> That
> > >> > > > > > >> > >> > > night, you get paged because your app is now
> > >> crashing
> > >> > > with
> > >> > > > > > >> OOMEs. As
> > >> > > > > > >> > >> with
> > >> > > > > > >> > >> > > all OOMEs, you'll have a really hard time
> finding
> > >> the
> > >> > > root
> > >> > > > > > cause,
> > >> > > > > > >> > and
> > >> > > > > > >> > >> > once
> > >> > > > > > >> > >> > > you do, you won't have a clear path to resolve
> the
> > >> issue.
> > >> > > > You
> > >> > > > > > >> could
> > >> > > > > > >> > >> only
> > >> > > > > > >> > >> > > tune down the commit interval and cache buffer
> > size
> > >> until
> > >> > > > you
> > >> > > > > > >> stop
> > >> > > > > > >> > >> > getting
> > >> > > > > > >> > >> > > crashes.
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > FYI, I know of multiple cases where people run
> EOS
> > >> with
> > >> > > > much
> > >> > > > > > >> larger
> > >> > > > > > >> > >> > commit
> > >> > > > > > >> > >> > > intervals to get better batching than the
> default,
> > >> so I
> > >> > > > don't
> > >> > > > > > >> think
> > >> > > > > > >> > >> this
> > >> > > > > > >> > >> > > pathological case would be as rare as you
> suspect.
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > Given that we already have the rudiments of an
> > idea
> > >> of
> > >> > > what
> > >> > > > > we
> > >> > > > > > >> could
> > >> > > > > > >> > >> do
> > >> > > > > > >> > >> > to
> > >> > > > > > >> > >> > > prevent this downside, we should take the time
> to
> > >> design
> > >> > > a
> > >> > > > > > >> solution.
> > >> > > > > > >> > >> We
> > >> > > > > > >> > >> > owe
> > >> > > > > > >> > >> > > it to our users to ensure that awesome new
> > features
> > >> don't
> > >> > > > > come
> > >> > > > > > >> with
> > >> > > > > > >> > >> > bitter
> > >> > > > > > >> > >> > > pills unless we can't avoid it.
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > 3. ALOS mode.
> > >> > > > > > >> > >> > > On the other hand, I didn't see an indication of
> > how
> > >> > > stores
> > >> > > > > > will
> > >> > > > > > >> be
> > >> > > > > > >> > >> > > handled under ALOS (aka non-EOS) mode.
> > >> Theoretically, the
> > >> > > > > > >> > >> > transactionality
> > >> > > > > > >> > >> > > of the store and the processing mode are
> > >> orthogonal. A
> > >> > > > > > >> transactional
> > >> > > > > > >> > >> > store
> > >> > > > > > >> > >> > > would serve ALOS just as well as a
> > >> non-transactional one
> > >> > > > (if
> > >> > > > > > not
> > >> > > > > > >> > >> better).
> > >> > > > > > >> > >> > > Under ALOS, though, the default commit interval
> is
> > >> five
> > >> > > > > > minutes,
> > >> > > > > > >> so
> > >> > > > > > >> > >> the
> > >> > > > > > >> > >> > > memory issue is far more pressing.
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > As I see it, we have several options to resolve
> > this
> > >> > > point.
> > >> > > > > We
> > >> > > > > > >> could
> > >> > > > > > >> > >> > > demonstrate that transactional stores work just
> > >> fine for
> > >> > > > ALOS
> > >> > > > > > >> and we
> > >> > > > > > >> > >> can
> > >> > > > > > >> > >> > > therefore just swap over unconditionally. We
> could
> > >> also
> > >> > > > > disable
> > >> > > > > > >> the
> > >> > > > > > >> > >> > > transactional mechanism under ALOS so that
> stores
> > >> operate
> > >> > > > > just
> > >> > > > > > >> the
> > >> > > > > > >> > >> same
> > >> > > > > > >> > >> > as
> > >> > > > > > >> > >> > > they do today when run in ALOS mode. Finally, we
> > >> could do
> > >> > > > the
> > >> > > > > > >> same
> > >> > > > > > >> > as
> > >> > > > > > >> > >> in
> > >> > > > > > >> > >> > > KIP-844 and make transactional stores opt-in
> (it'd
> > >> be
> > >> > > > better
> > >> > > > > to
> > >> > > > > > >> > avoid
> > >> > > > > > >> > >> the
> > >> > > > > > >> > >> > > extra opt-in mechanism, but it's a good
> > >> > > > get-out-of-jail-free
> > >> > > > > > >> card).
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > 4. (minor point) Deprecation of methods
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > You mentioned that the new `commit` 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?
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > Thanks again for the KIP! It's really nice that
> > you
> > >> and
> > >> > > > Alex
> > >> > > > > > will
> > >> > > > > > >> > get
> > >> > > > > > >> > >> the
> > >> > > > > > >> > >> > > chance to collaborate on both directions so that
> > we
> > >> can
> > >> > > get
> > >> > > > > the
> > >> > > > > > >> best
> > >> > > > > > >> > >> > > outcome for Streams and its users.
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > -John
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > On 2022/11/21 15:02:15 Nick Telford wrote:
> > >> > > > > > >> > >> > > > 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
> > >> > > > > > >> > >> > > > <
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >>
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > >> > > > > > >> > >> > > >,
> > >> > > > > > >> > >> > > > so that it can be discussed/reviewed
> separately
> > >> from
> > >> > > > > KIP-844.
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > Alex: I'm especially interested in what you
> > think!
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > I have a nearly complete implementation of the
> > >> changes
> > >> > > > > > >> outlined in
> > >> > > > > > >> > >> this
> > >> > > > > > >> > >> > > > KIP, please let me know if you'd like me to
> push
> > >> them
> > >> > > for
> > >> > > > > > >> review
> > >> > > > > > >> > in
> > >> > > > > > >> > >> > > advance
> > >> > > > > > >> > >> > > > of a vote.
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > Regards,
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > Nick
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >>
> > >> > > > > > >> > >
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >>
> > >
> >
>

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

Posted by Colt McNealy <co...@littlehorse.io>.
Nick,

Thank you for continuing this work. I have a few minor clarifying questions.

A) "Records written to any transaction are visible to all other
transactions immediately." I am confused here—I thought there could only be
one transaction going on at a time for a given state store given the
threading model for processing records on a Task. Do you mean Interactive
Queries by "other transactions"? (If so, then everything makes sense—I
thought that since IQ were read-only then they didn't count as
transactions).

B) Is it intentional that the default implementations of the flush() and
commit() methods in the StateStore class refer to each other in some sort
of unbounded recursion?

C) How will the getCommittedOffset() method work? At first I thought the
way to do it would be using a special key in the RocksDB store to store the
offset, and committing that with the transaction. But upon second thought,
since restoration from the changelog is an idempotent procedure, I think it
would be fine to 1) commit the RocksDB transaction and then 2) write the
offset to disk in a checkpoint file. If there is a crash between 1) and 2),
I think the only downside is 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 <ni...@gmail.com> wrote:

> Hi everyone,
>
> I've updated the KIP to reflect the latest version of the design:
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>
> There are several changes in there that reflect feedback from this thread,
> and there's a new section and a bunch of interface changes relating to
> Atomic Checkpointing, which is the final piece of the puzzle to making
> everything robust.
>
> Let me know what you think!
>
> Regards,
> Nick
>
> On Tue, 3 Jan 2023 at 11:33, Nick Telford <ni...@gmail.com> wrote:
>
> > Hi Lucas,
> >
> > Thanks for looking over my KIP.
> >
> > A) The bound is per-instance, not per-Task. This was a typo in the KIP
> > that I've now corrected. It was originally per-Task, but I changed it to
> > per-instance for exactly the reason you highlighted.
> > B) It's worth noting that transactionality is only enabled under EOS, and
> > in the default mode of operation (ALOS), there should be no change in
> > behavior at all. I think, under EOS, we can mitigate the impact on users
> by
> > sufficiently low default values for the memory bound configuration. I
> > understand your hesitation to include a significant change of behaviour,
> > especially in a minor release, but I suspect that most users will prefer
> > the memory impact (under EOS) to the existing behaviour of frequent state
> > restorations! If this is a problem, the changes can wait until the next
> > major release. I'll be running a patched version of streams in production
> > with these changes as soon as they're ready, so it won't disrupt me :-D
> > C) The main purpose of this sentence was just to note that some changes
> > will need to be made to the way Segments are handled in order to ensure
> > they also benefit from transactions. At the time I wrote it, I hadn't
> > figured out the specific changes necessary, so it was deliberately vague.
> > This is the one outstanding problem I'm currently working on, and I'll
> > update this section with more detail once I have figured out the exact
> > changes required.
> > D) newTransaction() provides the necessary isolation guarantees. While
> > the RocksDB implementation of transactions doesn't technically *need*
> > read-only users to call newTransaction(), other implementations (e.g. a
> > hypothetical PostgresStore) may require it. Calling newTransaction() when
> > no transaction is necessary is essentially free, as it will just return
> > this.
> >
> > I didn't do any profiling of the KIP-844 PoC, but I think it should be
> > fairly obvious where the performance problems stem from: writes under
> > KIP-844 require 3 extra memory-copies: 1 to encode it with the
> > tombstone/record flag, 1 to decode it from the tombstone/record flag,
> and 1
> > to copy the record from the "temporary" store to the "main" store, when
> the
> > transaction commits. The different approach taken by KIP-869 should
> perform
> > much better, as it avoids all these copies, and may actually perform
> > slightly better than trunk, due to batched writes in RocksDB performing
> > better than non-batched writes.[1]
> >
> > Regards,
> > Nick
> >
> > 1:
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >
> > On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <lbrutschy@confluent.io
> .invalid>
> > wrote:
> >
> >> Hi Nick,
> >>
> >> I'm just starting to read up on the whole discussion about KIP-892 and
> >> KIP-844. Thanks a lot for your work on this, I do think
> >> `WriteBatchWithIndex` may be the way to go here. I do have some
> >> questions about the latest draft.
> >>
> >>  A) If I understand correctly, you propose to put a bound on the
> >> (native) memory consumed by each task. However, I wonder if this is
> >> sufficient if we have temporary imbalances in the cluster. For
> >> example, depending on the timing of rebalances during a cluster
> >> restart, it could happen that a single streams node is assigned a lot
> >> more tasks than expected. With your proposed change, this would mean
> >> that the memory required by this one node could be a multiple of what
> >> is required during normal operation. I wonder if it wouldn't be safer
> >> to put a global bound on the memory use, across all tasks.
> >>  B) Generally, the memory concerns still give me the feeling that this
> >> should not be enabled by default for all users in a minor release.
> >>  C) In section "Transaction Management": the sentence "A similar
> >> analogue will be created to automatically manage `Segment`
> >> transactions.". Maybe this is just me lacking some background, 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 you to find out why the KIP-844 PoC is 20x slower (e.g.
> >> by providing a flame graph), that would be quite interesting.
> >>
> >> Cheers,
> >> Lucas
> >>
> >> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <ni...@gmail.com>
> >> wrote:
> >> >
> >> > Hi everyone,
> >> >
> >> > I've updated the KIP with a more detailed design, which reflects the
> >> > implementation I've been working on:
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >> >
> >> > This new design should address the outstanding points already made in
> >> the
> >> > thread.
> >> >
> >> > Please let me know if there are areas that are unclear or need more
> >> > clarification.
> >> >
> >> > I have a (nearly) working implementation. I'm confident that the
> >> remaining
> >> > work (making Segments behave) will not impact the documented design.
> >> >
> >> > Regards,
> >> >
> >> > Nick
> >> >
> >> > On Tue, 6 Dec 2022 at 19:24, Colt McNealy <co...@littlehorse.io>
> wrote:
> >> >
> >> > > Nick,
> >> > >
> >> > > Thank you for the reply; that makes sense. I was hoping that, since
> >> reading
> >> > > uncommitted records from IQ in EOS isn't part of the documented API,
> >> maybe
> >> > > you *wouldn't* have to wait for the next major release to make that
> >> change;
> >> > > but given 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 <
> nick.telford@gmail.com>
> >> > > wrote:
> >> > >
> >> > > > Hi Colt,
> >> > > >
> >> > > > 10: Yes, I agree it's not ideal. I originally intended to try to
> >> keep the
> >> > > > behaviour unchanged as much as possible, otherwise we'd have to
> >> wait for
> >> > > a
> >> > > > major version release to land these changes.
> >> > > > 20: Good point, ALOS doesn't need the same level of guarantee, and
> >> the
> >> > > > typically longer commit intervals would be problematic when
> reading
> >> only
> >> > > > "committed" records.
> >> > > >
> >> > > > I've been away for 5 days recovering from minor surgery, but I
> >> spent a
> >> > > > considerable amount of that time working through ideas for
> possible
> >> > > > solutions in my head. I think your suggestion of keeping ALOS
> >> as-is, but
> >> > > > buffering writes for EOS is the right path forwards, although I
> >> have a
> >> > > > solution that both expands on this, and provides for some more
> >> formal
> >> > > > guarantees.
> >> > > >
> >> > > > Essentially, adding support to KeyValueStores for "Transactions",
> >> with
> >> > > > clearly defined IsolationLevels. Using "Read Committed" when under
> >> EOS,
> >> > > and
> >> > > > "Read Uncommitted" under ALOS.
> >> > > >
> >> > > > The nice thing about this approach is that it gives us much more
> >> clearly
> >> > > > defined isolation behaviour that can be properly documented to
> >> ensure
> >> > > users
> >> > > > know what to expect.
> >> > > >
> >> > > > I'm still working out the kinks in the design, and will update the
> >> KIP
> >> > > when
> >> > > > I have something. The main struggle is trying to implement this
> >> without
> >> > > > making any major changes to the existing interfaces or breaking
> >> existing
> >> > > > implementations, because currently everything expects to operate
> >> directly
> >> > > > on a StateStore, and not a Transaction of that store. I think I'm
> >> getting
> >> > > > close, although sadly I won't be able to progress much until next
> >> week
> >> > > due
> >> > > > to some work commitments.
> >> > > >
> >> > > > Regards,
> >> > > > Nick
> >> > > >
> >> > > > On Thu, 1 Dec 2022 at 00:01, Colt McNealy <co...@littlehorse.io>
> >> wrote:
> >> > > >
> >> > > > > Nick,
> >> > > > >
> >> > > > > Thank you for the explanation, and also for the updated KIP. I
> am
> >> quite
> >> > > > > eager for this improvement to be released as it would greatly
> >> reduce
> >> > > the
> >> > > > > operational difficulties of EOS streams apps.
> >> > > > >
> >> > > > > Two questions:
> >> > > > >
> >> > > > > 10)
> >> > > > > >When reading records, we will use the
> >> > > > > WriteBatchWithIndex#getFromBatchAndDB
> >> > > > >  and WriteBatchWithIndex#newIteratorWithBase utilities in order
> to
> >> > > ensure
> >> > > > > that uncommitted writes are available to query.
> >> > > > > Why do extra work to enable the reading of uncommitted writes
> >> during
> >> > > IQ?
> >> > > > > Code complexity aside, reading uncommitted writes is, in my
> >> opinion, a
> >> > > > > minor flaw in EOS IQ; it would be very nice to have the
> guarantee
> >> that,
> >> > > > > with EOS, IQ only reads committed records. In order to avoid
> dirty
> >> > > reads,
> >> > > > > one currently must query a standby replica (but this still
> doesn't
> >> > > fully
> >> > > > > guarantee monotonic reads).
> >> > > > >
> >> > > > > 20) Is it also necessary to enable this optimization on ALOS
> >> stores?
> >> > > The
> >> > > > > motivation of KIP-844 was mainly to reduce the need to restore
> >> state
> >> > > from
> >> > > > > scratch on unclean EOS shutdowns; with ALOS it was acceptable to
> >> accept
> >> > > > > that there may have been uncommitted writes on disk. On a side
> >> note, if
> >> > > > you
> >> > > > > enable this type of store on ALOS processors, the community
> would
> >> > > > > 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 <
> >> nick.telford@gmail.com>
> >> > > > > wrote:
> >> > > > >
> >> > > > > > Hi everyone,
> >> > > > > >
> >> > > > > > I've drastically reduced the scope of this KIP to no longer
> >> include
> >> > > the
> >> > > > > > StateStore management of checkpointing. This can be added as a
> >> KIP
> >> > > > later
> >> > > > > on
> >> > > > > > to further optimize the consistency and performance of state
> >> stores.
> >> > > > > >
> >> > > > > > I've also added a section discussing some of the concerns
> 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 2022 at 16:36, Nick Telford <
> >> nick.telford@gmail.com>
> >> > > > > wrote:
> >> > > > > >
> >> > > > > > > Hi Colt,
> >> > > > > > >
> >> > > > > > > I didn't do any profiling, but the 844 implementation:
> >> > > > > > >
> >> > > > > > >    - Writes uncommitted records to a temporary RocksDB
> >> instance
> >> > > > > > >       - Since tombstones need to be flagged, all record
> >> values are
> >> > > > > > >       prefixed with a value/tombstone marker. This
> >> necessitates a
> >> > > > > memory
> >> > > > > > copy.
> >> > > > > > >    - On-commit, iterates all records in this temporary
> >> instance and
> >> > > > > > >    writes them to the main RocksDB store.
> >> > > > > > >    - While iterating, the value/tombstone marker needs to be
> >> parsed
> >> > > > and
> >> > > > > > >    the real value extracted. This necessitates another
> memory
> >> copy.
> >> > > > > > >
> >> > > > > > > My guess is that the cost of iterating the temporary RocksDB
> >> store
> >> > > is
> >> > > > > the
> >> > > > > > > major factor, with the 2 extra memory copies per-Record
> >> > > contributing
> >> > > > a
> >> > > > > > > significant amount too.
> >> > > > > > >
> >> > > > > > > Regards,
> >> > > > > > > Nick
> >> > > > > > >
> >> > > > > > > On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
> >> colt@littlehorse.io>
> >> > > > > wrote:
> >> > > > > > >
> >> > > > > > >> Hi all,
> >> > > > > > >>
> >> > > > > > >> Out of curiosity, why does the performance of the store
> >> degrade so
> >> > > > > > >> significantly with the 844 implementation? I wouldn't be
> too
> >> > > > surprised
> >> > > > > > by
> >> > > > > > >> a
> >> > > > > > >> 50-60% drop (caused by each record being written twice),
> but
> >> 96%
> >> > > is
> >> > > > > > >> extreme.
> >> > > > > > >>
> >> > > > > > >> The only thing I can think of which could create such a
> >> bottleneck
> >> > > > > 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 Nick Telford <
> >> > > > nick.telford@gmail.com>
> >> > > > > > >> wrote:
> >> > > > > > >>
> >> > > > > > >> > Hi everyone,
> >> > > > > > >> >
> >> > > > > > >> > I've updated the KIP to resolve all the points that have
> >> been
> >> > > > raised
> >> > > > > > so
> >> > > > > > >> > far, with one exception: the ALOS default commit interval
> >> of 5
> >> > > > > minutes
> >> > > > > > >> is
> >> > > > > > >> > likely to cause WriteBatchWithIndex memory to grow too
> >> large.
> >> > > > > > >> >
> >> > > > > > >> > There's a couple of different things I can think of to
> >> solve
> >> > > this:
> >> > > > > > >> >
> >> > > > > > >> >    - We already have a memory/record limit in the KIP to
> >> prevent
> >> > > > OOM
> >> > > > > > >> >    errors. Should we choose a default value for these? My
> >> > > concern
> >> > > > > here
> >> > > > > > >> is
> >> > > > > > >> > that
> >> > > > > > >> >    anything we choose might seem rather arbitrary. We
> could
> >> > > change
> >> > > > > > >> >    its behaviour such that under ALOS, it only triggers
> the
> >> > > commit
> >> > > > > of
> >> > > > > > >> the
> >> > > > > > >> >    StateStore, but under EOS, it triggers a commit of the
> >> Kafka
> >> > > > > > >> > transaction.
> >> > > > > > >> >    - We could introduce a separate `
> checkpoint.interval.ms`
> >> to
> >> > > > > allow
> >> > > > > > >> ALOS
> >> > > > > > >> >    to commit the StateStores more frequently than the
> >> general
> >> > > > > > >> >    commit.interval.ms? My concern here is that the
> >> semantics of
> >> > > > > this
> >> > > > > > >> > config
> >> > > > > > >> >    would depend 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 <
> >> > > > nick.telford@gmail.com>
> >> > > > > > >> wrote:
> >> > > > > > >> >
> >> > > > > > >> > > Hi Alex,
> >> > > > > > >> > >
> >> > > > > > >> > > Thanks for the feedback.
> >> > > > > > >> > >
> >> > > > > > >> > > I've updated the discussion of OOM issues by describing
> >> how
> >> > > > we'll
> >> > > > > > >> handle
> >> > > > > > >> > > it. Here's the new text:
> >> > > > > > >> > >
> >> > > > > > >> > > To mitigate this, we will automatically force a Task
> >> commit if
> >> > > > the
> >> > > > > > >> total
> >> > > > > > >> > >> uncommitted records returned by
> >> > > > > > >> > >> StateStore#approximateNumUncommittedEntries()
> exceeds a
> >> > > > > threshold,
> >> > > > > > >> > >> configured by max.uncommitted.state.entries.per.task;
> >> or the
> >> > > > > total
> >> > > > > > >> > >> memory used for buffering uncommitted records returned
> >> by
> >> > > > > > >> > >> StateStore#approximateNumUncommittedBytes() exceeds
> the
> >> > > > threshold
> >> > > > > > >> > >> configured by max.uncommitted.state.bytes.per.task.
> >> This will
> >> > > > > > roughly
> >> > > > > > >> > >> bound the memory required per-Task for buffering
> >> uncommitted
> >> > > > > > records,
> >> > > > > > >> > >> irrespective of the commit.interval.ms, and will
> >> effectively
> >> > > > > bound
> >> > > > > > >> the
> >> > > > > > >> > >> number of records that will need to be restored in the
> >> event
> >> > > > of a
> >> > > > > > >> > failure.
> >> > > > > > >> > >>
> >> > > > > > >> > >
> >> > > > > > >> > >
> >> > > > > > >> > > These limits will be checked in StreamTask#process and
> a
> >> > > > premature
> >> > > > > > >> commit
> >> > > > > > >> > >> will be requested via Task#requestCommit().
> >> > > > > > >> > >>
> >> > > > > > >> > >
> >> > > > > > >> > >
> >> > > > > > >> > > Note that these new methods provide default
> >> implementations
> >> > > that
> >> > > > > > >> ensure
> >> > > > > > >> > >> existing custom stores and non-transactional stores
> >> (e.g.
> >> > > > > > >> > >> InMemoryKeyValueStore) do not force any early commits.
> >> > > > > > >> > >
> >> > > > > > >> > >
> >> > > > > > >> > > I've chosen to have the StateStore expose
> approximations
> >> of
> >> > > its
> >> > > > > > buffer
> >> > > > > > >> > > size/count instead of opaquely requesting a commit in
> >> order to
> >> > > > > > >> delegate
> >> > > > > > >> > the
> >> > > > > > >> > > decision making to the Task itself. This enables Tasks
> >> to look
> >> > > > at
> >> > > > > > >> *all*
> >> > > > > > >> > of
> >> > > > > > >> > > their StateStores, and determine whether an early
> commit
> >> is
> >> > > > > > necessary.
> >> > > > > > >> > > Notably, it enables pre-Task thresholds, instead of
> >> per-Store,
> >> > > > > which
> >> > > > > > >> > > prevents Tasks with many StateStores from using much
> more
> >> > > memory
> >> > > > > > than
> >> > > > > > >> > Tasks
> >> > > > > > >> > > with one StateStore. This makes sense, since commits
> are
> >> done
> >> > > > > > by-Task,
> >> > > > > > >> > not
> >> > > > > > >> > > by-Store.
> >> > > > > > >> > >
> >> > > > > > >> > > Prizes* for anyone who can come up with a better name
> >> for the
> >> > > > new
> >> > > > > > >> config
> >> > > > > > >> > > properties!
> >> > > > > > >> > >
> >> > > > > > >> > > Thanks for pointing out the potential performance
> issues
> >> of
> >> > > > WBWI.
> >> > > > > > From
> >> > > > > > >> > the
> >> > > > > > >> > > benchmarks that user posted[1], it looks like WBWI
> still
> >> > > > performs
> >> > > > > > >> > > considerably better than individual puts, which is the
> >> > > existing
> >> > > > > > >> design,
> >> > > > > > >> > so
> >> > > > > > >> > > I'd actually expect a performance boost from WBWI, just
> >> not as
> >> > > > > great
> >> > > > > > >> as
> >> > > > > > >> > > we'd get from a plain WriteBatch. This does suggest
> that
> >> a
> >> > > good
> >> > > > > > >> > > optimization would be to use a regular WriteBatch for
> >> > > > restoration
> >> > > > > > (in
> >> > > > > > >> > > RocksDBStore#restoreBatch), since we know that those
> >> records
> >> > > > will
> >> > > > > > >> never
> >> > > > > > >> > be
> >> > > > > > >> > > queried before they're committed.
> >> > > > > > >> > >
> >> > > > > > >> > > 1:
> >> > > > > > >> >
> >> > > > > >
> >> > > >
> >> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> >> > > > > > >> > >
> >> > > > > > >> > > * Just kidding, no prizes, sadly.
> >> > > > > > >> > >
> >> > > > > > >> > > On Wed, 23 Nov 2022 at 12:28, Alexander Sorokoumov
> >> > > > > > >> > > <as...@confluent.io.invalid> wrote:
> >> > > > > > >> > >
> >> > > > > > >> > >> 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 to keep the current checkpointing
> >> as a
> >> > > > > > fallback
> >> > > > > > >> > >> mechanism. We want to keep existing users' workflows
> >> intact
> >> > > if
> >> > > > we
> >> > > > > > >> can. A
> >> > > > > > >> > >> non-intrusive way would be to add a separate
> StateStore
> >> > > method,
> >> > > > > > say,
> >> > > > > > >> > >> StateStore#managesCheckpointing(), that controls
> >> whether the
> >> > > > > state
> >> > > > > > >> store
> >> > > > > > >> > >> implementation owns checkpointing.
> >> > > > > > >> > >>
> >> > > > > > >> > >> I think that a solution to the transactional writes
> >> should
> >> > > > > address
> >> > > > > > >> the
> >> > > > > > >> > >> OOMEs. One possible way to address that is to wire
> >> > > StateStore's
> >> > > > > > >> commit
> >> > > > > > >> > >> request by adding, say, StateStore#commitNeeded that
> is
> >> > > checked
> >> > > > > in
> >> > > > > > >> > >> StreamTask#commitNeeded via the corresponding
> >> > > > > > ProcessorStateManager.
> >> > > > > > >> > With
> >> > > > > > >> > >> that change, RocksDBStore will have to track the
> current
> >> > > > > > transaction
> >> > > > > > >> > size
> >> > > > > > >> > >> and request a commit when the size goes over a
> >> (configurable)
> >> > > > > > >> threshold.
> >> > > > > > >> > >>
> >> > > > > > >> > >> AFAIU WriteBatchWithIndex might perform significantly
> >> slower
> >> > > > than
> >> > > > > > >> > non-txn
> >> > > > > > >> > >> puts as the batch size grows [1]. We should have a
> >> > > > configuration
> >> > > > > to
> >> > > > > > >> fall
> >> > > > > > >> > >> back to the current behavior (and/or disable txn
> stores
> >> for
> >> > > > ALOS)
> >> > > > > > >> unless
> >> > > > > > >> > >> the benchmarks show negligible overhead for longer
> >> commits /
> >> > > > > > >> > large-enough
> >> > > > > > >> > >> batch sizes.
> >> > > > > > >> > >>
> >> > > > > > >> > >> If you prefer to keep the KIP smaller, I would rather
> >> cut out
> >> > > > > > >> > >> state-store-managed checkpointing rather than proper
> >> OOMe
> >> > > > > handling
> >> > > > > > >> and
> >> > > > > > >> > >> being able to switch to non-txn behavior. The
> >> checkpointing
> >> > > 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 configuration parts as well as review and
> test
> >> the
> >> > > > > patch.
> >> > > > > > >> > >>
> >> > > > > > >> > >> Best,
> >> > > > > > >> > >> Alex
> >> > > > > > >> > >>
> >> > > > > > >> > >>
> >> > > > > > >> > >> 1. https://github.com/facebook/rocksdb/issues/608
> >> > > > > > >> > >>
> >> > > > > > >> > >> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
> >> > > > > > nick.telford@gmail.com
> >> > > > > > >> >
> >> > > > > > >> > >> wrote:
> >> > > > > > >> > >>
> >> > > > > > >> > >> > Hi John,
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > Thanks for the review and feedback!
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > 1. Custom Stores: I've been mulling over this
> problem
> >> > > myself.
> >> > > > > As
> >> > > > > > it
> >> > > > > > >> > >> stands,
> >> > > > > > >> > >> > custom stores would essentially lose checkpointing
> >> with no
> >> > > > > > >> indication
> >> > > > > > >> > >> that
> >> > > > > > >> > >> > they're expected to make changes, besides a line in
> >> the
> >> > > > release
> >> > > > > > >> > notes. I
> >> > > > > > >> > >> > agree that the best solution would be to provide a
> >> default
> >> > > > that
> >> > > > > > >> > >> checkpoints
> >> > > > > > >> > >> > to a file. The one thing I would change is that the
> >> > > > > checkpointing
> >> > > > > > >> is
> >> > > > > > >> > to
> >> > > > > > >> > >> a
> >> > > > > > >> > >> > store-local file, instead of a per-Task file. This
> >> way the
> >> > > > > > >> StateStore
> >> > > > > > >> > >> still
> >> > > > > > >> > >> > technically owns its own checkpointing (via a
> default
> >> > > > > > >> implementation),
> >> > > > > > >> > >> and
> >> > > > > > >> > >> > the StateManager/Task execution engine doesn't need
> >> to know
> >> > > > > > >> anything
> >> > > > > > >> > >> about
> >> > > > > > >> > >> > checkpointing, which greatly simplifies some of the
> >> logic.
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > 2. OOME errors: The main reasons why I didn't
> explore
> >> a
> >> > > > > solution
> >> > > > > > to
> >> > > > > > >> > >> this is
> >> > > > > > >> > >> > a) to keep this KIP as simple as possible, and b)
> >> because
> >> > > I'm
> >> > > > > not
> >> > > > > > >> > >> exactly
> >> > > > > > >> > >> > how to signal that a Task should commit prematurely.
> >> I'm
> >> > > > > > confident
> >> > > > > > >> > it's
> >> > > > > > >> > >> > possible, and I think it's worth adding a section on
> >> > > handling
> >> > > > > > this.
> >> > > > > > >> > >> Besides
> >> > > > > > >> > >> > my proposal to force an early commit once memory
> usage
> >> > > > reaches
> >> > > > > a
> >> > > > > > >> > >> threshold,
> >> > > > > > >> > >> > is there any other approach that you might suggest
> for
> >> > > > tackling
> >> > > > > > >> this
> >> > > > > > >> > >> > problem?
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > 3. ALOS: I can add in an explicit paragraph, but my
> >> > > > assumption
> >> > > > > is
> >> > > > > > >> that
> >> > > > > > >> > >> > since transactional behaviour comes at little/no
> >> cost, that
> >> > > > it
> >> > > > > > >> should
> >> > > > > > >> > be
> >> > > > > > >> > >> > available by default on all stores, irrespective of
> >> the
> >> > > > > > processing
> >> > > > > > >> > mode.
> >> > > > > > >> > >> > While ALOS doesn't use transactions, the Task itself
> >> still
> >> > > > > > >> "commits",
> >> > > > > > >> > so
> >> > > > > > >> > >> > the behaviour should be correct under ALOS too. I'm
> >> not
> >> > > > > convinced
> >> > > > > > >> that
> >> > > > > > >> > >> it's
> >> > > > > > >> > >> > worth having both transactional/non-transactional
> >> stores
> >> > > > > > >> available, as
> >> > > > > > >> > >> it
> >> > > > > > >> > >> > would considerably increase the complexity of the
> >> codebase,
> >> > > > for
> >> > > > > > >> very
> >> > > > > > >> > >> little
> >> > > > > > >> > >> > benefit.
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > 4. Method deprecation: Are you referring to
> >> > > > > > >> StateStore#getPosition()?
> >> > > > > > >> > >> As I
> >> > > > > > >> > >> > understand it, Position contains the position of the
> >> > > *source*
> >> > > > > > >> topics,
> >> > > > > > >> > >> > whereas the commit offsets would be the *changelog*
> >> > > offsets.
> >> > > > So
> >> > > > > > >> it's
> >> > > > > > >> > >> still
> >> > > > > > >> > >> > necessary to retain the Position data, as well as
> the
> >> > > > changelog
> >> > > > > > >> > offsets.
> >> > > > > > >> > >> > What I meant in the KIP is that Position offsets are
> >> > > > currently
> >> > > > > > >> stored
> >> > > > > > >> > >> in a
> >> > > > > > >> > >> > file, and since we can atomically store metadata
> >> along with
> >> > > > the
> >> > > > > > >> record
> >> > > > > > >> > >> > batch we commit to RocksDB, we can move our Position
> >> > > offsets
> >> > > > in
> >> > > > > > to
> >> > > > > > >> > this
> >> > > > > > >> > >> > metadata too, and gain the same transactional
> >> guarantees
> >> > > that
> >> > > > > we
> >> > > > > > >> will
> >> > > > > > >> > >> for
> >> > > > > > >> > >> > changelog offsets, ensuring that the Position
> offsets
> >> are
> >> > > > > > >> consistent
> >> > > > > > >> > >> with
> >> > > > > > >> > >> > the records that are read from the database.
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > Regards,
> >> > > > > > >> > >> > Nick
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > On Tue, 22 Nov 2022 at 16:25, John Roesler <
> >> > > > > vvcephei@apache.org>
> >> > > > > > >> > wrote:
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > > Thanks for publishing this alternative, Nick!
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > The benchmark you mentioned in the KIP-844
> >> discussion
> >> > > seems
> >> > > > > > like
> >> > > > > > >> a
> >> > > > > > >> > >> > > compelling reason to revisit the built-in
> >> > > transactionality
> >> > > > > > >> > mechanism.
> >> > > > > > >> > >> I
> >> > > > > > >> > >> > > also appreciate you analysis, showing that for
> most
> >> use
> >> > > > > cases,
> >> > > > > > >> the
> >> > > > > > >> > >> write
> >> > > > > > >> > >> > > batch approach should be just fine.
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > There are a couple of points that would hold me
> >> back from
> >> > > > > > >> approving
> >> > > > > > >> > >> this
> >> > > > > > >> > >> > > KIP right now:
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > 1. Loss of coverage for custom stores.
> >> > > > > > >> > >> > > The fact that you can plug in a (relatively)
> simple
> >> > > > > > >> implementation
> >> > > > > > >> > of
> >> > > > > > >> > >> the
> >> > > > > > >> > >> > > XStateStore interfaces and automagically get a
> >> > > distributed
> >> > > > > > >> database
> >> > > > > > >> > >> out
> >> > > > > > >> > >> > of
> >> > > > > > >> > >> > > it is a significant benefit of Kafka Streams. I'd
> >> hate to
> >> > > > > lose
> >> > > > > > >> it,
> >> > > > > > >> > so
> >> > > > > > >> > >> it
> >> > > > > > >> > >> > > would be better to spend some time and come up
> with
> >> a way
> >> > > > to
> >> > > > > > >> > preserve
> >> > > > > > >> > >> > that
> >> > > > > > >> > >> > > property. For example, can we provide a default
> >> > > > > implementation
> >> > > > > > of
> >> > > > > > >> > >> > > `commit(..)` that re-implements the existing
> >> > > > checkpoint-file
> >> > > > > > >> > >> approach? Or
> >> > > > > > >> > >> > > perhaps add an `isTransactional()` flag to the
> state
> >> > > store
> >> > > > > > >> interface
> >> > > > > > >> > >> so
> >> > > > > > >> > >> > > that the runtime can decide whether to continue to
> >> manage
> >> > > > > > >> checkpoint
> >> > > > > > >> > >> > files
> >> > > > > > >> > >> > > vs delegating transactionality to the stores?
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > 2. Guarding against OOME
> >> > > > > > >> > >> > > I appreciate your analysis, but I don't think it's
> >> > > > sufficient
> >> > > > > > to
> >> > > > > > >> say
> >> > > > > > >> > >> that
> >> > > > > > >> > >> > > we will solve the memory problem later if it
> becomes
> >> > > > > necessary.
> >> > > > > > >> The
> >> > > > > > >> > >> > > experience leading to that situation would be
> quite
> >> bad:
> >> > > > > > Imagine,
> >> > > > > > >> > you
> >> > > > > > >> > >> > > upgrade to AK 3.next, your tests pass, so you
> >> deploy to
> >> > > > > > >> production.
> >> > > > > > >> > >> That
> >> > > > > > >> > >> > > night, you get paged because your app is now
> >> crashing
> >> > > with
> >> > > > > > >> OOMEs. As
> >> > > > > > >> > >> with
> >> > > > > > >> > >> > > all OOMEs, you'll have a really hard time finding
> >> the
> >> > > root
> >> > > > > > cause,
> >> > > > > > >> > and
> >> > > > > > >> > >> > once
> >> > > > > > >> > >> > > you do, you won't have a clear path to resolve the
> >> issue.
> >> > > > You
> >> > > > > > >> could
> >> > > > > > >> > >> only
> >> > > > > > >> > >> > > tune down the commit interval and cache buffer
> size
> >> until
> >> > > > you
> >> > > > > > >> stop
> >> > > > > > >> > >> > getting
> >> > > > > > >> > >> > > crashes.
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > FYI, I know of multiple cases where people run EOS
> >> with
> >> > > > much
> >> > > > > > >> larger
> >> > > > > > >> > >> > commit
> >> > > > > > >> > >> > > intervals to get better batching than the default,
> >> so I
> >> > > > don't
> >> > > > > > >> think
> >> > > > > > >> > >> this
> >> > > > > > >> > >> > > pathological case would be as rare as you suspect.
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > Given that we already have the rudiments of an
> idea
> >> of
> >> > > what
> >> > > > > we
> >> > > > > > >> could
> >> > > > > > >> > >> do
> >> > > > > > >> > >> > to
> >> > > > > > >> > >> > > prevent this downside, we should take the time to
> >> design
> >> > > a
> >> > > > > > >> solution.
> >> > > > > > >> > >> We
> >> > > > > > >> > >> > owe
> >> > > > > > >> > >> > > it to our users to ensure that awesome new
> features
> >> don't
> >> > > > > come
> >> > > > > > >> with
> >> > > > > > >> > >> > bitter
> >> > > > > > >> > >> > > pills unless we can't avoid it.
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > 3. ALOS mode.
> >> > > > > > >> > >> > > On the other hand, I didn't see an indication of
> how
> >> > > stores
> >> > > > > > will
> >> > > > > > >> be
> >> > > > > > >> > >> > > handled under ALOS (aka non-EOS) mode.
> >> Theoretically, the
> >> > > > > > >> > >> > transactionality
> >> > > > > > >> > >> > > of the store and the processing mode are
> >> orthogonal. A
> >> > > > > > >> transactional
> >> > > > > > >> > >> > store
> >> > > > > > >> > >> > > would serve ALOS just as well as a
> >> non-transactional one
> >> > > > (if
> >> > > > > > not
> >> > > > > > >> > >> better).
> >> > > > > > >> > >> > > Under ALOS, though, the default commit interval is
> >> five
> >> > > > > > minutes,
> >> > > > > > >> so
> >> > > > > > >> > >> the
> >> > > > > > >> > >> > > memory issue is far more pressing.
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > As I see it, we have several options to resolve
> this
> >> > > point.
> >> > > > > We
> >> > > > > > >> could
> >> > > > > > >> > >> > > demonstrate that transactional stores work just
> >> fine for
> >> > > > ALOS
> >> > > > > > >> and we
> >> > > > > > >> > >> can
> >> > > > > > >> > >> > > therefore just swap over unconditionally. We could
> >> also
> >> > > > > disable
> >> > > > > > >> the
> >> > > > > > >> > >> > > transactional mechanism under ALOS so that stores
> >> operate
> >> > > > > just
> >> > > > > > >> the
> >> > > > > > >> > >> same
> >> > > > > > >> > >> > as
> >> > > > > > >> > >> > > they do today when run in ALOS mode. Finally, we
> >> could do
> >> > > > the
> >> > > > > > >> same
> >> > > > > > >> > as
> >> > > > > > >> > >> in
> >> > > > > > >> > >> > > KIP-844 and make transactional stores opt-in (it'd
> >> be
> >> > > > better
> >> > > > > to
> >> > > > > > >> > avoid
> >> > > > > > >> > >> the
> >> > > > > > >> > >> > > extra opt-in mechanism, but it's a good
> >> > > > get-out-of-jail-free
> >> > > > > > >> card).
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > 4. (minor point) Deprecation of methods
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > You mentioned that the new `commit` 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?
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > Thanks again for the KIP! It's really nice that
> you
> >> and
> >> > > > Alex
> >> > > > > > will
> >> > > > > > >> > get
> >> > > > > > >> > >> the
> >> > > > > > >> > >> > > chance to collaborate on both directions so that
> we
> >> can
> >> > > get
> >> > > > > the
> >> > > > > > >> best
> >> > > > > > >> > >> > > outcome for Streams and its users.
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > -John
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > On 2022/11/21 15:02:15 Nick Telford wrote:
> >> > > > > > >> > >> > > > 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
> >> > > > > > >> > >> > > > <
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> >
> >> > > > > > >> > >>
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >> > > > > > >> > >> > > >,
> >> > > > > > >> > >> > > > so that it can be discussed/reviewed separately
> >> from
> >> > > > > KIP-844.
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > Alex: I'm especially interested in what you
> think!
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > I have a nearly complete implementation of the
> >> changes
> >> > > > > > >> outlined in
> >> > > > > > >> > >> this
> >> > > > > > >> > >> > > > KIP, please let me know if you'd like me to push
> >> them
> >> > > for
> >> > > > > > >> review
> >> > > > > > >> > in
> >> > > > > > >> > >> > > advance
> >> > > > > > >> > >> > > > of a vote.
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > Regards,
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > Nick
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> >
> >> > > > > > >> > >>
> >> > > > > > >> > >
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >>
> >
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi everyone,

I've updated the KIP to reflect the latest version of the design:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores

There are several changes in there that reflect feedback from this thread,
and there's a new section and a bunch of interface changes relating to
Atomic Checkpointing, which is the final piece of the puzzle to making
everything robust.

Let me know what you think!

Regards,
Nick

On Tue, 3 Jan 2023 at 11:33, Nick Telford <ni...@gmail.com> wrote:

> Hi Lucas,
>
> Thanks for looking over my KIP.
>
> A) The bound is per-instance, not per-Task. This was a typo in the KIP
> that I've now corrected. It was originally per-Task, but I changed it to
> per-instance for exactly the reason you highlighted.
> B) It's worth noting that transactionality is only enabled under EOS, and
> in the default mode of operation (ALOS), there should be no change in
> behavior at all. I think, under EOS, we can mitigate the impact on users by
> sufficiently low default values for the memory bound configuration. I
> understand your hesitation to include a significant change of behaviour,
> especially in a minor release, but I suspect that most users will prefer
> the memory impact (under EOS) to the existing behaviour of frequent state
> restorations! If this is a problem, the changes can wait until the next
> major release. I'll be running a patched version of streams in production
> with these changes as soon as they're ready, so it won't disrupt me :-D
> C) The main purpose of this sentence was just to note that some changes
> will need to be made to the way Segments are handled in order to ensure
> they also benefit from transactions. At the time I wrote it, I hadn't
> figured out the specific changes necessary, so it was deliberately vague.
> This is the one outstanding problem I'm currently working on, and I'll
> update this section with more detail once I have figured out the exact
> changes required.
> D) newTransaction() provides the necessary isolation guarantees. While
> the RocksDB implementation of transactions doesn't technically *need*
> read-only users to call newTransaction(), other implementations (e.g. a
> hypothetical PostgresStore) may require it. Calling newTransaction() when
> no transaction is necessary is essentially free, as it will just return
> this.
>
> I didn't do any profiling of the KIP-844 PoC, but I think it should be
> fairly obvious where the performance problems stem from: writes under
> KIP-844 require 3 extra memory-copies: 1 to encode it with the
> tombstone/record flag, 1 to decode it from the tombstone/record flag, and 1
> to copy the record from the "temporary" store to the "main" store, when the
> transaction commits. The different approach taken by KIP-869 should perform
> much better, as it avoids all these copies, and may actually perform
> slightly better than trunk, due to batched writes in RocksDB performing
> better than non-batched writes.[1]
>
> Regards,
> Nick
>
> 1: https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>
> On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <lb...@confluent.io.invalid>
> wrote:
>
>> Hi Nick,
>>
>> I'm just starting to read up on the whole discussion about KIP-892 and
>> KIP-844. Thanks a lot for your work on this, I do think
>> `WriteBatchWithIndex` may be the way to go here. I do have some
>> questions about the latest draft.
>>
>>  A) If I understand correctly, you propose to put a bound on the
>> (native) memory consumed by each task. However, I wonder if this is
>> sufficient if we have temporary imbalances in the cluster. For
>> example, depending on the timing of rebalances during a cluster
>> restart, it could happen that a single streams node is assigned a lot
>> more tasks than expected. With your proposed change, this would mean
>> that the memory required by this one node could be a multiple of what
>> is required during normal operation. I wonder if it wouldn't be safer
>> to put a global bound on the memory use, across all tasks.
>>  B) Generally, the memory concerns still give me the feeling that this
>> should not be enabled by default for all users in a minor release.
>>  C) In section "Transaction Management": the sentence "A similar
>> analogue will be created to automatically manage `Segment`
>> transactions.". Maybe this is just me lacking some background, 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 you to find out why the KIP-844 PoC is 20x slower (e.g.
>> by providing a flame graph), that would be quite interesting.
>>
>> Cheers,
>> Lucas
>>
>> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <ni...@gmail.com>
>> wrote:
>> >
>> > Hi everyone,
>> >
>> > I've updated the KIP with a more detailed design, which reflects the
>> > implementation I've been working on:
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> >
>> > This new design should address the outstanding points already made in
>> the
>> > thread.
>> >
>> > Please let me know if there are areas that are unclear or need more
>> > clarification.
>> >
>> > I have a (nearly) working implementation. I'm confident that the
>> remaining
>> > work (making Segments behave) will not impact the documented design.
>> >
>> > Regards,
>> >
>> > Nick
>> >
>> > On Tue, 6 Dec 2022 at 19:24, Colt McNealy <co...@littlehorse.io> wrote:
>> >
>> > > Nick,
>> > >
>> > > Thank you for the reply; that makes sense. I was hoping that, since
>> reading
>> > > uncommitted records from IQ in EOS isn't part of the documented API,
>> maybe
>> > > you *wouldn't* have to wait for the next major release to make that
>> change;
>> > > but given 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 <ni...@gmail.com>
>> > > wrote:
>> > >
>> > > > Hi Colt,
>> > > >
>> > > > 10: Yes, I agree it's not ideal. I originally intended to try to
>> keep the
>> > > > behaviour unchanged as much as possible, otherwise we'd have to
>> wait for
>> > > a
>> > > > major version release to land these changes.
>> > > > 20: Good point, ALOS doesn't need the same level of guarantee, and
>> the
>> > > > typically longer commit intervals would be problematic when reading
>> only
>> > > > "committed" records.
>> > > >
>> > > > I've been away for 5 days recovering from minor surgery, but I
>> spent a
>> > > > considerable amount of that time working through ideas for possible
>> > > > solutions in my head. I think your suggestion of keeping ALOS
>> as-is, but
>> > > > buffering writes for EOS is the right path forwards, although I
>> have a
>> > > > solution that both expands on this, and provides for some more
>> formal
>> > > > guarantees.
>> > > >
>> > > > Essentially, adding support to KeyValueStores for "Transactions",
>> with
>> > > > clearly defined IsolationLevels. Using "Read Committed" when under
>> EOS,
>> > > and
>> > > > "Read Uncommitted" under ALOS.
>> > > >
>> > > > The nice thing about this approach is that it gives us much more
>> clearly
>> > > > defined isolation behaviour that can be properly documented to
>> ensure
>> > > users
>> > > > know what to expect.
>> > > >
>> > > > I'm still working out the kinks in the design, and will update the
>> KIP
>> > > when
>> > > > I have something. The main struggle is trying to implement this
>> without
>> > > > making any major changes to the existing interfaces or breaking
>> existing
>> > > > implementations, because currently everything expects to operate
>> directly
>> > > > on a StateStore, and not a Transaction of that store. I think I'm
>> getting
>> > > > close, although sadly I won't be able to progress much until next
>> week
>> > > due
>> > > > to some work commitments.
>> > > >
>> > > > Regards,
>> > > > Nick
>> > > >
>> > > > On Thu, 1 Dec 2022 at 00:01, Colt McNealy <co...@littlehorse.io>
>> wrote:
>> > > >
>> > > > > Nick,
>> > > > >
>> > > > > Thank you for the explanation, and also for the updated KIP. I am
>> quite
>> > > > > eager for this improvement to be released as it would greatly
>> reduce
>> > > the
>> > > > > operational difficulties of EOS streams apps.
>> > > > >
>> > > > > Two questions:
>> > > > >
>> > > > > 10)
>> > > > > >When reading records, we will use the
>> > > > > WriteBatchWithIndex#getFromBatchAndDB
>> > > > >  and WriteBatchWithIndex#newIteratorWithBase utilities in order to
>> > > ensure
>> > > > > that uncommitted writes are available to query.
>> > > > > Why do extra work to enable the reading of uncommitted writes
>> during
>> > > IQ?
>> > > > > Code complexity aside, reading uncommitted writes is, in my
>> opinion, a
>> > > > > minor flaw in EOS IQ; it would be very nice to have the guarantee
>> that,
>> > > > > with EOS, IQ only reads committed records. In order to avoid dirty
>> > > reads,
>> > > > > one currently must query a standby replica (but this still doesn't
>> > > fully
>> > > > > guarantee monotonic reads).
>> > > > >
>> > > > > 20) Is it also necessary to enable this optimization on ALOS
>> stores?
>> > > The
>> > > > > motivation of KIP-844 was mainly to reduce the need to restore
>> state
>> > > from
>> > > > > scratch on unclean EOS shutdowns; with ALOS it was acceptable to
>> accept
>> > > > > that there may have been uncommitted writes on disk. On a side
>> note, if
>> > > > you
>> > > > > enable this type of store on ALOS processors, the community would
>> > > > > 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 <
>> nick.telford@gmail.com>
>> > > > > wrote:
>> > > > >
>> > > > > > Hi everyone,
>> > > > > >
>> > > > > > I've drastically reduced the scope of this KIP to no longer
>> include
>> > > the
>> > > > > > StateStore management of checkpointing. This can be added as a
>> KIP
>> > > > later
>> > > > > on
>> > > > > > to further optimize the consistency and performance of state
>> stores.
>> > > > > >
>> > > > > > I've also added a section discussing some of the concerns 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 2022 at 16:36, Nick Telford <
>> nick.telford@gmail.com>
>> > > > > wrote:
>> > > > > >
>> > > > > > > Hi Colt,
>> > > > > > >
>> > > > > > > I didn't do any profiling, but the 844 implementation:
>> > > > > > >
>> > > > > > >    - Writes uncommitted records to a temporary RocksDB
>> instance
>> > > > > > >       - Since tombstones need to be flagged, all record
>> values are
>> > > > > > >       prefixed with a value/tombstone marker. This
>> necessitates a
>> > > > > memory
>> > > > > > copy.
>> > > > > > >    - On-commit, iterates all records in this temporary
>> instance and
>> > > > > > >    writes them to the main RocksDB store.
>> > > > > > >    - While iterating, the value/tombstone marker needs to be
>> parsed
>> > > > and
>> > > > > > >    the real value extracted. This necessitates another memory
>> copy.
>> > > > > > >
>> > > > > > > My guess is that the cost of iterating the temporary RocksDB
>> store
>> > > is
>> > > > > the
>> > > > > > > major factor, with the 2 extra memory copies per-Record
>> > > contributing
>> > > > a
>> > > > > > > significant amount too.
>> > > > > > >
>> > > > > > > Regards,
>> > > > > > > Nick
>> > > > > > >
>> > > > > > > On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
>> colt@littlehorse.io>
>> > > > > wrote:
>> > > > > > >
>> > > > > > >> Hi all,
>> > > > > > >>
>> > > > > > >> Out of curiosity, why does the performance of the store
>> degrade so
>> > > > > > >> significantly with the 844 implementation? I wouldn't be too
>> > > > surprised
>> > > > > > by
>> > > > > > >> a
>> > > > > > >> 50-60% drop (caused by each record being written twice), but
>> 96%
>> > > is
>> > > > > > >> extreme.
>> > > > > > >>
>> > > > > > >> The only thing I can think of which could create such a
>> bottleneck
>> > > > > 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 Nick Telford <
>> > > > nick.telford@gmail.com>
>> > > > > > >> wrote:
>> > > > > > >>
>> > > > > > >> > Hi everyone,
>> > > > > > >> >
>> > > > > > >> > I've updated the KIP to resolve all the points that have
>> been
>> > > > raised
>> > > > > > so
>> > > > > > >> > far, with one exception: the ALOS default commit interval
>> of 5
>> > > > > minutes
>> > > > > > >> is
>> > > > > > >> > likely to cause WriteBatchWithIndex memory to grow too
>> large.
>> > > > > > >> >
>> > > > > > >> > There's a couple of different things I can think of to
>> solve
>> > > this:
>> > > > > > >> >
>> > > > > > >> >    - We already have a memory/record limit in the KIP to
>> prevent
>> > > > OOM
>> > > > > > >> >    errors. Should we choose a default value for these? My
>> > > concern
>> > > > > here
>> > > > > > >> is
>> > > > > > >> > that
>> > > > > > >> >    anything we choose might seem rather arbitrary. We could
>> > > change
>> > > > > > >> >    its behaviour such that under ALOS, it only triggers the
>> > > commit
>> > > > > of
>> > > > > > >> the
>> > > > > > >> >    StateStore, but under EOS, it triggers a commit of the
>> Kafka
>> > > > > > >> > transaction.
>> > > > > > >> >    - We could introduce a separate `checkpoint.interval.ms`
>> to
>> > > > > allow
>> > > > > > >> ALOS
>> > > > > > >> >    to commit the StateStores more frequently than the
>> general
>> > > > > > >> >    commit.interval.ms? My concern here is that the
>> semantics of
>> > > > > this
>> > > > > > >> > config
>> > > > > > >> >    would depend 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 <
>> > > > nick.telford@gmail.com>
>> > > > > > >> wrote:
>> > > > > > >> >
>> > > > > > >> > > Hi Alex,
>> > > > > > >> > >
>> > > > > > >> > > Thanks for the feedback.
>> > > > > > >> > >
>> > > > > > >> > > I've updated the discussion of OOM issues by describing
>> how
>> > > > we'll
>> > > > > > >> handle
>> > > > > > >> > > it. Here's the new text:
>> > > > > > >> > >
>> > > > > > >> > > To mitigate this, we will automatically force a Task
>> commit if
>> > > > the
>> > > > > > >> total
>> > > > > > >> > >> uncommitted records returned by
>> > > > > > >> > >> StateStore#approximateNumUncommittedEntries()  exceeds a
>> > > > > threshold,
>> > > > > > >> > >> configured by max.uncommitted.state.entries.per.task;
>> or the
>> > > > > total
>> > > > > > >> > >> memory used for buffering uncommitted records returned
>> by
>> > > > > > >> > >> StateStore#approximateNumUncommittedBytes() exceeds the
>> > > > threshold
>> > > > > > >> > >> configured by max.uncommitted.state.bytes.per.task.
>> This will
>> > > > > > roughly
>> > > > > > >> > >> bound the memory required per-Task for buffering
>> uncommitted
>> > > > > > records,
>> > > > > > >> > >> irrespective of the commit.interval.ms, and will
>> effectively
>> > > > > bound
>> > > > > > >> the
>> > > > > > >> > >> number of records that will need to be restored in the
>> event
>> > > > of a
>> > > > > > >> > failure.
>> > > > > > >> > >>
>> > > > > > >> > >
>> > > > > > >> > >
>> > > > > > >> > > These limits will be checked in StreamTask#process and a
>> > > > premature
>> > > > > > >> commit
>> > > > > > >> > >> will be requested via Task#requestCommit().
>> > > > > > >> > >>
>> > > > > > >> > >
>> > > > > > >> > >
>> > > > > > >> > > Note that these new methods provide default
>> implementations
>> > > that
>> > > > > > >> ensure
>> > > > > > >> > >> existing custom stores and non-transactional stores
>> (e.g.
>> > > > > > >> > >> InMemoryKeyValueStore) do not force any early commits.
>> > > > > > >> > >
>> > > > > > >> > >
>> > > > > > >> > > I've chosen to have the StateStore expose approximations
>> of
>> > > its
>> > > > > > buffer
>> > > > > > >> > > size/count instead of opaquely requesting a commit in
>> order to
>> > > > > > >> delegate
>> > > > > > >> > the
>> > > > > > >> > > decision making to the Task itself. This enables Tasks
>> to look
>> > > > at
>> > > > > > >> *all*
>> > > > > > >> > of
>> > > > > > >> > > their StateStores, and determine whether an early commit
>> is
>> > > > > > necessary.
>> > > > > > >> > > Notably, it enables pre-Task thresholds, instead of
>> per-Store,
>> > > > > which
>> > > > > > >> > > prevents Tasks with many StateStores from using much more
>> > > memory
>> > > > > > than
>> > > > > > >> > Tasks
>> > > > > > >> > > with one StateStore. This makes sense, since commits are
>> done
>> > > > > > by-Task,
>> > > > > > >> > not
>> > > > > > >> > > by-Store.
>> > > > > > >> > >
>> > > > > > >> > > Prizes* for anyone who can come up with a better name
>> for the
>> > > > new
>> > > > > > >> config
>> > > > > > >> > > properties!
>> > > > > > >> > >
>> > > > > > >> > > Thanks for pointing out the potential performance issues
>> of
>> > > > WBWI.
>> > > > > > From
>> > > > > > >> > the
>> > > > > > >> > > benchmarks that user posted[1], it looks like WBWI still
>> > > > performs
>> > > > > > >> > > considerably better than individual puts, which is the
>> > > existing
>> > > > > > >> design,
>> > > > > > >> > so
>> > > > > > >> > > I'd actually expect a performance boost from WBWI, just
>> not as
>> > > > > great
>> > > > > > >> as
>> > > > > > >> > > we'd get from a plain WriteBatch. This does suggest that
>> a
>> > > good
>> > > > > > >> > > optimization would be to use a regular WriteBatch for
>> > > > restoration
>> > > > > > (in
>> > > > > > >> > > RocksDBStore#restoreBatch), since we know that those
>> records
>> > > > will
>> > > > > > >> never
>> > > > > > >> > be
>> > > > > > >> > > queried before they're committed.
>> > > > > > >> > >
>> > > > > > >> > > 1:
>> > > > > > >> >
>> > > > > >
>> > > >
>> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
>> > > > > > >> > >
>> > > > > > >> > > * Just kidding, no prizes, sadly.
>> > > > > > >> > >
>> > > > > > >> > > On Wed, 23 Nov 2022 at 12:28, Alexander Sorokoumov
>> > > > > > >> > > <as...@confluent.io.invalid> wrote:
>> > > > > > >> > >
>> > > > > > >> > >> 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 to keep the current checkpointing
>> as a
>> > > > > > fallback
>> > > > > > >> > >> mechanism. We want to keep existing users' workflows
>> intact
>> > > if
>> > > > we
>> > > > > > >> can. A
>> > > > > > >> > >> non-intrusive way would be to add a separate StateStore
>> > > method,
>> > > > > > say,
>> > > > > > >> > >> StateStore#managesCheckpointing(), that controls
>> whether the
>> > > > > state
>> > > > > > >> store
>> > > > > > >> > >> implementation owns checkpointing.
>> > > > > > >> > >>
>> > > > > > >> > >> I think that a solution to the transactional writes
>> should
>> > > > > address
>> > > > > > >> the
>> > > > > > >> > >> OOMEs. One possible way to address that is to wire
>> > > StateStore's
>> > > > > > >> commit
>> > > > > > >> > >> request by adding, say, StateStore#commitNeeded that is
>> > > checked
>> > > > > in
>> > > > > > >> > >> StreamTask#commitNeeded via the corresponding
>> > > > > > ProcessorStateManager.
>> > > > > > >> > With
>> > > > > > >> > >> that change, RocksDBStore will have to track the current
>> > > > > > transaction
>> > > > > > >> > size
>> > > > > > >> > >> and request a commit when the size goes over a
>> (configurable)
>> > > > > > >> threshold.
>> > > > > > >> > >>
>> > > > > > >> > >> AFAIU WriteBatchWithIndex might perform significantly
>> slower
>> > > > than
>> > > > > > >> > non-txn
>> > > > > > >> > >> puts as the batch size grows [1]. We should have a
>> > > > configuration
>> > > > > to
>> > > > > > >> fall
>> > > > > > >> > >> back to the current behavior (and/or disable txn stores
>> for
>> > > > ALOS)
>> > > > > > >> unless
>> > > > > > >> > >> the benchmarks show negligible overhead for longer
>> commits /
>> > > > > > >> > large-enough
>> > > > > > >> > >> batch sizes.
>> > > > > > >> > >>
>> > > > > > >> > >> If you prefer to keep the KIP smaller, I would rather
>> cut out
>> > > > > > >> > >> state-store-managed checkpointing rather than proper
>> OOMe
>> > > > > handling
>> > > > > > >> and
>> > > > > > >> > >> being able to switch to non-txn behavior. The
>> checkpointing
>> > > 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 configuration parts as well as review and test
>> the
>> > > > > patch.
>> > > > > > >> > >>
>> > > > > > >> > >> Best,
>> > > > > > >> > >> Alex
>> > > > > > >> > >>
>> > > > > > >> > >>
>> > > > > > >> > >> 1. https://github.com/facebook/rocksdb/issues/608
>> > > > > > >> > >>
>> > > > > > >> > >> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
>> > > > > > nick.telford@gmail.com
>> > > > > > >> >
>> > > > > > >> > >> wrote:
>> > > > > > >> > >>
>> > > > > > >> > >> > Hi John,
>> > > > > > >> > >> >
>> > > > > > >> > >> > Thanks for the review and feedback!
>> > > > > > >> > >> >
>> > > > > > >> > >> > 1. Custom Stores: I've been mulling over this problem
>> > > myself.
>> > > > > As
>> > > > > > it
>> > > > > > >> > >> stands,
>> > > > > > >> > >> > custom stores would essentially lose checkpointing
>> with no
>> > > > > > >> indication
>> > > > > > >> > >> that
>> > > > > > >> > >> > they're expected to make changes, besides a line in
>> the
>> > > > release
>> > > > > > >> > notes. I
>> > > > > > >> > >> > agree that the best solution would be to provide a
>> default
>> > > > that
>> > > > > > >> > >> checkpoints
>> > > > > > >> > >> > to a file. The one thing I would change is that the
>> > > > > checkpointing
>> > > > > > >> is
>> > > > > > >> > to
>> > > > > > >> > >> a
>> > > > > > >> > >> > store-local file, instead of a per-Task file. This
>> way the
>> > > > > > >> StateStore
>> > > > > > >> > >> still
>> > > > > > >> > >> > technically owns its own checkpointing (via a default
>> > > > > > >> implementation),
>> > > > > > >> > >> and
>> > > > > > >> > >> > the StateManager/Task execution engine doesn't need
>> to know
>> > > > > > >> anything
>> > > > > > >> > >> about
>> > > > > > >> > >> > checkpointing, which greatly simplifies some of the
>> logic.
>> > > > > > >> > >> >
>> > > > > > >> > >> > 2. OOME errors: The main reasons why I didn't explore
>> a
>> > > > > solution
>> > > > > > to
>> > > > > > >> > >> this is
>> > > > > > >> > >> > a) to keep this KIP as simple as possible, and b)
>> because
>> > > I'm
>> > > > > not
>> > > > > > >> > >> exactly
>> > > > > > >> > >> > how to signal that a Task should commit prematurely.
>> I'm
>> > > > > > confident
>> > > > > > >> > it's
>> > > > > > >> > >> > possible, and I think it's worth adding a section on
>> > > handling
>> > > > > > this.
>> > > > > > >> > >> Besides
>> > > > > > >> > >> > my proposal to force an early commit once memory usage
>> > > > reaches
>> > > > > a
>> > > > > > >> > >> threshold,
>> > > > > > >> > >> > is there any other approach that you might suggest for
>> > > > tackling
>> > > > > > >> this
>> > > > > > >> > >> > problem?
>> > > > > > >> > >> >
>> > > > > > >> > >> > 3. ALOS: I can add in an explicit paragraph, but my
>> > > > assumption
>> > > > > is
>> > > > > > >> that
>> > > > > > >> > >> > since transactional behaviour comes at little/no
>> cost, that
>> > > > it
>> > > > > > >> should
>> > > > > > >> > be
>> > > > > > >> > >> > available by default on all stores, irrespective of
>> the
>> > > > > > processing
>> > > > > > >> > mode.
>> > > > > > >> > >> > While ALOS doesn't use transactions, the Task itself
>> still
>> > > > > > >> "commits",
>> > > > > > >> > so
>> > > > > > >> > >> > the behaviour should be correct under ALOS too. I'm
>> not
>> > > > > convinced
>> > > > > > >> that
>> > > > > > >> > >> it's
>> > > > > > >> > >> > worth having both transactional/non-transactional
>> stores
>> > > > > > >> available, as
>> > > > > > >> > >> it
>> > > > > > >> > >> > would considerably increase the complexity of the
>> codebase,
>> > > > for
>> > > > > > >> very
>> > > > > > >> > >> little
>> > > > > > >> > >> > benefit.
>> > > > > > >> > >> >
>> > > > > > >> > >> > 4. Method deprecation: Are you referring to
>> > > > > > >> StateStore#getPosition()?
>> > > > > > >> > >> As I
>> > > > > > >> > >> > understand it, Position contains the position of the
>> > > *source*
>> > > > > > >> topics,
>> > > > > > >> > >> > whereas the commit offsets would be the *changelog*
>> > > offsets.
>> > > > So
>> > > > > > >> it's
>> > > > > > >> > >> still
>> > > > > > >> > >> > necessary to retain the Position data, as well as the
>> > > > changelog
>> > > > > > >> > offsets.
>> > > > > > >> > >> > What I meant in the KIP is that Position offsets are
>> > > > currently
>> > > > > > >> stored
>> > > > > > >> > >> in a
>> > > > > > >> > >> > file, and since we can atomically store metadata
>> along with
>> > > > the
>> > > > > > >> record
>> > > > > > >> > >> > batch we commit to RocksDB, we can move our Position
>> > > offsets
>> > > > in
>> > > > > > to
>> > > > > > >> > this
>> > > > > > >> > >> > metadata too, and gain the same transactional
>> guarantees
>> > > that
>> > > > > we
>> > > > > > >> will
>> > > > > > >> > >> for
>> > > > > > >> > >> > changelog offsets, ensuring that the Position offsets
>> are
>> > > > > > >> consistent
>> > > > > > >> > >> with
>> > > > > > >> > >> > the records that are read from the database.
>> > > > > > >> > >> >
>> > > > > > >> > >> > Regards,
>> > > > > > >> > >> > Nick
>> > > > > > >> > >> >
>> > > > > > >> > >> > On Tue, 22 Nov 2022 at 16:25, John Roesler <
>> > > > > vvcephei@apache.org>
>> > > > > > >> > wrote:
>> > > > > > >> > >> >
>> > > > > > >> > >> > > Thanks for publishing this alternative, Nick!
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > The benchmark you mentioned in the KIP-844
>> discussion
>> > > seems
>> > > > > > like
>> > > > > > >> a
>> > > > > > >> > >> > > compelling reason to revisit the built-in
>> > > transactionality
>> > > > > > >> > mechanism.
>> > > > > > >> > >> I
>> > > > > > >> > >> > > also appreciate you analysis, showing that for most
>> use
>> > > > > cases,
>> > > > > > >> the
>> > > > > > >> > >> write
>> > > > > > >> > >> > > batch approach should be just fine.
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > There are a couple of points that would hold me
>> back from
>> > > > > > >> approving
>> > > > > > >> > >> this
>> > > > > > >> > >> > > KIP right now:
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > 1. Loss of coverage for custom stores.
>> > > > > > >> > >> > > The fact that you can plug in a (relatively) simple
>> > > > > > >> implementation
>> > > > > > >> > of
>> > > > > > >> > >> the
>> > > > > > >> > >> > > XStateStore interfaces and automagically get a
>> > > distributed
>> > > > > > >> database
>> > > > > > >> > >> out
>> > > > > > >> > >> > of
>> > > > > > >> > >> > > it is a significant benefit of Kafka Streams. I'd
>> hate to
>> > > > > lose
>> > > > > > >> it,
>> > > > > > >> > so
>> > > > > > >> > >> it
>> > > > > > >> > >> > > would be better to spend some time and come up with
>> a way
>> > > > to
>> > > > > > >> > preserve
>> > > > > > >> > >> > that
>> > > > > > >> > >> > > property. For example, can we provide a default
>> > > > > implementation
>> > > > > > of
>> > > > > > >> > >> > > `commit(..)` that re-implements the existing
>> > > > checkpoint-file
>> > > > > > >> > >> approach? Or
>> > > > > > >> > >> > > perhaps add an `isTransactional()` flag to the state
>> > > store
>> > > > > > >> interface
>> > > > > > >> > >> so
>> > > > > > >> > >> > > that the runtime can decide whether to continue to
>> manage
>> > > > > > >> checkpoint
>> > > > > > >> > >> > files
>> > > > > > >> > >> > > vs delegating transactionality to the stores?
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > 2. Guarding against OOME
>> > > > > > >> > >> > > I appreciate your analysis, but I don't think it's
>> > > > sufficient
>> > > > > > to
>> > > > > > >> say
>> > > > > > >> > >> that
>> > > > > > >> > >> > > we will solve the memory problem later if it becomes
>> > > > > necessary.
>> > > > > > >> The
>> > > > > > >> > >> > > experience leading to that situation would be quite
>> bad:
>> > > > > > Imagine,
>> > > > > > >> > you
>> > > > > > >> > >> > > upgrade to AK 3.next, your tests pass, so you
>> deploy to
>> > > > > > >> production.
>> > > > > > >> > >> That
>> > > > > > >> > >> > > night, you get paged because your app is now
>> crashing
>> > > with
>> > > > > > >> OOMEs. As
>> > > > > > >> > >> with
>> > > > > > >> > >> > > all OOMEs, you'll have a really hard time finding
>> the
>> > > root
>> > > > > > cause,
>> > > > > > >> > and
>> > > > > > >> > >> > once
>> > > > > > >> > >> > > you do, you won't have a clear path to resolve the
>> issue.
>> > > > You
>> > > > > > >> could
>> > > > > > >> > >> only
>> > > > > > >> > >> > > tune down the commit interval and cache buffer size
>> until
>> > > > you
>> > > > > > >> stop
>> > > > > > >> > >> > getting
>> > > > > > >> > >> > > crashes.
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > FYI, I know of multiple cases where people run EOS
>> with
>> > > > much
>> > > > > > >> larger
>> > > > > > >> > >> > commit
>> > > > > > >> > >> > > intervals to get better batching than the default,
>> so I
>> > > > don't
>> > > > > > >> think
>> > > > > > >> > >> this
>> > > > > > >> > >> > > pathological case would be as rare as you suspect.
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > Given that we already have the rudiments of an idea
>> of
>> > > what
>> > > > > we
>> > > > > > >> could
>> > > > > > >> > >> do
>> > > > > > >> > >> > to
>> > > > > > >> > >> > > prevent this downside, we should take the time to
>> design
>> > > a
>> > > > > > >> solution.
>> > > > > > >> > >> We
>> > > > > > >> > >> > owe
>> > > > > > >> > >> > > it to our users to ensure that awesome new features
>> don't
>> > > > > come
>> > > > > > >> with
>> > > > > > >> > >> > bitter
>> > > > > > >> > >> > > pills unless we can't avoid it.
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > 3. ALOS mode.
>> > > > > > >> > >> > > On the other hand, I didn't see an indication of how
>> > > stores
>> > > > > > will
>> > > > > > >> be
>> > > > > > >> > >> > > handled under ALOS (aka non-EOS) mode.
>> Theoretically, the
>> > > > > > >> > >> > transactionality
>> > > > > > >> > >> > > of the store and the processing mode are
>> orthogonal. A
>> > > > > > >> transactional
>> > > > > > >> > >> > store
>> > > > > > >> > >> > > would serve ALOS just as well as a
>> non-transactional one
>> > > > (if
>> > > > > > not
>> > > > > > >> > >> better).
>> > > > > > >> > >> > > Under ALOS, though, the default commit interval is
>> five
>> > > > > > minutes,
>> > > > > > >> so
>> > > > > > >> > >> the
>> > > > > > >> > >> > > memory issue is far more pressing.
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > As I see it, we have several options to resolve this
>> > > point.
>> > > > > We
>> > > > > > >> could
>> > > > > > >> > >> > > demonstrate that transactional stores work just
>> fine for
>> > > > ALOS
>> > > > > > >> and we
>> > > > > > >> > >> can
>> > > > > > >> > >> > > therefore just swap over unconditionally. We could
>> also
>> > > > > disable
>> > > > > > >> the
>> > > > > > >> > >> > > transactional mechanism under ALOS so that stores
>> operate
>> > > > > just
>> > > > > > >> the
>> > > > > > >> > >> same
>> > > > > > >> > >> > as
>> > > > > > >> > >> > > they do today when run in ALOS mode. Finally, we
>> could do
>> > > > the
>> > > > > > >> same
>> > > > > > >> > as
>> > > > > > >> > >> in
>> > > > > > >> > >> > > KIP-844 and make transactional stores opt-in (it'd
>> be
>> > > > better
>> > > > > to
>> > > > > > >> > avoid
>> > > > > > >> > >> the
>> > > > > > >> > >> > > extra opt-in mechanism, but it's a good
>> > > > get-out-of-jail-free
>> > > > > > >> card).
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > 4. (minor point) Deprecation of methods
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > You mentioned that the new `commit` 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?
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > Thanks again for the KIP! It's really nice that you
>> and
>> > > > Alex
>> > > > > > will
>> > > > > > >> > get
>> > > > > > >> > >> the
>> > > > > > >> > >> > > chance to collaborate on both directions so that we
>> can
>> > > get
>> > > > > the
>> > > > > > >> best
>> > > > > > >> > >> > > outcome for Streams and its users.
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > -John
>> > > > > > >> > >> > >
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > On 2022/11/21 15:02:15 Nick Telford wrote:
>> > > > > > >> > >> > > > 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
>> > > > > > >> > >> > > > <
>> > > > > > >> > >> > >
>> > > > > > >> > >> >
>> > > > > > >> > >>
>> > > > > > >> >
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
>> > > > > > >> > >> > > >,
>> > > > > > >> > >> > > > so that it can be discussed/reviewed separately
>> from
>> > > > > KIP-844.
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > Alex: I'm especially interested in what you think!
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > I have a nearly complete implementation of the
>> changes
>> > > > > > >> outlined in
>> > > > > > >> > >> this
>> > > > > > >> > >> > > > KIP, please let me know if you'd like me to push
>> them
>> > > for
>> > > > > > >> review
>> > > > > > >> > in
>> > > > > > >> > >> > > advance
>> > > > > > >> > >> > > > of a vote.
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > Regards,
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > Nick
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > >
>> > > > > > >> > >> >
>> > > > > > >> > >>
>> > > > > > >> > >
>> > > > > > >> >
>> > > > > > >>
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>>
>

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

Posted by Nick Telford <ni...@gmail.com>.
Hi Lucas,

Thanks for looking over my KIP.

A) The bound is per-instance, not per-Task. This was a typo in the KIP that
I've now corrected. It was originally per-Task, but I changed it to
per-instance for exactly the reason you highlighted.
B) It's worth noting that transactionality is only enabled under EOS, and
in the default mode of operation (ALOS), there should be no change in
behavior at all. I think, under EOS, we can mitigate the impact on users by
sufficiently low default values for the memory bound configuration. I
understand your hesitation to include a significant change of behaviour,
especially in a minor release, but I suspect that most users will prefer
the memory impact (under EOS) to the existing behaviour of frequent state
restorations! If this is a problem, the changes can wait until the next
major release. I'll be running a patched version of streams in production
with these changes as soon as they're ready, so it won't disrupt me :-D
C) The main purpose of this sentence was just to note that some changes
will need to be made to the way Segments are handled in order to ensure
they also benefit from transactions. At the time I wrote it, I hadn't
figured out the specific changes necessary, so it was deliberately vague.
This is the one outstanding problem I'm currently working on, and I'll
update this section with more detail once I have figured out the exact
changes required.
D) newTransaction() provides the necessary isolation guarantees. While the
RocksDB implementation of transactions doesn't technically *need* read-only
users to call newTransaction(), other implementations (e.g. a hypothetical
PostgresStore) may require it. Calling newTransaction() when no transaction
is necessary is essentially free, as it will just return this.

I didn't do any profiling of the KIP-844 PoC, but I think it should be
fairly obvious where the performance problems stem from: writes under
KIP-844 require 3 extra memory-copies: 1 to encode it with the
tombstone/record flag, 1 to decode it from the tombstone/record flag, and 1
to copy the record from the "temporary" store to the "main" store, when the
transaction commits. The different approach taken by KIP-869 should perform
much better, as it avoids all these copies, and may actually perform
slightly better than trunk, due to batched writes in RocksDB performing
better than non-batched writes.[1]

Regards,
Nick

1: https://github.com/adamretter/rocksjava-write-methods-benchmark#results

On Mon, 2 Jan 2023 at 16:18, Lucas Brutschy <lb...@confluent.io.invalid>
wrote:

> Hi Nick,
>
> I'm just starting to read up on the whole discussion about KIP-892 and
> KIP-844. Thanks a lot for your work on this, I do think
> `WriteBatchWithIndex` may be the way to go here. I do have some
> questions about the latest draft.
>
>  A) If I understand correctly, you propose to put a bound on the
> (native) memory consumed by each task. However, I wonder if this is
> sufficient if we have temporary imbalances in the cluster. For
> example, depending on the timing of rebalances during a cluster
> restart, it could happen that a single streams node is assigned a lot
> more tasks than expected. With your proposed change, this would mean
> that the memory required by this one node could be a multiple of what
> is required during normal operation. I wonder if it wouldn't be safer
> to put a global bound on the memory use, across all tasks.
>  B) Generally, the memory concerns still give me the feeling that this
> should not be enabled by default for all users in a minor release.
>  C) In section "Transaction Management": the sentence "A similar
> analogue will be created to automatically manage `Segment`
> transactions.". Maybe this is just me lacking some background, 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 you to find out why the KIP-844 PoC is 20x slower (e.g.
> by providing a flame graph), that would be quite interesting.
>
> Cheers,
> Lucas
>
> On Thu, Dec 22, 2022 at 8:30 PM Nick Telford <ni...@gmail.com>
> wrote:
> >
> > Hi everyone,
> >
> > I've updated the KIP with a more detailed design, which reflects the
> > implementation I've been working on:
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> >
> > This new design should address the outstanding points already made in the
> > thread.
> >
> > Please let me know if there are areas that are unclear or need more
> > clarification.
> >
> > I have a (nearly) working implementation. I'm confident that the
> remaining
> > work (making Segments behave) will not impact the documented design.
> >
> > Regards,
> >
> > Nick
> >
> > On Tue, 6 Dec 2022 at 19:24, Colt McNealy <co...@littlehorse.io> wrote:
> >
> > > Nick,
> > >
> > > Thank you for the reply; that makes sense. I was hoping that, since
> reading
> > > uncommitted records from IQ in EOS isn't part of the documented API,
> maybe
> > > you *wouldn't* have to wait for the next major release to make that
> change;
> > > but given 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 <ni...@gmail.com>
> > > wrote:
> > >
> > > > Hi Colt,
> > > >
> > > > 10: Yes, I agree it's not ideal. I originally intended to try to
> keep the
> > > > behaviour unchanged as much as possible, otherwise we'd have to wait
> for
> > > a
> > > > major version release to land these changes.
> > > > 20: Good point, ALOS doesn't need the same level of guarantee, and
> the
> > > > typically longer commit intervals would be problematic when reading
> only
> > > > "committed" records.
> > > >
> > > > I've been away for 5 days recovering from minor surgery, but I spent
> a
> > > > considerable amount of that time working through ideas for possible
> > > > solutions in my head. I think your suggestion of keeping ALOS as-is,
> but
> > > > buffering writes for EOS is the right path forwards, although I have
> a
> > > > solution that both expands on this, and provides for some more formal
> > > > guarantees.
> > > >
> > > > Essentially, adding support to KeyValueStores for "Transactions",
> with
> > > > clearly defined IsolationLevels. Using "Read Committed" when under
> EOS,
> > > and
> > > > "Read Uncommitted" under ALOS.
> > > >
> > > > The nice thing about this approach is that it gives us much more
> clearly
> > > > defined isolation behaviour that can be properly documented to ensure
> > > users
> > > > know what to expect.
> > > >
> > > > I'm still working out the kinks in the design, and will update the
> KIP
> > > when
> > > > I have something. The main struggle is trying to implement this
> without
> > > > making any major changes to the existing interfaces or breaking
> existing
> > > > implementations, because currently everything expects to operate
> directly
> > > > on a StateStore, and not a Transaction of that store. I think I'm
> getting
> > > > close, although sadly I won't be able to progress much until next
> week
> > > due
> > > > to some work commitments.
> > > >
> > > > Regards,
> > > > Nick
> > > >
> > > > On Thu, 1 Dec 2022 at 00:01, Colt McNealy <co...@littlehorse.io>
> wrote:
> > > >
> > > > > Nick,
> > > > >
> > > > > Thank you for the explanation, and also for the updated KIP. I am
> quite
> > > > > eager for this improvement to be released as it would greatly
> reduce
> > > the
> > > > > operational difficulties of EOS streams apps.
> > > > >
> > > > > Two questions:
> > > > >
> > > > > 10)
> > > > > >When reading records, we will use the
> > > > > WriteBatchWithIndex#getFromBatchAndDB
> > > > >  and WriteBatchWithIndex#newIteratorWithBase utilities in order to
> > > ensure
> > > > > that uncommitted writes are available to query.
> > > > > Why do extra work to enable the reading of uncommitted writes
> during
> > > IQ?
> > > > > Code complexity aside, reading uncommitted writes is, in my
> opinion, a
> > > > > minor flaw in EOS IQ; it would be very nice to have the guarantee
> that,
> > > > > with EOS, IQ only reads committed records. In order to avoid dirty
> > > reads,
> > > > > one currently must query a standby replica (but this still doesn't
> > > fully
> > > > > guarantee monotonic reads).
> > > > >
> > > > > 20) Is it also necessary to enable this optimization on ALOS
> stores?
> > > The
> > > > > motivation of KIP-844 was mainly to reduce the need to restore
> state
> > > from
> > > > > scratch on unclean EOS shutdowns; with ALOS it was acceptable to
> accept
> > > > > that there may have been uncommitted writes on disk. On a side
> note, if
> > > > you
> > > > > enable this type of store on ALOS processors, the community would
> > > > > 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 <
> nick.telford@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi everyone,
> > > > > >
> > > > > > I've drastically reduced the scope of this KIP to no longer
> include
> > > the
> > > > > > StateStore management of checkpointing. This can be added as a
> KIP
> > > > later
> > > > > on
> > > > > > to further optimize the consistency and performance of state
> stores.
> > > > > >
> > > > > > I've also added a section discussing some of the concerns 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 2022 at 16:36, Nick Telford <
> nick.telford@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > > Hi Colt,
> > > > > > >
> > > > > > > I didn't do any profiling, but the 844 implementation:
> > > > > > >
> > > > > > >    - Writes uncommitted records to a temporary RocksDB instance
> > > > > > >       - Since tombstones need to be flagged, all record values
> are
> > > > > > >       prefixed with a value/tombstone marker. This
> necessitates a
> > > > > memory
> > > > > > copy.
> > > > > > >    - On-commit, iterates all records in this temporary
> instance and
> > > > > > >    writes them to the main RocksDB store.
> > > > > > >    - While iterating, the value/tombstone marker needs to be
> parsed
> > > > and
> > > > > > >    the real value extracted. This necessitates another memory
> copy.
> > > > > > >
> > > > > > > My guess is that the cost of iterating the temporary RocksDB
> store
> > > is
> > > > > the
> > > > > > > major factor, with the 2 extra memory copies per-Record
> > > contributing
> > > > a
> > > > > > > significant amount too.
> > > > > > >
> > > > > > > Regards,
> > > > > > > Nick
> > > > > > >
> > > > > > > On Mon, 28 Nov 2022 at 16:12, Colt McNealy <
> colt@littlehorse.io>
> > > > > wrote:
> > > > > > >
> > > > > > >> Hi all,
> > > > > > >>
> > > > > > >> Out of curiosity, why does the performance of the store
> degrade so
> > > > > > >> significantly with the 844 implementation? I wouldn't be too
> > > > surprised
> > > > > > by
> > > > > > >> a
> > > > > > >> 50-60% drop (caused by each record being written twice), but
> 96%
> > > is
> > > > > > >> extreme.
> > > > > > >>
> > > > > > >> The only thing I can think of which could create such a
> bottleneck
> > > > > 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 Nick Telford <
> > > > nick.telford@gmail.com>
> > > > > > >> wrote:
> > > > > > >>
> > > > > > >> > Hi everyone,
> > > > > > >> >
> > > > > > >> > I've updated the KIP to resolve all the points that have
> been
> > > > raised
> > > > > > so
> > > > > > >> > far, with one exception: the ALOS default commit interval
> of 5
> > > > > minutes
> > > > > > >> is
> > > > > > >> > likely to cause WriteBatchWithIndex memory to grow too
> large.
> > > > > > >> >
> > > > > > >> > There's a couple of different things I can think of to solve
> > > this:
> > > > > > >> >
> > > > > > >> >    - We already have a memory/record limit in the KIP to
> prevent
> > > > OOM
> > > > > > >> >    errors. Should we choose a default value for these? My
> > > concern
> > > > > here
> > > > > > >> is
> > > > > > >> > that
> > > > > > >> >    anything we choose might seem rather arbitrary. We could
> > > change
> > > > > > >> >    its behaviour such that under ALOS, it only triggers the
> > > commit
> > > > > of
> > > > > > >> the
> > > > > > >> >    StateStore, but under EOS, it triggers a commit of the
> Kafka
> > > > > > >> > transaction.
> > > > > > >> >    - We could introduce a separate `checkpoint.interval.ms`
> to
> > > > > allow
> > > > > > >> ALOS
> > > > > > >> >    to commit the StateStores more frequently than the
> general
> > > > > > >> >    commit.interval.ms? My concern here is that the
> semantics of
> > > > > this
> > > > > > >> > config
> > > > > > >> >    would depend 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 <
> > > > nick.telford@gmail.com>
> > > > > > >> wrote:
> > > > > > >> >
> > > > > > >> > > Hi Alex,
> > > > > > >> > >
> > > > > > >> > > Thanks for the feedback.
> > > > > > >> > >
> > > > > > >> > > I've updated the discussion of OOM issues by describing
> how
> > > > we'll
> > > > > > >> handle
> > > > > > >> > > it. Here's the new text:
> > > > > > >> > >
> > > > > > >> > > To mitigate this, we will automatically force a Task
> commit if
> > > > the
> > > > > > >> total
> > > > > > >> > >> uncommitted records returned by
> > > > > > >> > >> StateStore#approximateNumUncommittedEntries()  exceeds a
> > > > > threshold,
> > > > > > >> > >> configured by max.uncommitted.state.entries.per.task; or
> the
> > > > > total
> > > > > > >> > >> memory used for buffering uncommitted records returned by
> > > > > > >> > >> StateStore#approximateNumUncommittedBytes() exceeds the
> > > > threshold
> > > > > > >> > >> configured by max.uncommitted.state.bytes.per.task. This
> will
> > > > > > roughly
> > > > > > >> > >> bound the memory required per-Task for buffering
> uncommitted
> > > > > > records,
> > > > > > >> > >> irrespective of the commit.interval.ms, and will
> effectively
> > > > > bound
> > > > > > >> the
> > > > > > >> > >> number of records that will need to be restored in the
> event
> > > > of a
> > > > > > >> > failure.
> > > > > > >> > >>
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > These limits will be checked in StreamTask#process and a
> > > > premature
> > > > > > >> commit
> > > > > > >> > >> will be requested via Task#requestCommit().
> > > > > > >> > >>
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > Note that these new methods provide default
> implementations
> > > that
> > > > > > >> ensure
> > > > > > >> > >> existing custom stores and non-transactional stores (e.g.
> > > > > > >> > >> InMemoryKeyValueStore) do not force any early commits.
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > I've chosen to have the StateStore expose approximations
> of
> > > its
> > > > > > buffer
> > > > > > >> > > size/count instead of opaquely requesting a commit in
> order to
> > > > > > >> delegate
> > > > > > >> > the
> > > > > > >> > > decision making to the Task itself. This enables Tasks to
> look
> > > > at
> > > > > > >> *all*
> > > > > > >> > of
> > > > > > >> > > their StateStores, and determine whether an early commit
> is
> > > > > > necessary.
> > > > > > >> > > Notably, it enables pre-Task thresholds, instead of
> per-Store,
> > > > > which
> > > > > > >> > > prevents Tasks with many StateStores from using much more
> > > memory
> > > > > > than
> > > > > > >> > Tasks
> > > > > > >> > > with one StateStore. This makes sense, since commits are
> done
> > > > > > by-Task,
> > > > > > >> > not
> > > > > > >> > > by-Store.
> > > > > > >> > >
> > > > > > >> > > Prizes* for anyone who can come up with a better name for
> the
> > > > new
> > > > > > >> config
> > > > > > >> > > properties!
> > > > > > >> > >
> > > > > > >> > > Thanks for pointing out the potential performance issues
> of
> > > > WBWI.
> > > > > > From
> > > > > > >> > the
> > > > > > >> > > benchmarks that user posted[1], it looks like WBWI still
> > > > performs
> > > > > > >> > > considerably better than individual puts, which is the
> > > existing
> > > > > > >> design,
> > > > > > >> > so
> > > > > > >> > > I'd actually expect a performance boost from WBWI, just
> not as
> > > > > great
> > > > > > >> as
> > > > > > >> > > we'd get from a plain WriteBatch. This does suggest that a
> > > good
> > > > > > >> > > optimization would be to use a regular WriteBatch for
> > > > restoration
> > > > > > (in
> > > > > > >> > > RocksDBStore#restoreBatch), since we know that those
> records
> > > > will
> > > > > > >> never
> > > > > > >> > be
> > > > > > >> > > queried before they're committed.
> > > > > > >> > >
> > > > > > >> > > 1:
> > > > > > >> >
> > > > > >
> > > >
> https://github.com/adamretter/rocksjava-write-methods-benchmark#results
> > > > > > >> > >
> > > > > > >> > > * Just kidding, no prizes, sadly.
> > > > > > >> > >
> > > > > > >> > > On Wed, 23 Nov 2022 at 12:28, Alexander Sorokoumov
> > > > > > >> > > <as...@confluent.io.invalid> wrote:
> > > > > > >> > >
> > > > > > >> > >> 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 to keep the current checkpointing
> as a
> > > > > > fallback
> > > > > > >> > >> mechanism. We want to keep existing users' workflows
> intact
> > > if
> > > > we
> > > > > > >> can. A
> > > > > > >> > >> non-intrusive way would be to add a separate StateStore
> > > method,
> > > > > > say,
> > > > > > >> > >> StateStore#managesCheckpointing(), that controls whether
> the
> > > > > state
> > > > > > >> store
> > > > > > >> > >> implementation owns checkpointing.
> > > > > > >> > >>
> > > > > > >> > >> I think that a solution to the transactional writes
> should
> > > > > address
> > > > > > >> the
> > > > > > >> > >> OOMEs. One possible way to address that is to wire
> > > StateStore's
> > > > > > >> commit
> > > > > > >> > >> request by adding, say, StateStore#commitNeeded that is
> > > checked
> > > > > in
> > > > > > >> > >> StreamTask#commitNeeded via the corresponding
> > > > > > ProcessorStateManager.
> > > > > > >> > With
> > > > > > >> > >> that change, RocksDBStore will have to track the current
> > > > > > transaction
> > > > > > >> > size
> > > > > > >> > >> and request a commit when the size goes over a
> (configurable)
> > > > > > >> threshold.
> > > > > > >> > >>
> > > > > > >> > >> AFAIU WriteBatchWithIndex might perform significantly
> slower
> > > > than
> > > > > > >> > non-txn
> > > > > > >> > >> puts as the batch size grows [1]. We should have a
> > > > configuration
> > > > > to
> > > > > > >> fall
> > > > > > >> > >> back to the current behavior (and/or disable txn stores
> for
> > > > ALOS)
> > > > > > >> unless
> > > > > > >> > >> the benchmarks show negligible overhead for longer
> commits /
> > > > > > >> > large-enough
> > > > > > >> > >> batch sizes.
> > > > > > >> > >>
> > > > > > >> > >> If you prefer to keep the KIP smaller, I would rather
> cut out
> > > > > > >> > >> state-store-managed checkpointing rather than proper OOMe
> > > > > handling
> > > > > > >> and
> > > > > > >> > >> being able to switch to non-txn behavior. The
> checkpointing
> > > 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 configuration parts as well as review and test
> the
> > > > > patch.
> > > > > > >> > >>
> > > > > > >> > >> Best,
> > > > > > >> > >> Alex
> > > > > > >> > >>
> > > > > > >> > >>
> > > > > > >> > >> 1. https://github.com/facebook/rocksdb/issues/608
> > > > > > >> > >>
> > > > > > >> > >> On Tue, Nov 22, 2022 at 6:31 PM Nick Telford <
> > > > > > nick.telford@gmail.com
> > > > > > >> >
> > > > > > >> > >> wrote:
> > > > > > >> > >>
> > > > > > >> > >> > Hi John,
> > > > > > >> > >> >
> > > > > > >> > >> > Thanks for the review and feedback!
> > > > > > >> > >> >
> > > > > > >> > >> > 1. Custom Stores: I've been mulling over this problem
> > > myself.
> > > > > As
> > > > > > it
> > > > > > >> > >> stands,
> > > > > > >> > >> > custom stores would essentially lose checkpointing
> with no
> > > > > > >> indication
> > > > > > >> > >> that
> > > > > > >> > >> > they're expected to make changes, besides a line in the
> > > > release
> > > > > > >> > notes. I
> > > > > > >> > >> > agree that the best solution would be to provide a
> default
> > > > that
> > > > > > >> > >> checkpoints
> > > > > > >> > >> > to a file. The one thing I would change is that the
> > > > > checkpointing
> > > > > > >> is
> > > > > > >> > to
> > > > > > >> > >> a
> > > > > > >> > >> > store-local file, instead of a per-Task file. This way
> the
> > > > > > >> StateStore
> > > > > > >> > >> still
> > > > > > >> > >> > technically owns its own checkpointing (via a default
> > > > > > >> implementation),
> > > > > > >> > >> and
> > > > > > >> > >> > the StateManager/Task execution engine doesn't need to
> know
> > > > > > >> anything
> > > > > > >> > >> about
> > > > > > >> > >> > checkpointing, which greatly simplifies some of the
> logic.
> > > > > > >> > >> >
> > > > > > >> > >> > 2. OOME errors: The main reasons why I didn't explore a
> > > > > solution
> > > > > > to
> > > > > > >> > >> this is
> > > > > > >> > >> > a) to keep this KIP as simple as possible, and b)
> because
> > > I'm
> > > > > not
> > > > > > >> > >> exactly
> > > > > > >> > >> > how to signal that a Task should commit prematurely.
> I'm
> > > > > > confident
> > > > > > >> > it's
> > > > > > >> > >> > possible, and I think it's worth adding a section on
> > > handling
> > > > > > this.
> > > > > > >> > >> Besides
> > > > > > >> > >> > my proposal to force an early commit once memory usage
> > > > reaches
> > > > > a
> > > > > > >> > >> threshold,
> > > > > > >> > >> > is there any other approach that you might suggest for
> > > > tackling
> > > > > > >> this
> > > > > > >> > >> > problem?
> > > > > > >> > >> >
> > > > > > >> > >> > 3. ALOS: I can add in an explicit paragraph, but my
> > > > assumption
> > > > > is
> > > > > > >> that
> > > > > > >> > >> > since transactional behaviour comes at little/no cost,
> that
> > > > it
> > > > > > >> should
> > > > > > >> > be
> > > > > > >> > >> > available by default on all stores, irrespective of the
> > > > > > processing
> > > > > > >> > mode.
> > > > > > >> > >> > While ALOS doesn't use transactions, the Task itself
> still
> > > > > > >> "commits",
> > > > > > >> > so
> > > > > > >> > >> > the behaviour should be correct under ALOS too. I'm not
> > > > > convinced
> > > > > > >> that
> > > > > > >> > >> it's
> > > > > > >> > >> > worth having both transactional/non-transactional
> stores
> > > > > > >> available, as
> > > > > > >> > >> it
> > > > > > >> > >> > would considerably increase the complexity of the
> codebase,
> > > > for
> > > > > > >> very
> > > > > > >> > >> little
> > > > > > >> > >> > benefit.
> > > > > > >> > >> >
> > > > > > >> > >> > 4. Method deprecation: Are you referring to
> > > > > > >> StateStore#getPosition()?
> > > > > > >> > >> As I
> > > > > > >> > >> > understand it, Position contains the position of the
> > > *source*
> > > > > > >> topics,
> > > > > > >> > >> > whereas the commit offsets would be the *changelog*
> > > offsets.
> > > > So
> > > > > > >> it's
> > > > > > >> > >> still
> > > > > > >> > >> > necessary to retain the Position data, as well as the
> > > > changelog
> > > > > > >> > offsets.
> > > > > > >> > >> > What I meant in the KIP is that Position offsets are
> > > > currently
> > > > > > >> stored
> > > > > > >> > >> in a
> > > > > > >> > >> > file, and since we can atomically store metadata along
> with
> > > > the
> > > > > > >> record
> > > > > > >> > >> > batch we commit to RocksDB, we can move our Position
> > > offsets
> > > > in
> > > > > > to
> > > > > > >> > this
> > > > > > >> > >> > metadata too, and gain the same transactional
> guarantees
> > > that
> > > > > we
> > > > > > >> will
> > > > > > >> > >> for
> > > > > > >> > >> > changelog offsets, ensuring that the Position offsets
> are
> > > > > > >> consistent
> > > > > > >> > >> with
> > > > > > >> > >> > the records that are read from the database.
> > > > > > >> > >> >
> > > > > > >> > >> > Regards,
> > > > > > >> > >> > Nick
> > > > > > >> > >> >
> > > > > > >> > >> > On Tue, 22 Nov 2022 at 16:25, John Roesler <
> > > > > vvcephei@apache.org>
> > > > > > >> > wrote:
> > > > > > >> > >> >
> > > > > > >> > >> > > Thanks for publishing this alternative, Nick!
> > > > > > >> > >> > >
> > > > > > >> > >> > > The benchmark you mentioned in the KIP-844 discussion
> > > seems
> > > > > > like
> > > > > > >> a
> > > > > > >> > >> > > compelling reason to revisit the built-in
> > > transactionality
> > > > > > >> > mechanism.
> > > > > > >> > >> I
> > > > > > >> > >> > > also appreciate you analysis, showing that for most
> use
> > > > > cases,
> > > > > > >> the
> > > > > > >> > >> write
> > > > > > >> > >> > > batch approach should be just fine.
> > > > > > >> > >> > >
> > > > > > >> > >> > > There are a couple of points that would hold me back
> from
> > > > > > >> approving
> > > > > > >> > >> this
> > > > > > >> > >> > > KIP right now:
> > > > > > >> > >> > >
> > > > > > >> > >> > > 1. Loss of coverage for custom stores.
> > > > > > >> > >> > > The fact that you can plug in a (relatively) simple
> > > > > > >> implementation
> > > > > > >> > of
> > > > > > >> > >> the
> > > > > > >> > >> > > XStateStore interfaces and automagically get a
> > > distributed
> > > > > > >> database
> > > > > > >> > >> out
> > > > > > >> > >> > of
> > > > > > >> > >> > > it is a significant benefit of Kafka Streams. I'd
> hate to
> > > > > lose
> > > > > > >> it,
> > > > > > >> > so
> > > > > > >> > >> it
> > > > > > >> > >> > > would be better to spend some time and come up with
> a way
> > > > to
> > > > > > >> > preserve
> > > > > > >> > >> > that
> > > > > > >> > >> > > property. For example, can we provide a default
> > > > > implementation
> > > > > > of
> > > > > > >> > >> > > `commit(..)` that re-implements the existing
> > > > checkpoint-file
> > > > > > >> > >> approach? Or
> > > > > > >> > >> > > perhaps add an `isTransactional()` flag to the state
> > > store
> > > > > > >> interface
> > > > > > >> > >> so
> > > > > > >> > >> > > that the runtime can decide whether to continue to
> manage
> > > > > > >> checkpoint
> > > > > > >> > >> > files
> > > > > > >> > >> > > vs delegating transactionality to the stores?
> > > > > > >> > >> > >
> > > > > > >> > >> > > 2. Guarding against OOME
> > > > > > >> > >> > > I appreciate your analysis, but I don't think it's
> > > > sufficient
> > > > > > to
> > > > > > >> say
> > > > > > >> > >> that
> > > > > > >> > >> > > we will solve the memory problem later if it becomes
> > > > > necessary.
> > > > > > >> The
> > > > > > >> > >> > > experience leading to that situation would be quite
> bad:
> > > > > > Imagine,
> > > > > > >> > you
> > > > > > >> > >> > > upgrade to AK 3.next, your tests pass, so you deploy
> to
> > > > > > >> production.
> > > > > > >> > >> That
> > > > > > >> > >> > > night, you get paged because your app is now crashing
> > > with
> > > > > > >> OOMEs. As
> > > > > > >> > >> with
> > > > > > >> > >> > > all OOMEs, you'll have a really hard time finding the
> > > root
> > > > > > cause,
> > > > > > >> > and
> > > > > > >> > >> > once
> > > > > > >> > >> > > you do, you won't have a clear path to resolve the
> issue.
> > > > You
> > > > > > >> could
> > > > > > >> > >> only
> > > > > > >> > >> > > tune down the commit interval and cache buffer size
> until
> > > > you
> > > > > > >> stop
> > > > > > >> > >> > getting
> > > > > > >> > >> > > crashes.
> > > > > > >> > >> > >
> > > > > > >> > >> > > FYI, I know of multiple cases where people run EOS
> with
> > > > much
> > > > > > >> larger
> > > > > > >> > >> > commit
> > > > > > >> > >> > > intervals to get better batching than the default,
> so I
> > > > don't
> > > > > > >> think
> > > > > > >> > >> this
> > > > > > >> > >> > > pathological case would be as rare as you suspect.
> > > > > > >> > >> > >
> > > > > > >> > >> > > Given that we already have the rudiments of an idea
> of
> > > what
> > > > > we
> > > > > > >> could
> > > > > > >> > >> do
> > > > > > >> > >> > to
> > > > > > >> > >> > > prevent this downside, we should take the time to
> design
> > > a
> > > > > > >> solution.
> > > > > > >> > >> We
> > > > > > >> > >> > owe
> > > > > > >> > >> > > it to our users to ensure that awesome new features
> don't
> > > > > come
> > > > > > >> with
> > > > > > >> > >> > bitter
> > > > > > >> > >> > > pills unless we can't avoid it.
> > > > > > >> > >> > >
> > > > > > >> > >> > > 3. ALOS mode.
> > > > > > >> > >> > > On the other hand, I didn't see an indication of how
> > > stores
> > > > > > will
> > > > > > >> be
> > > > > > >> > >> > > handled under ALOS (aka non-EOS) mode.
> Theoretically, the
> > > > > > >> > >> > transactionality
> > > > > > >> > >> > > of the store and the processing mode are orthogonal.
> A
> > > > > > >> transactional
> > > > > > >> > >> > store
> > > > > > >> > >> > > would serve ALOS just as well as a non-transactional
> one
> > > > (if
> > > > > > not
> > > > > > >> > >> better).
> > > > > > >> > >> > > Under ALOS, though, the default commit interval is
> five
> > > > > > minutes,
> > > > > > >> so
> > > > > > >> > >> the
> > > > > > >> > >> > > memory issue is far more pressing.
> > > > > > >> > >> > >
> > > > > > >> > >> > > As I see it, we have several options to resolve this
> > > point.
> > > > > We
> > > > > > >> could
> > > > > > >> > >> > > demonstrate that transactional stores work just fine
> for
> > > > ALOS
> > > > > > >> and we
> > > > > > >> > >> can
> > > > > > >> > >> > > therefore just swap over unconditionally. We could
> also
> > > > > disable
> > > > > > >> the
> > > > > > >> > >> > > transactional mechanism under ALOS so that stores
> operate
> > > > > just
> > > > > > >> the
> > > > > > >> > >> same
> > > > > > >> > >> > as
> > > > > > >> > >> > > they do today when run in ALOS mode. Finally, we
> could do
> > > > the
> > > > > > >> same
> > > > > > >> > as
> > > > > > >> > >> in
> > > > > > >> > >> > > KIP-844 and make transactional stores opt-in (it'd be
> > > > better
> > > > > to
> > > > > > >> > avoid
> > > > > > >> > >> the
> > > > > > >> > >> > > extra opt-in mechanism, but it's a good
> > > > get-out-of-jail-free
> > > > > > >> card).
> > > > > > >> > >> > >
> > > > > > >> > >> > > 4. (minor point) Deprecation of methods
> > > > > > >> > >> > >
> > > > > > >> > >> > > You mentioned that the new `commit` 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?
> > > > > > >> > >> > >
> > > > > > >> > >> > > Thanks again for the KIP! It's really nice that you
> and
> > > > Alex
> > > > > > will
> > > > > > >> > get
> > > > > > >> > >> the
> > > > > > >> > >> > > chance to collaborate on both directions so that we
> can
> > > get
> > > > > the
> > > > > > >> best
> > > > > > >> > >> > > outcome for Streams and its users.
> > > > > > >> > >> > >
> > > > > > >> > >> > > -John
> > > > > > >> > >> > >
> > > > > > >> > >> > >
> > > > > > >> > >> > > On 2022/11/21 15:02:15 Nick Telford wrote:
> > > > > > >> > >> > > > 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
> > > > > > >> > >> > > > <
> > > > > > >> > >> > >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-892%3A+Transactional+Semantics+for+StateStores
> > > > > > >> > >> > > >,
> > > > > > >> > >> > > > so that it can be discussed/reviewed separately
> from
> > > > > KIP-844.
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > Alex: I'm especially interested in what you think!
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > I have a nearly complete implementation of the
> changes
> > > > > > >> outlined in
> > > > > > >> > >> this
> > > > > > >> > >> > > > KIP, please let me know if you'd like me to push
> them
> > > for
> > > > > > >> review
> > > > > > >> > in
> > > > > > >> > >> > > advance
> > > > > > >> > >> > > > of a vote.
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > Regards,
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > Nick
> > > > > > >> > >> > > >
> > > > > > >> > >> > >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
>