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/17 14:15:42 UTC

[jira] [Commented] (FLINK-5716) Make streaming SourceContexts aware of source idleness

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

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

GitHub user tzulitai opened a pull request:

    https://github.com/apache/flink/pull/3347

    [FLINK-5716] [streaming] Make StreamSourceContexts aware of source idleness

    This PR is the second part of adding `StreamStatus` to the streaming runtime to facilitate idle sources in Flink.
    
    It allows the `AutomaticWatermarkContext` for ingestion time and `ManualWatermarkContext` for event time to detect source idleness. It works based on a fixed-interval check implementation, where we check if the context has collected any records or watermarks from the source UDF in-between two checks. If no records or watermarks were collected in-between 2 checks, we toggle the `SourceStreamTask` to be `IDLE`. As soon as a record or watermark is collected afterwards, we toggle back to be `ACTIVE`.
    
    This idleness check is disabled by default. It is not yet configurable with this PR (that would be worked on as a separate issue [FLINK-5018](https://issues.apache.org/jira/browse/FLINK-5018)).
    
    It also introduces a new user-exposed method to the `SourceFunction.SourceContext` interface:
    ```
    interface SourceFunction.SourceContext {
        ...
        void markAsTemporarilyIdle();
        ...
    }
    ```
    
    The purpose of this method is to allow the source UDF to proactively mark the source as `IDLE` without waiting for the underlying interval checks. UDFs should make a best effort to call this. For example, the Kafka and Kinesis consumers source instances can call this as soon as they determine they will not have subscribed partitions on startup.
    
    ## Others
    
    Introduced an internal interface `StreamStatusMaintainer` -
    ```
    interface StreamStatusMaintainer extends StreamStatusProvider {
        void toggleStreamStatus(StreamStatus streamStatus);
    }
    ```
    
    Main reason for this is to keep the `StreamSource` operator unaccessible to the `OperatorChain`.
    
    ## Tests
    
    Added `StreamSourceContextIdleDetectionTests` to test the proposed functionality. I did not add task-level tests that uses `StreamTaskTestHarness` yet because the idle timeout is not yet configurable beyond the `SourceContext` interfaces. I propose to add tests with task harnesses along with [FLINK-5018](https://issues.apache.org/jira/browse/FLINK-5018).

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/tzulitai/flink FLINK-5716

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/3347.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3347
    
----
commit 66851ad987dedceb74475472d13a267101a95f61
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Date:   2017-02-16T18:43:44Z

    [FLINK-5716] [streaming] Make StreamSourceContexts aware of source idleness

----


> Make streaming SourceContexts aware of source idleness
> ------------------------------------------------------
>
>                 Key: FLINK-5716
>                 URL: https://issues.apache.org/jira/browse/FLINK-5716
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Streaming
>            Reporter: Tzu-Li (Gordon) Tai
>            Assignee: Tzu-Li (Gordon) Tai
>
> This task comes after FLINK-5017, which adds the new element {{StreamStatus}} to be incorporated with watermark progression logic.
> This task tracks the implementation of source idleness awareness and status toggling in {{SourceFunction.SourceContext}}s.
> The source contexts should work on an "idle interval", where we determine the containing {{SourceStreamTask}} to be idle if no new records / watermarks have been collected in-between 2 continuous checks. (default value is 0. Letting this value be user configurable is tracked by FLINK-5018).



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