[ 
https://issues.apache.org/jira/browse/KAFKA-10091?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17127720#comment-17127720
 ] 

Guozhang Wang commented on KAFKA-10091:
---------------------------------------

Thanks John. I think you're concerning about some issues with consumer itself, 
i.e. there are data in the broker but consumer keeps skipping them because 
other partitions just have too big traffic and hence those partitions are 
"starved". I think current consumer's fetch mechanism based on session could 
guarantee that we would not starve a partition, but on the other hand I also 
agrees that as a config, the max.wait.time is actually hard to set by users 
would do need to guarantee time ordering.

So may be what we should discuss here is, whether the existing time-amount 
based config can be just simplified to a boolean knob: i.e. we would guarantee 
that task is idling until there are no empty buffered partitions, no matter how 
long we may need to wait. And the metrics we've already added would tell users 
that how much latency they may trade for this guarantee in practice.

> Improve task idling
> -------------------
>
>                 Key: KAFKA-10091
>                 URL: https://issues.apache.org/jira/browse/KAFKA-10091
>             Project: Kafka
>          Issue Type: Task
>          Components: streams
>            Reporter: John Roesler
>            Priority: Major
>
> When Streams is processing a task with multiple inputs, each time it is ready 
> to process a record, it has to choose which input to process next. It always 
> takes from the input for which the next record has the least timestamp. The 
> result of this is that Streams processes data in timestamp order. However, if 
> the buffer for one of the inputs is empty, Streams doesn't know what 
> timestamp the next record for that input will be.
> Streams introduced a configuration "max.task.idle.ms" in KIP-353 to address 
> this issue.
> [https://cwiki.apache.org/confluence/display/KAFKA/KIP-353%3A+Improve+Kafka+Streams+Timestamp+Synchronization]
> The config allows Streams to wait some amount of time for data to arrive on 
> the empty input, so that it can make a timestamp-ordered decision about which 
> input to pull from next.
> However, this config can be hard to use reliably and efficiently, since what 
> we're really waiting for is the next poll that _would_ return data from the 
> empty input's partition, and this guarantee is a function of the poll 
> interval, the max poll interval, and the internal logic that governs when 
> Streams will poll again.
> The ideal case is you'd be able to guarantee at a minimum that _any_ amount 
> of idling would guarantee you poll data from the empty partition if there's 
> data to fetch.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to