This is such a great proposal. Parallel/Cooperative consumption is a long
missing pattern in Apache Kafka and it is a viable solution for the common
head-of-line-blocking problem. Developers have long attempted to solve this
with bespoke consumer proxies that state manage the inflight payloads, so
this would definitely be helpful to have in Apache Kafka.

On Mon, 22 May 2023 at 05:09, Andrew Otto <o...@wikimedia.org> wrote:

> Wow looks very cool.
>
> One Q.
>
> In the Future Work session:
> > The concept can be extended to give key-based ordering so that partial
> ordering and fine-grained sharing can be achieved at the same time.
>
> I think this will be a pretty important feature needed to make shared
> consumer groups useful.  Perhaps it would be worth considering how this
> could be implemented and mentioning it in the KIP?
>
>
>
>
>
>
> On Mon, May 22, 2023 at 6:20 AM Stanislav Kozlovski
> <stanis...@confluent.io.invalid> wrote:
>
> > Hey Andrew!
> >
> > Kudos on the proposal. It is greatly written - a joy to read. It is
> > definitely an interesting solution to the queueing problem - I would not
> > have guessed we could solve it like this. Thank you for working on this.
> >
> > Happy to get the discussion started - I have a few comments/questions on
> > first read:
> >
> > 1. Tiered Storage
> >
> > I notice no mention of Tiered Storage (KIP-405). Does that complicate the
> > design, especially when fetching historical data? It would be good to
> have
> > at least one sentence mentioning it, even if it doesn't impact it. Right
> > now I'm unsure if it was considered.
> >
> > 2. SSO initialized to the latest offset
> >
> > > "By default, the SSO for each share-partition is initialized to the
> > latest offset for the corresponding topic-partitions."
> >
> > Have we considered allowing this to be configurable to latest/earliest?
> > This would be consistent with the auto.offset.reset config of vanilla
> > consumer groups.
> > Thinking from a user's perspective, it sounds valid to want to start from
> > the start of a topic when starting a share group. Historical processing
> > comes to mind
> >
> > 3. Durable Storage
> >
> > The KIP mentions that "The cluster records this information durably",
> which
> > implies that it saves it somewhere. Does the ShareCoordinator have its
> own
> > topic? Would it be compacted?
> >
> > In particular, I am interested in what such a topic's retention would be
> > like. The vanilla consumer offsets topic has some special retention
> > semantics (KIP-211) where we start counting the retention after the
> > consumer group becomes empty (inactive) - the default being 7 days. Need
> to
> > make sure the retention here isn't too short either, as the offsets topic
> > originally had 24 hours of retention and that proved problematic.
> >
> > In general, some extra detail about the persistence would be greatly
> > appreciated!
> >
> > 4. Batch Acknowledgement
> >
> > > "In the situation where some records in a batch have been released or
> > rejected separately, subsequent fetches of those records are more likely
> to
> > have gaps."
> >
> > Can we expand a bit more on this edge case? I am interested in learning
> > what gets returned on subsequent fetch requests.
> > In particular, - how does this work with compression? As far as I
> remember,
> > we can compress the whole batch there, which might make individual record
> > filtering tricky.
> >
> > 5. Member Management
> >
> > How is consumer group member management handled? I didn't see any
> specific
> > mention - is it the same as a vanilla group?
> > In particular - how will bad consumers be handled?
> >
> > I guess I see two cases:
> > 1. bad consumer that doesn't even heartbeat
> > 2. bad consumer that heartbeats well but for some reason every message
> > processing times out. e.g imagine it was network partitioned from some
> > third-party system that is a critical part of its message processing loop
> >
> > One evident problem I can foresee in production systems is one (or a few)
> > slow consumer applications bringing the SSO/SEO advancement down to a
> > crawl.
> > Imagine an example where the same consumer app always hits the timeout
> > limit - what would the behavior be in such a case? Do we keep that
> consumer
> > app indefinitely (if so, do we run the risk of having it invalidate
> > completely valid messages)? Are there any equivalents to the consumer
> group
> > rebalances which fence off such bad consumers?
> >
> > 6. Processing Semantics (exactly once)
> >
> > > The delivery counts are only maintained approximately and the Acquired
> > state is not persisted.
> >
> > Does this introduce the risk of zombie consumers on
> share-partition-leader
> > failure? i.e restarting and giving another consumer the acquired state
> for
> > the same record
> >
> > I notice that the KIP says:
> > > Finally, this KIP does not include support for acknowledging delivery
> > using transactions for exactly-once semantics.
> > at the very end. It would be helpful to address this earlier in the
> > example, as one of the key points. And it would be good to be clearer on
> > what the processing semantics are. They seem to be *at-least-once* to me.
> >
> >
> > 7. nit: Acronyms
> >
> > I feel like SSO and SEO may be bad acronyms. Googling "Kafka SEO" is
> bound
> > to return weird results.
> > What do we think about the tradeoff of using more-unique acronyms (like
> > SGEO, SSGO) at the expense of one extra letter?
> >
> > Again - thanks for working on this! I think it's a great initiative. I'm
> > excited to see us perfect this proposal and enable a brand new use case
> in
> > Kafka!
> >
> > Best,
> > Stanislav
> >
> >
> >
> > On Mon, May 15, 2023 at 2:55 PM Andrew Schofield <
> > andrew_schofi...@live.com>
> > wrote:
> >
> > > Hi,
> > > I would like to start a discussion thread on KIP-932: Queues for Kafka.
> > > This KIP proposes an alternative to consumer groups to enable
> cooperative
> > > consumption by consumers without partition assignment. You end up with
> > > queue semantics on top of regular Kafka topics, with per-message
> > > acknowledgement and automatic handling of messages which repeatedly
> fail
> > to
> > > be processed.
> > >
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-932%3A+Queues+for+Kafka
> > >
> > > Please take a look and let me know what you think.
> > >
> > > Thanks.
> > > Andrew
> >
> >
> >
> > --
> > Best,
> > Stanislav
> >
>

Reply via email to