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

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_r29854889
  
    --- Diff: 
flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java
 ---
    @@ -210,4 +210,36 @@ public void clearWriters() {
                        output.clearBuffers();
                }
        }
    +
    +   private static class OperatorCollector<T> implements Output<T> {
    +           private OneInputStreamOperator operator;
    +
    +           public OperatorCollector(OneInputStreamOperator<?, T> operator) 
{
    +                   this.operator = operator;
    +           }
    +
    +           @Override
    +           public void collect(T record) {
    +
    +                   try {
    +                           operator.receiveElement(record);
    +                   } catch (Exception e) {
    +                           if (LOG.isErrorEnabled()) {
    +                                   LOG.error("Could not forward element to 
operator: {}", StringUtils.stringifyException(e));
    +                           }
    +                           throw new RuntimeException(e);
    +                   }
    +           }
    +
    +           @Override
    +           public void close() {
    +                   try {
    +                           operator.close();
    +                   } catch (Exception e) {
    +                           if (LOG.isErrorEnabled()) {
    +                                   LOG.error("Could not forward close call 
to operator: " + e);
    --- End diff --
    
    This will only do a toString to the exception. Replace the + by a ,


> 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