confluentinc / librdkafka

The Apache Kafka C/C++ library
Other
214 stars 3.14k forks source link

Assign operation of consumer requires a group id #3261

Open ctron opened 3 years ago

ctron commented 3 years ago

Read the FAQ first: https://github.com/edenhill/librdkafka/wiki/FAQ

Description

librdkafka requires a `group.id` for the `assign` operation of a consumer, while the Java client does not. https://github.com/edenhill/librdkafka/blob/f5c1045dc898abd1bd0b97da90b4717876f5d251/src/rdkafka_subscription.c#L110-L113 Java: https://github.com/apache/kafka/blob/1adb580faae89e0a298c0cb4ba08b238d91f9d03/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L1101 Compared to "subscribe": https://github.com/apache/kafka/blob/1adb580faae89e0a298c0cb4ba08b238d91f9d03/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L1022 How to reproduce ================ * Set up a consumer without a `group.id` * Call the `assign` operation with a valid topic/partition list Result: `Subscription error: Local: Unknown group` **IMPORTANT**: Always try to reproduce the issue on the latest released version (see https://github.com/edenhill/librdkafka/releases), if it can't be reproduced on the latest version the issue has been fixed. Checklist ========= **IMPORTANT**: We will close issues where the checklist has not been completed. Please provide the following information: - [x] librdkafka version (release number or git tag): `1.5.0` - [x] Apache Kafka version: `2.6.2` - [x] librdkafka client configuration: `("bootstrap.servers", &cfg.bootstrap_servers)("enable.partition.eof", "false")("session.timeout.ms", "6000")("enable.auto.commit", "false")` - [x] Operating system: `Fedora 33` - [x] Provide logs (with `debug=..` as necessary) from librdkafka: I don't see any log statements in the code that affects the issue - [x] Provide broker log excerpts: it is a local issue - [x] Critical issue: not critical
edenhill commented 3 years ago

Yes, this is a known limitation that will not be fixed. You will need a group.id configured to be able to commit or retrieve committed offsets, and if not the group.id is not really used so you can just generate a unique string (e.g., uuid).

mhowlett commented 3 years ago

This was a limitation of the Java client until recently: https://github.com/apache/kafka/pull/7943

I would say this is worth changing in librdkafka because it's pretty high visibility and does shift perceptions negatively because it doesn't behave in the way people expect. "I don't want to use a consumer group - WTF!" was my reaction to this when I was new to librdkafka. I may not have considered committing offsets, can't recall, but i may not have needed it and the more natural time / time with less friction / most impactful time to learn about that would have been at the point where I tried to do it.

It also doesn't look hard to change in the code.

edenhill commented 3 years ago

If we allow group.id to not be set it means commits will not work, so should we then automatically disable enable.auto.commit? That will trip people up since we would disable something they might rely on working. So instead we could require enable.auto.commit to be configured to false when group.id is not configured, but then we're pretty much back to where we started with users needing to understand how these things fit together.

I'm not super convinced this niche use-case needs to be solved.

mhowlett commented 3 years ago

I do think that would be better (forcing enble.auto.commit to be false) - because the error would be informative - would teach me something i didn't know rather than leave me wondering. In the Java case, it appears they just disable auto commit (seems ok - i don't like defaults conditionally changing, but we do it elsewhere - you are right though it is potentially confusing). What would be even imo better is if enable.auto.commit defaulted to false (but that can't really be changed at this point). Especially as I don't like auto commit because it gives neither at-least-once or at-most-once semantics.

edenhill commented 3 years ago

Okay, so:

edenhill commented 3 years ago

I guess we should also prohibit RD_KAFKA_OFFSET_STORED as starting offset in seek(), assign(), etc when group.id is not set.

gfeyer commented 1 year ago

Hi guys. What's the status of this issue? Was it fixed by any chance? Having the same problem when using assign()

fabionaspolini commented 1 year ago

Hi guys. Is there any update on this issue?

xxchan commented 5 months ago

I think this behavior is defined in KIP-289: Improve the default group id behavior in KafkaConsumer (Instead of https://github.com/apache/kafka/pull/7943, which just changed null to None IMO)

INTRODUCTION.md mentioned KIP-289 is implemented, but it seems not the case.

xxchan commented 5 months ago

BTW, if you don't want consumer groups being unnecessarily created, I think you can just disable enable.auto.commit, and specify an arbitrary group.id. In this case it doesn't seem to have any effect.