You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/02/06 11:26:41 UTC

[jira] [Commented] (FLINK-5017) Introduce StreamStatus stream element to allow for temporarily idle streaming sources

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

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

Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/2801
  
    @aljoscha and me had an offline discussion to let `StreamStatus` bypass the operators, instead of exposing them and have another `processStreamStatus(..)` method in the operator interface. This change is incorporated in the 1st commit.
    
    The second commit is some general improvements, including updating the Javadocs to match the new "operator bypass" implementation.
    
    Third commit adds a test to verify that watermarks generated by operators in a task's operator chain is ignored and not forwarded if the task is `IDLE`.
    
    @aljoscha this is ready for another review, thanks!


> Introduce StreamStatus stream element to allow for temporarily idle streaming sources
> -------------------------------------------------------------------------------------
>
>                 Key: FLINK-5017
>                 URL: https://issues.apache.org/jira/browse/FLINK-5017
>             Project: Flink
>          Issue Type: New Feature
>          Components: Streaming
>            Reporter: Tzu-Li (Gordon) Tai
>            Assignee: Tzu-Li (Gordon) Tai
>             Fix For: 1.3.0
>
>         Attachments: operator_chain_with_multiple_network_outputs.png
>
>
> A {{StreamStatus}} element informs receiving operators whether or not they should continue to expect watermarks from the sending operator. There are 2 kinds of status, namely {{IDLE}} and {{ACTIVE}}. Watermark status elements are generated at the sources, and may be propagated through the operators of the topology using {{Output#emitWatermarkStatus(WatermarkStatus)}}.
> Sources and downstream operators should emit either of the status elements once it changes between "watermark-idle" and "watermark-active" states.
> A source is considered "watermark-idle" if it will not emit records for an indefinite amount of time. This is the case, for example, for Flink's Kafka Consumer, where sources might initially have no assigned partitions to read from, or no records can be read from the assigned partitions. Once the source detects that it will resume emitting data, it is considered "watermark-active".
> Downstream operators with multiple inputs (ex. head operators of a {{OneInputStreamTask}} or {{TwoInputStreamTask}}) should not wait for watermarks from an upstream operator that is "watermark-idle" when deciding whether or not to advance the operator's current watermark. When a downstream operator determines that all upstream operators are "watermark-idle" (i.e. when all input channels have received the watermark idle status element), then the operator is considered to also be "watermark-idle", as it will temporarily be unable to advance its own watermark. This is always the case for operators that only read from a single upstream operator. Once an operator is considered "watermark-idle", it should itself forward its idle status to inform downstream operators. The operator is considered to be back to "watermark-active" as soon as at least one of its upstream operators resume to be "watermark-active" (i.e. when at least one input channel receives the watermark active status element), and should also forward its active status to inform downstream operators.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)