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

2024-04-17 Thread Nick Telford
Hi Walker,

Feel free to ask away, either on the mailing list of the Confluent
Community Slack, where I hang out :-)

The implementation is *mostly* complete, although it needs some polishing.
It's worth noting that KIP-1035 is a hard prerequisite for this.

Regards,
Nick


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

2023-10-29 Thread Colt McNealy
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 
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  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  .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 
> > >> 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  > >> .invalid>
> > >>> wrote:
> > >>>
> >  Hi all,
> > 
> >  I think I liked your suggestion of allowing EOS with
> 

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

2023-10-29 Thread Guozhang Wang
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  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 
> > 
> > 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 
> >> 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  >> .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
> 

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

2023-10-19 Thread Bruno Cadonna

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 
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 
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 
.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 
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 

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

2023-10-19 Thread Lucas Brutschy
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  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 
> 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 
> > 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  > .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 
> > > > wrote:
> > > > >
> > > > > Hi Guozhang,
> > > > >
> > > > > The KIP as it stands introduces a new configuration,
> > > > > default.state.isolation.level, which is 

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

2023-10-18 Thread Nick Telford
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 
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 
> 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  .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 
> > > 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 

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

2023-10-18 Thread Lucas Brutschy
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  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 
> 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 
> > 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. 

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

2023-10-17 Thread Nick Telford
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 
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 
> 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.

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

2023-10-16 Thread Lucas Brutschy
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  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 
> 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 

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

2023-10-16 Thread Nick Telford
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 
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 
> 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  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 

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

2023-10-13 Thread Guozhang Wang
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  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  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  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 

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

2023-10-13 Thread Nick Telford
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  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  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:

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

2023-10-13 Thread Bruno Cadonna

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  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 

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

2023-10-13 Thread Nick Telford
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  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. 

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

2023-10-13 Thread Bruno Cadonna

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 

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

2023-10-13 Thread Nick Telford
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 

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

2023-10-12 Thread Sophie Blee-Goldman
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  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  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 
> > 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 

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

2023-09-19 Thread Nick Telford
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  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 
> 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
> >> 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 implements ReadOnlyKeyValueStore >> V>, AutoCloseable {}
> >>
> >> interface ReadOnlyKeyValueStore {
> >>  ...
> >>  ReadOnlyKeyValueView readOnlyView(IsolationLevel
> isolationLevel);
> >> }
> >>
> >> But this would be a breaking change, as 

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

2023-09-19 Thread Bruno Cadonna
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  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
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 implements ReadOnlyKeyValueStore, AutoCloseable {}

interface ReadOnlyKeyValueStore {
 ...
 ReadOnlyKeyValueView 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 

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

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

On Mon, 18 Sept 2023 at 17:24, Nick Telford  wrote:

> Hi 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
> 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 implements ReadOnlyKeyValueStore V>, AutoCloseable {}
>
> interface ReadOnlyKeyValueStore {
> ...
> ReadOnlyKeyValueView 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.
> 

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

2023-09-18 Thread Nick Telford
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
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 implements ReadOnlyKeyValueStore, AutoCloseable {}

interface ReadOnlyKeyValueStore {
...
ReadOnlyKeyValueView 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: 

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

2023-09-18 Thread Lucas Brutschy
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  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  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 
> > 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 

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

2023-09-17 Thread Colt McNealy
> 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  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 
> 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  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 

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

2023-09-13 Thread Nick Telford
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  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  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 

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

2023-09-13 Thread Nick Telford
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  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
> > 

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

2023-09-13 Thread Bruno Cadonna

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  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.


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

2023-09-13 Thread Nick Telford
Bruno,

Thinking about 3a. in addition to adding the IsolationLevel to
QueryStoreParameters and Query, what about also adding a method like
"ReadOnlyKeyValueStore 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  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  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 

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

2023-09-13 Thread Nick Telford
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  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 

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

2023-09-13 Thread Bruno Cadonna

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

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

2023-09-11 Thread Colt McNealy
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  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  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: 

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

2023-09-11 Thread Nick Telford
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  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 

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

2023-09-10 Thread Colt McNealy
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  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 

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

2023-08-24 Thread Nick Telford
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 

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

2023-08-14 Thread Bruno Cadonna

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  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

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

2023-07-21 Thread Nick Telford
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  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  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  wrote:
>>
>>> Hi Nick,
>>>
>>> Thanks for the insights! Very interesting!
>>>
>>> As far as I understand, you want to atomically update the state store
>>> from the 

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

2023-07-21 Thread Nick Telford
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  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  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
>> 

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

2023-07-03 Thread Nick Telford
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  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 

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

2023-07-03 Thread Bruno Cadonna

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  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

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

2023-07-01 Thread Nick Telford
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  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, 

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

2023-06-22 Thread Nick Telford
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  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 

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

2023-06-22 Thread Bruno Cadonna

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

2023-06-21 Thread John Roesler
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  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

2023-06-21 Thread Nick Telford
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  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

2023-06-21 Thread John Roesler

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

2023-06-21 Thread Nick Telford
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

2023-06-21 Thread Bruno Cadonna

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,  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,  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 

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

2023-06-20 Thread Nick Telford
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,  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,  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,  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 
> >> 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,
> >> 

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

2023-06-20 Thread John Roesler

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,  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,  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 

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 

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

2023-06-20 Thread Nick Telford
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,  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,  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 
> 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 

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

2023-06-20 Thread John Roesler

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,  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  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: [DISCUSS] KIP-892: Transactional Semantics for StateStores

2023-06-20 Thread Nick Telford
> 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,  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  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 

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

2023-06-20 Thread John Roesler

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  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 

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

2023-06-20 Thread Nick Telford
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

2023-06-20 Thread Nick Telford
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  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 

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

2023-06-20 Thread John Roesler

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 

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

2023-06-20 Thread Nick Telford
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 

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

2023-06-20 Thread Bruno Cadonna

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  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 

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

2023-05-15 Thread Nick Telford
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  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  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 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 

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

2023-05-03 Thread Nick Telford
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  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 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 

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

2023-05-02 Thread Bruno Cadonna

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 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  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  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 

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

2023-04-30 Thread Colt McNealy
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  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  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  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
> >> 

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

2023-04-27 Thread Nick Telford
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  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  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 
>> 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 

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

2023-04-19 Thread Nick Telford
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  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 
> 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  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 

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

2023-04-19 Thread Colt McNealy
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  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  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 
> > 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 

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

2023-04-19 Thread Nick Telford
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  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 
> 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 
> 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 

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

2023-04-18 Thread Colt McNealy
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  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  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  .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 

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

2023-04-18 Thread Nick Telford
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  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 
> 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 
>> wrote:
>> >
>> > Hi everyone,
>> >
>> > I've updated the KIP with a more detailed design, which reflects the
>> > implementation I've been working on:
>> >
>> 

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

2023-01-03 Thread Nick Telford
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 
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 
> 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  wrote:
> >
> > > Nick,
> > >
> > > Thank you for the reply; that makes sense. I was hoping that, since
> reading
> > > 

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

2023-01-02 Thread Lucas Brutschy
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  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  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 
> > 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  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
> > > > 

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

2022-12-22 Thread Nick Telford
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  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 
> 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  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 
> > > 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 

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

2022-12-06 Thread Colt McNealy
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  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  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 
> > 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 
> > 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
> > 

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

2022-12-06 Thread Nick Telford
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  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 
> 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 
> 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 
> 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 

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

2022-11-30 Thread Colt McNealy
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 
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  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  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 
> >> 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 
> >> wrote:
> >> >
> >> > > Hi Alex,
> >> > >
> >> > > Thanks for the feedback.
> >> > >
> >> > > I've updated the 

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

2022-11-30 Thread Nick Telford
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  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  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 
>> 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 
>> 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, 

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

2022-11-28 Thread Nick Telford
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  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 
> 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 
> 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 

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

2022-11-28 Thread Colt McNealy
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  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  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
> >  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

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

2022-11-28 Thread Nick Telford
For now, I've settled on choosing an arbitrary default memory limit of
64MiB per-Task for buffering uncommitted records. I noticed that Kafka
Streams already provides some arbitrary default configuration of RocksDB
memory settings (i.e. memtable size etc.), and that many users will already
be explicitly configuring this for their purposes.

I think a further optimization for ALOS to only commit the StateStores when
exceeding this limit is reasonable, to preserve the user's desired
commit.interval.ms as much as possible.

On Mon, 28 Nov 2022 at 15:55, Nick Telford  wrote:

> 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  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
>>  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

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

2022-11-28 Thread Nick Telford
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  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
>  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 

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

2022-11-23 Thread Nick Telford
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
 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 
> 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 

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

2022-11-23 Thread Alexander Sorokoumov
Hey Nick,

Thank you for the KIP! With such a significant performance degradation in
the secondary store approach, we should definitely consider
WriteBatchWithIndex. I also like encapsulating checkpointing inside the
default state store implementation to improve performance.

+1 to John's comment 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  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  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 

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

2022-11-22 Thread Nick Telford
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  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, 

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

2022-11-22 Thread John Roesler
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
> ,
> 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
>