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

ASF GitHub Bot commented on FLINK-5479:
---------------------------------------

Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/5634
  
    @tweise @tzulitai 
    
    I would suggest to solve this the following way, which should be both 
simple and cover our cases:
    
      - We extend the current periodic watermark generators for idleness. We 
can do that for example by maintaining a record counter and remembering the 
last counter and a System.nanoTime() timestamp each time the call whether to 
generate a watermark is called. If no record came for too long, return a 
special watermark object that indicated "idle". Or change the return type to 
return either 'none', 'idle', or 'watermark'
    
      - The Kinesis Concumer needs per-shard watermarks, same way as the Kafka 
Consumer does. That part needs to be added to the Kinesis consumer anyways.
    
    That way, we automatically get per-shard idleness in Kinesis and 
per-partition idleness in Kafka without doing anything specific for the source 
connectors.
    
    We can then also remove the idleness logic from the source context - it 
would be duplicate there.


> Per-partition watermarks in FlinkKafkaConsumer should consider idle partitions
> ------------------------------------------------------------------------------
>
>                 Key: FLINK-5479
>                 URL: https://issues.apache.org/jira/browse/FLINK-5479
>             Project: Flink
>          Issue Type: Improvement
>          Components: Kafka Connector
>            Reporter: Tzu-Li (Gordon) Tai
>            Priority: Blocker
>             Fix For: 1.6.0
>
>
> Reported in ML: 
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Kafka-topic-partition-skewness-causes-watermark-not-being-emitted-td11008.html
> Similar to what's happening to idle sources blocking watermark progression in 
> downstream operators (see FLINK-5017), the per-partition watermark mechanism 
> in {{FlinkKafkaConsumer}} is also being blocked of progressing watermarks 
> when a partition is idle. The watermark of idle partitions is always 
> {{Long.MIN_VALUE}}, therefore the overall min watermark across all partitions 
> of a consumer subtask will never proceed.
> It's normally not a common case to have Kafka partitions not producing any 
> data, but it'll probably be good to handle this as well. I think we should 
> have a localized solution similar to FLINK-5017 for the per-partition 
> watermarks in {{AbstractFetcher}}.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to