flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From rmetzger <...@git.apache.org>
Subject [GitHub] flink pull request: [FLINK-1977] Rework Stream Operators to always...
Date Thu, 07 May 2015 13:50:56 GMT
Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/659#discussion_r29853065
  
    --- 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 {
    +	
    +	/**
    +	 * Checks whether the stream has reached its end.
    +	 *
    +	 * <p>This method must obey the contract about blocking behavior declared in the
    +	 * description of this class.</p>
    +	 * 
    +	 * @return True, if the end of the stream has been reached, false if more data is available.
    +	 * 
    +	 * @throws InterruptedException The calling thread may be interrupted to pull the function
out of this
    +	 *                              method during checkpoints.
    +	 * @throws Exception Any other exception that is thrown causes the source to fail and
results in failure of
    +	 *                   the streaming program, or triggers recovery, depending on the program
setup.
    +	 */
    +	boolean reachedEnd() throws Exception;
    +
    +
    +	/**
    +	 * Produces the next record.
    +	 * 
    +	 * <p>This method must obey the contract about blocking behavior declared in the
    +	 * description of this class.</p>
    +	 * 
    +	 * @return The next record produced by this stream source.
    +	 * 
    +	 * @throws InterruptedException The calling thread may be interrupted to pull the function
out of this
    +	 *                              method during checkpoints.
    +	 * @throws Exception Any other exception that is thrown causes the source to fail and
results in failure of
    +	 *                   the streaming program, or triggers recovery, depending on the program
setup.
    +	 */
    +	T next() throws Exception;
    +
    +	Instant getLowWatermark();
    --- End diff --
    
    How about turning the interface into an abstract class which always returns `Instant.now()`
?
    A lot of sources are probably doing it like that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

Mime
View raw message