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

Matthias J. Sax commented on KAFKA-10091:
-----------------------------------------

I agree with John, blocking "forever" does not seem to be an option (and if a 
user really want this, they can set `max.task.idle.ms` to MAX_VALUE).

I also agree that the "consumer starvation" problem is certainly solved, but 
this is not good enough for us. Given the consumers internal "partition 
rotation" mechanism, we would only need to know if a consumer tries to fetch 
from a partition and do data come back. Maybe we can get some internal/public 
consumer API to expose this information. If a buffer is empty and a fetch 
happens with no data returned, we start the timer. If the timeout passed, we 
should do one more fetch and this also does not return any data, we go into 
"forces processing mode". – This seem to be the only strictly correct 
implementation (and we discussed this in the passed but deferred it, and 
decided that KIP-353 was a good enough first step by itself)

> 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