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

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

GitHub user gallenvara opened a pull request:

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

    [FLINK-6116] Watermarks don't work when unioning with same DataStream.

    ## What is the purpose of the change
    
    In self-union case, the stream edges between the source and target will be 
regard as the single one. The `streamOutputMap` in `StreamGraph` will create 
only one `RecordWriterOutput` instance.
    
    
    ## Brief change log
    
      - Add edge subId for differentiating streamEdge between the same source 
and target node.
    
    
    ## Verifying this change
    
    This change added tests and can be verified as follows:
    
    `testUnion` case in `DataStreamTest`
    
    ## 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? (no)
      - If yes, how is the feature documented? (not documented)
    


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

    $ git pull https://github.com/gallenvara/flink flink-6116

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

    https://github.com/apache/flink/pull/4649.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 #4649
    
----
commit 63fd621cfab75673a03b75b205cdd216267958a8
Author: gallenvara <[email protected]>
Date:   2017-09-06T08:24:39Z

    add edge subId for StreamEdge.

----


> Watermarks don't work when unioning with same DataStream
> --------------------------------------------------------
>
>                 Key: FLINK-6116
>                 URL: https://issues.apache.org/jira/browse/FLINK-6116
>             Project: Flink
>          Issue Type: Bug
>          Components: DataStream API
>    Affects Versions: 1.2.0, 1.3.0
>            Reporter: Aljoscha Krettek
>            Priority: Critical
>
> In this example job we don't get any watermarks in the {{WatermarkObserver}}:
> {code}
> public class WatermarkTest {
>       public static void main(String[] args) throws Exception {
>               final StreamExecutionEnvironment env = 
> StreamExecutionEnvironment.getExecutionEnvironment();
>               
> env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime);
>               env.getConfig().setAutoWatermarkInterval(1000);
>               env.setParallelism(1);
>               DataStreamSource<String> input = env.addSource(new 
> SourceFunction<String>() {
>                       @Override
>                       public void run(SourceContext<String> ctx) throws 
> Exception {
>                               while (true) {
>                                       ctx.collect("hello!");
>                                       Thread.sleep(800);
>                               }
>                       }
>                       @Override
>                       public void cancel() {
>                       }
>               });
>               input.union(input)
>                               .flatMap(new IdentityFlatMap())
>                               .transform("WatermarkOp", 
> BasicTypeInfo.STRING_TYPE_INFO, new WatermarkObserver());
>               env.execute();
>       }
>       public static class WatermarkObserver
>                       extends AbstractStreamOperator<String>
>                       implements OneInputStreamOperator<String, String> {
>               @Override
>               public void processElement(StreamRecord<String> element) throws 
> Exception {
>                       System.out.println("GOT ELEMENT: " + element);
>               }
>               @Override
>               public void processWatermark(Watermark mark) throws Exception {
>                       super.processWatermark(mark);
>                       System.out.println("GOT WATERMARK: " + mark);
>               }
>       }
>       private static class IdentityFlatMap
>                       extends RichFlatMapFunction<String, String> {
>               @Override
>               public void flatMap(String value, Collector<String> out) throws 
> Exception {
>                       out.collect(value);
>               }
>       }
> }
> {code}
> When commenting out the `union` it works.



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

Reply via email to