Hi,

Any comments/updates? I am not sure the next steps if no one has any further 
comments.

Thanks, CH

-----Original Message-----
From: ChienHsing Wu <chien...@opentext.com> 
Sent: Tuesday, November 20, 2018 2:46 PM
To: dev@kafka.apache.org
Subject: RE: [EXTERNAL] - Re: [DISCUSS] KIP-387: Fair Message Consumption 
Across Partitions in KafkaConsumer

Hi Matt,

Thanks for the feedback. 

The issue with the current design is that it stays on the previous partition 
even if the last poll call consumes the max.poll.records; it will consume all 
records in that partition available at the consumer side to serve multiple poll 
calls before moving to the next partition. 

Introducing another threshold at partition level will decrease the number of 
records consumed in one partition within one poll call but will still use that 
same partition as the starting one in the next poll call. 

The same effect can be achieved by setting max.poll.records to 100 I believe. 
The main difference is that the client will need to make more poll calls when 
that value is set to 100, and because of the non-blocking nature I believe the 
cost of extra poll calls are not significant. 

Further thoughts?

Thanks, CH

-----Original Message-----
From: Matt Farmer <m...@frmr.me>
Sent: Monday, November 19, 2018 9:32 PM
To: dev@kafka.apache.org
Subject: [EXTERNAL] - Re: [DISCUSS] KIP-387: Fair Message Consumption Across 
Partitions in KafkaConsumer

Hi there,

Thanks for the KIP.

We’ve run into issues with this at Mailchimp so something to address consuming 
behavior would save us from having to always ensure we’re running enough 
consumers that each consumer has only one partition (which is our usual MO).

I wonder though if it would be simpler and more powerful to define the maximum 
number of records the consumer should pull from one partition before pulling 
some records from another?

So if you set max.poll.records to 500 and then some new setting, 
max.poll.records.per.partition, to 100 then the Consumer would switch what 
partition it reads from every 100 records - looping back around to the first 
partition that had records if there aren’t 5 or more partitions with records.

What do you think?

On Mon, Nov 19, 2018 at 9:11 AM ChienHsing Wu <chien...@opentext.com> wrote:

> Hi, could anyone please review this KIP?
>
> Thanks, ChienHsing
>
> From: ChienHsing Wu
> Sent: Friday, November 09, 2018 1:10 PM
> To: dev@kafka.apache.org
> Subject: RE: [DISCUSS] KIP-387: Fair Message Consumption Across 
> Partitions in KafkaConsumer
>
> Just to check: Will anyone review this? It's been silent for a week...
> Thanks, ChienHsing
>
> From: ChienHsing Wu
> Sent: Monday, November 05, 2018 4:18 PM
> To: 'dev@kafka.apache.org' <dev@kafka.apache.org<mailto:
> dev@kafka.apache.org>>
> Subject: [DISCUSS] KIP-387: Fair Message Consumption Across Partitions 
> in KafkaConsumer
>
> Hi I just put together the KIP page as requested. This email is to 
> start the discussion thread.
>
> KIP: KIP-387: Fair Message Consumption Across Partitions in 
> KafkaConsumer< 
> https://urldefense.proofpoint.com/v2/url?u=https-3A__cwiki.apache.org_
> confluence_display_KAFKA_KIP-2D387-253A-2BFair-2BMessage-2BConsumption
> -2BAcross-2BPartitions-2Bin-2BKafkaConsumer&d=DwIFaQ&c=ZgVRmm3mf2P1-XD
> AyDsu4A&r=Az03wMrbL9ToLW0OFyo3wo3985rhAKPMLmmg6RA3V7I&m=HhHPjfcrk8XioE
> m16n75UIKYwi8c8YrzVrp5tBK7LX8&s=gBGG4GvzPu-xhQ-uUqlq30U-bzwcKZ_lNP1bF5
> 49_KU&e=
> >
> Pull Request: 
> https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache
> _kafka_pull_5838&d=DwIFaQ&c=ZgVRmm3mf2P1-XDAyDsu4A&r=Az03wMrbL9ToLW0OF
> yo3wo3985rhAKPMLmmg6RA3V7I&m=HhHPjfcrk8XioEm16n75UIKYwi8c8YrzVrp5tBK7L
> X8&s=cJ2JGXAUQx4ymtMv_MLtGq7QiUJV3xBzKcS_Nwla08A&e=
> Jira: 
> https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org
> _jira_browse_KAFKA-2D3932&d=DwIFaQ&c=ZgVRmm3mf2P1-XDAyDsu4A&r=Az03wMrb
> L9ToLW0OFyo3wo3985rhAKPMLmmg6RA3V7I&m=HhHPjfcrk8XioEm16n75UIKYwi8c8Yrz
> Vrp5tBK7LX8&s=TfIIF2Ui9YEVxxwAbko0j-fT_mMVHf5Yywapc0w8eEA&e=
>
> Thanks, CH
>

Reply via email to