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

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_r29855133
  
    --- Diff: 
flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
 ---
    @@ -24,56 +24,110 @@
     import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
     import org.apache.flink.runtime.plugable.DeserializationDelegate;
     import org.apache.flink.streaming.api.graph.StreamEdge;
    +import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
     import org.apache.flink.streaming.runtime.io.CoReaderIterator;
     import org.apache.flink.streaming.runtime.io.CoRecordReader;
    -import org.apache.flink.streaming.runtime.io.IndexedReaderIterator;
     import org.apache.flink.streaming.runtime.io.InputGateFactory;
     import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
     import 
org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
    -import org.apache.flink.util.MutableObjectIterator;
    +import org.apache.flink.util.StringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
     
    -public class CoStreamTask<IN1, IN2, OUT> extends StreamTask<IN1, OUT> {
    +public class TwoInputStreamTask<IN1, IN2, OUT> extends StreamTask<OUT, 
TwoInputStreamOperator<IN1, IN2, OUT>> {
    +
    +   private static final Logger LOG = 
LoggerFactory.getLogger(TwoInputStreamTask.class);
     
        protected StreamRecordSerializer<IN1> inputDeserializer1 = null;
        protected StreamRecordSerializer<IN2> inputDeserializer2 = null;
     
    -   MutableObjectIterator<StreamRecord<IN1>> inputIter1;
    -   MutableObjectIterator<StreamRecord<IN2>> inputIter2;
    -
        CoRecordReader<DeserializationDelegate<StreamRecord<IN1>>, 
DeserializationDelegate<StreamRecord<IN2>>> coReader;
        CoReaderIterator<StreamRecord<IN1>, StreamRecord<IN2>> coIter;
     
    -   private static int numTasks;
    +   @Override
    +   public void invoke() throws Exception {
    +           this.isRunning = true;
     
    -   public CoStreamTask() {
    -           numTasks = newTask();
    -           instanceID = numTasks;
    -   }
    +           boolean operatorOpen = false;
     
    -   private void setDeserializers() {
    -           inputDeserializer1 = 
configuration.getTypeSerializerIn1(userClassLoader);
    -           inputDeserializer2 = 
configuration.getTypeSerializerIn2(userClassLoader);
    -   }
    +           if (LOG.isDebugEnabled()) {
    +                   LOG.debug("Task {} invoked", getName());
    +           }
     
    -   @Override
    -   public void setInputsOutputs() {
    -           outputHandler = new OutputHandler<OUT>(this);
    +           try {
    +
    +                   openOperator();
    +                   operatorOpen = true;
    +
    +                   int next;
    +                   StreamRecord<IN1> reuse1 = 
inputDeserializer1.createInstance();
    +                   StreamRecord<IN2> reuse2 = 
inputDeserializer2.createInstance();
    +
    +                   while (isRunning) {
    +                           try {
    +                                   next = coIter.next(reuse1, reuse2);
    +                           } catch (IOException e) {
    +                                   if (isRunning) {
    +                                           throw new 
RuntimeException("Could not read next record.", e);
    +                                   } else {
    +                                           // Task already cancelled do 
nothing
    +                                           next = 0;
    +                                   }
    +                           } catch (IllegalStateException e) {
    +                                   if (isRunning) {
    +                                           throw new 
RuntimeException("Could not read next record.", e);
    +                                   } else {
    +                                           // Task already cancelled do 
nothing
    +                                           next = 0;
    +                                   }
    +                           }
    +
    +                           if (next == 0) {
    +                                   break;
    +                           } else if (next == 1) {
    +                                   
streamOperator.receiveElement1(reuse1.getObject());
    +                                   reuse1 = 
inputDeserializer1.createInstance();
    +                           } else {
    +                                   
streamOperator.receiveElement2(reuse2.getObject());
    +                                   reuse2 = 
inputDeserializer2.createInstance();
    +                           }
    +                   }
     
    -           setConfigInputs();
    +                   closeOperator();
    +                   operatorOpen = false;
    +
    +                   if (LOG.isDebugEnabled()) {
    +                           LOG.debug("Task {} invocation finished", 
getName());
    +                   }
    +
    +           } catch (Exception e) {
    +
    +                   if (operatorOpen) {
    +                           try {
    +                                   closeOperator();
    +                           } catch (Throwable t) {
    --- End diff --
    
    I have the impression that there is a lot of very similar code ;)
    This is the third construct like this ;)


> 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