Hey Paolo,

When you use the subscribe() API, you are using Kafka for group
coordination. All members with the same group.id will balance the
partitions of subscribed topics between themselves. When you use assign(),
no such balancing takes place but you can still using Kafka to store
offsets. However, we have a safety check in place such that if there is an
active consumer group (i.e. one using group coordination with the
subscribe() API), then offset commits from consumers that are not part of
that group are rejected. So in your situation, if C1 and C2 are active at
the same time, only C2 will be able to commit offsets. If you look in C1's
log, you should see some errors from the auto offset commits. I believe
this explains the three cases mentioned above.

-Jason

On Thu, Jul 27, 2017 at 3:59 AM, Paolo Patierno <ppatie...@live.com> wrote:

> Hi devs,
>
> I have one simple topic named "test" with only one partition. then ...
>
> I have a consumer C1 using assign() for having assigned partition 0 of
> topic "test" and this consumer has group.id = "mygroup"
>
> I have another consumer C2 using subscribe() for having assigned
> partitions from topic "test" (so it will have assigned partition 0) and
> this consumer has group.id = "mygroup" (the same as above)
>
>
> USE CASE 1
>
> Both consumers have auto commit enabled
>
> Reading messages from the __consumer_offset I always see only ONE message
> of this type
>
>
> [mygroup,test,0]::[OffsetMetadata[7,NO_METADATA],CommitTime
> 1501149394800,ExpirationTime 1501235794800]
>
>
> So even if both consumers receives messages from "test" and have auto
> commit enabled, only one offset commit message is there. Is that normal ?
>
>
> USE CASE 2
>
> Consumer C1 has auto commit enabled
>
> Consumer C2 has auto commit disabled and it doesn't execute any offset
> commit manually
>
> Both consumers get messages from "test" but, there are no offset commit
> messages into __consumer_offset. Is that normal ?
>
> I expect that the C1 with auto commit enabled sends the offset commit
> message but it doesn't happen. It seems that the C2 settings about auto
> commit disabled influences the C1 settings on that.
>
>
> USE CASE 3 (opposite of 2)
>
> Consumer C1 has auto commit disabled
>
> Consumer C2 has auto commit enabled
>
> Both consumers get messages from "test" and I can see offset commit
> messages into __consumer_offset (from C2).
>
>
> [mygroup,test,0]::[OffsetMetadata[7,NO_METADATA],CommitTime
> 1501149394800,ExpirationTime 1501235794800]
>
>
> In this use case, the auto commit disabled by C1 doesn't influence the
> enabled from C2. It happened in the use case 2 (in the opposite direction).
>
>
> Conclusion ... it's possible that there is something that I didn't
> understand about this interaction between consumers in the same group but
> asking for partitions in two different way (assign vs subscribe).
>
>
> Thanks,
>
> Paolo
>
>
> Paolo Patierno
> Senior Software Engineer (IoT) @ Red Hat
> Microsoft MVP on Windows Embedded & IoT
> Microsoft Azure Advisor
>
> Twitter : @ppatierno<http://twitter.com/ppatierno>
> Linkedin : paolopatierno<http://it.linkedin.com/in/paolopatierno>
> Blog : DevExperience<http://paolopatierno.wordpress.com/>
>

Reply via email to