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

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

GitHub user xccui opened a pull request:

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

    [FLINK-7245] [stream] Support holding back watermarks with static delays

    ## What is the purpose of the change
    
    *This pull request aims to allow the operators to support holding back 
watermarks with **static** delays.*
    
    
    ## Brief change log
    
      - *Introduce a new method `getWatermarkToEmit(Watermark inputWatermark)`, 
which allows to generate a new watermark with different timestamp before 
emitting it.*
      - *Add two operators `KeyedProcessOperatorWithWatermarkDelay` and 
`KeyedCoProcessOperatorWithWatermarkDelay` that support holding back watermarks 
with static delays.*
    
    ## Verifying this change
    
    This change  is verified by two new test classes 
`KeyedProcessOperatorWithWatermarkDelayTest` and 
`KeyedCoProcessOperatorWithWatermarkDelayTest`. They test whether watermarks 
received by the two added operators can be held back with the given delays and 
the provided delays are non-negative.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (**no**)
      - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (**no**)
      - The serializers: (**no**)
      - The runtime per-record code paths (performance sensitive): (**no**)
      - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (**no**)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (**yes**)
      - If yes, how is the feature documented? (**JavaDocs**)

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

    $ git pull https://github.com/xccui/flink FLINK-7245

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

    https://github.com/apache/flink/pull/4530.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 #4530
    
----
commit a24a11522af54c547d014d30adbefa23997d0f8d
Author: Xingcan Cui <xingc...@gmail.com>
Date:   2017-08-09T12:54:16Z

    [FLINK-7245] [stream] Support holding back watermarks with static delays

commit f730ab45c88f8bcbc27e411901e27dee84aa26b2
Author: Xingcan Cui <xingc...@gmail.com>
Date:   2017-08-11T16:15:53Z

    Refine codes

----


> Enhance the operators to support holding back watermarks
> --------------------------------------------------------
>
>                 Key: FLINK-7245
>                 URL: https://issues.apache.org/jira/browse/FLINK-7245
>             Project: Flink
>          Issue Type: New Feature
>          Components: DataStream API
>            Reporter: Xingcan Cui
>            Assignee: Xingcan Cui
>
> Currently the watermarks are applied and emitted by the 
> {{AbstractStreamOperator}} instantly. 
> {code:java}
> public void processWatermark(Watermark mark) throws Exception {
>       if (timeServiceManager != null) {
>               timeServiceManager.advanceWatermark(mark);
>       }
>       output.emitWatermark(mark);
> }
> {code}
> Some calculation results (with timestamp fields) triggered by these 
> watermarks (e.g., join or aggregate results) may be regarded as delayed by 
> the downstream operators since their timestamps must be less than or equal to 
> the corresponding triggers. 
> This issue aims to add another "working mode", which supports holding back 
> watermarks, to current operators. These watermarks should be blocked and 
> stored by the operators until all the corresponding new generated results are 
> emitted.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to