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

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_r29854617
  
    --- Diff: 
flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
 ---
    @@ -0,0 +1,122 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.runtime.tasks;
    +
    +import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
    +import org.apache.flink.streaming.runtime.io.IndexedReaderIterator;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import 
org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
    +import org.apache.flink.util.StringUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +public class OneInputStreamTask<IN, OUT> extends StreamTask<OUT, 
OneInputStreamOperator<IN, OUT>> {
    +
    +   private static final Logger LOG = 
LoggerFactory.getLogger(OneInputStreamTask.class);
    +
    +   protected StreamRecordSerializer<IN> inSerializer;
    +   protected IndexedReaderIterator<StreamRecord<IN>> recordIterator;
    +
    +
    +   @Override
    +   public void registerInputOutput() {
    +           super.registerInputOutput();
    +           InputHandler<IN> inputHandler = new InputHandler<IN>(this);
    +           inSerializer = inputHandler.getInputSerializer();
    +           recordIterator = inputHandler.getInputIter();
    +   }
    +
    +   /*
    +    * Reads the next record from the reader iterator and stores it in the
    +    * nextRecord variable
    +    */
    +   protected StreamRecord<IN> readNext() throws IOException {
    +           StreamRecord<IN> nextRecord = inSerializer.createInstance();
    +           try {
    +                   return recordIterator.next(nextRecord);
    +           } catch (IOException e) {
    +                   if (isRunning) {
    +                           throw new RuntimeException("Could not read next 
record due to: "
    +                                           + 
StringUtils.stringifyException(e));
    --- End diff --
    
    I know the code is probably copy pasted, but we could use the opportunity 
to get rid of as many `stringifyException` as possible.
    Just passing the Throwable to the RE is nicer ;)


> 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