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

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

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

    https://github.com/apache/flink/pull/659#discussion_r29854598
  
    --- Diff: 
flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/Source.java
 ---
    @@ -0,0 +1,87 @@
    +/*
    + * 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.api.functions.source;
    +
    +import org.apache.flink.api.common.functions.Function;
    +import org.joda.time.Instant;
    +
    +import java.io.Serializable;
    +
    +/**
    + * Base interface for all stream data sources in Flink. The contract of a 
stream source
    + * is similar to an iterator - it is consumed as in the following pseudo 
code:
    + * 
    + * <pre>{@code
    + * StreamSource<T> source = ...;
    + * Collector<T> out = ...;
    + * while (!source.reachedEnd()) {
    + *   out.collect(source.next());
    + * }
    + * }
    + * </pre>
    + * 
    + * <b>Note about blocking behavior</b>
    + * <p>This implementations of the methods in the stream sources must have 
certain guarantees about
    + * blocking behavior. One of the two characteristics must be fulfilled.</p>
    + * <ul>
    + *     <li>The methods must react to thread interrupt calls and break out 
of blocking calls with
    + *         an {@link InterruptedException}.</li>
    + *     <li>The method may ignore interrupt calls and/or swallow 
InterruptedExceptions, if it is guaranteed
    + *         that the method returns quasi immediately irrespectively of the 
input. This is true for example
    + *         for file streams, where the call is guaranteed to return after 
a very short I/O delay in
    + *         the order of milliseconds.</li>
    + * </ul>
    + * 
    + * @param <T> The type of the records produced by this source.
    + */
    +public interface Source<T> extends Function, Serializable {
    --- End diff --
    
    `StreamSource` or `SourceFunction`.


> 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