Hi, Andrew,

Thanks for the KIP. A few comments below.

10. ShareFetchResponse: To consume transactional data, currently
FetchResponse includes the AbortedTransactions fields for the client to
properly skip aborted records. ShareFetchResponse doesn't include that. How
do we prevent the consumer from reading aborted records in a share group?

11. "adding "share"  to the existing group types of "consumer"  and
"classic" "
What's the "classic" type?

12. bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-name
group --entity-name G1 --alter --add-config group.type=share
So, one could change the group type? What happens to the states associated
with the group (members, epoch, offsets, etc)?

13. Why is Server Assignor at member level, instead of group level?

14. Member.metadata: How is that being used? It isn't exposed to the client.

15. What public interface does SimpleAssignor implement?

16. "This means that existing members will have to rejoin the share group
following a coordinator failover."
When an existing member issues a ShareGroupHeartbeatRequest to the new
coordinator, does the coordinator return UNKNOWN_MEMBER_ID and a new
memberId?

17. auto.offset.reset has the option to throw an exception to the client if
the current offset does not exist any more on the server (e.g. due to
retention). Should group.share.auto.offset.reset support  that too?

18. SHARE_CHECKPOINT and SHARE_DELTA records:
18.1 When does the coordinator write them?
18.2 If the client commits the acknowledgement successfully, could
the acknowledgement be lost on the broker if the coordinator fails over?

19. In the current consumer model, coordinator failover doesn't cause
duplicate records in consumers. In the share group model, I guess this is
no longer true since we are not persisting the acquired state?

20. "The calls to KafkaShareConsumer.acknowledge(ConsumerRecord,
AcknowledgeType) must be issued in the order in which the records appear in
the ConsumerRecords object, which will be in order of increasing offset for
each share-partition."
20.1  What happens if the acknowledge() call doesn't follow this? Does
the caller get an exception?
20.2 The example with Acknowledge 119. It seems the acknowledgement is out
of order since records at offset 111-118 haven't been acknowledged?

21. "Indeed, these message sets are not returned to consumer". Are we
excluding those control records for non-shared consumers too?

22. The design doesn't seem to support fetching from the followers. This
might be ok, but it will be useful to explicitly mention this.

23. Examples with control records for SHARE_DELTA:
23.1 Some of the state changes contain cumulative state instead of delta.
For example, "record 110 (available, delivery count 1), records 111-118
acquired, record 119 acknowledged" for "Acknowledge 119".
23.2 SHARE_DELTA sometimes include available records with DeliveryCount of
0. But we don't do that for every record. What's the convention?
    {
      "BaseOffset": 111,
      "LastOffset": 118,
      "State": 0 (Available),
      "DeliveryCount": 0
    }

24. "when a broker becomes the leader of a share-partition, it must read
the most recent SHARE_CHECKPOINT": How does a broker find this efficiently
on restart?

25. AcknowledgeCommitCallback: How would an application use it? It doesn't
indicate which record's acknowledgement has failed.

26. AlterShareGroupOffsetsResult alterShareGroupOffsets(String groupId,
Map<TopicPartition, OffsetAndMetadata> offsets): How is the metadata used?
It doesn't seem there is an API to use it in either the client application
or the broker.

27. It would be useful to add a section on downgradability since the KIP
changes the record format in the internal offset topic.

Thanks,

Jun












On Wed, Oct 11, 2023 at 8:25 AM Andrew Schofield <
andrew_schofield_j...@outlook.com> wrote:

> Hi Jack,
> Thanks for your comments.
>
> I have added a new section on Log Retention which describes the behaviour
> of the SPSO as the LSO advances. That makes total sense
> and was an omission from the KIP.
>
> I have added the other ideas as potential future work. I do like the idea
> of having the SPSO influence the advancements of the LSO
> for topics which are primarily being using with share groups.
>
> I have published an updated version of the KIP.
>
> Thanks,
> Andrew
>
> > On 4 Oct 2023, at 10:09, Jack Vanlightly <vanligh...@apache.org> wrote:
> >
> > I would like to see more explicit discussion of topic retention and
> share groups. There are a few options here from simple to more
> sophisticated. There are also topic-level and share-group level options.
> >
> > The simple thing would be to ensure that the SPSO of each share group is
> bounded by the Log Start Offset (LSO) of each partition which itself is
> managed by the retention policy. This is a topic-level control which
> applies to all share-groups. I would say that this shared retention is the
> largest drawback of modeling queues on shared logs and this is worth noting.
> >
> > More sophisticated approaches can be to allow the LSO to advance not
> (only) by retention policy but by the advancement of the lowest SPSO. This
> can keep the amount of data lower by garbage collecting messages that have
> been acknowledged by all share groups. Some people may like that behaviour
> on those topics where share groups are the only consumption model and no
> replay is needed.
> >
> > There are per-share-group possibilities such as share-group TTLs where
> messages can be archived on a per share group basis.
> >
> > Thanks
> > Jack
>
>

Reply via email to