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

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

Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/659#discussion_r29854456
  
    --- Diff: 
flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/windowing/StreamDiscretizer.java
 ---
    @@ -183,6 +173,16 @@ public void 
open(org.apache.flink.configuration.Configuration parameters) throws
                }
        }
     
    +   @Override
    +   public void close() throws Exception {
    +           super.close();
    +           if (activePolicyThread != null) {
    +                   activePolicyThread.interrupt();
    +           }
    +
    +           emitWindow();
    --- End diff --
    
    can you call a output collector in close() ? As least for the batch part, I 
though that's dangerous.


> Rework Stream Operators to always be push based
> -----------------------------------------------
>
>                 Key: FLINK-1977
>                 URL: https://issues.apache.org/jira/browse/FLINK-1977
>             Project: Flink
>          Issue Type: Improvement
>            Reporter: Aljoscha Krettek
>            Assignee: Aljoscha Krettek
>
> This is a result of the discussion on the mailing list. This is an excerpt 
> from the mailing list that gives the basic idea of the change:
> I propose to change all streaming operators to be push based, with a
> slightly improved interface: In addition to collect(), which I would
> call receiveElement() I would add receivePunctuation() and
> receiveBarrier(). The first operator in the chain would also get data
> from the outside invokable that reads from the input iterator and
> calls receiveElement() for the first operator in a chain.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to