GitHub user aljoscha opened a pull request: https://github.com/apache/flink/pull/988
[FLINK-2398][api-breaking] Introduce StreamGraphGenerator This decouples the building of the StreamGraph from the API methods. Before, the methods would build the StreamGraph as they go. Now the API methods build a hierachy of StreamTransformation nodes. From these a StreamGraph is generated upon execution. This also introduces some API breaking changes: - The result of methods that create sinks is now DataStreamSink instead of DataStream - Iterations cannot have feedback edges with differing parallelism - "Preserve partitioning" is not the default for feedback edges. The previous option for this is removed. - You can close an iteration several times, no need for a union. - Strict checking of whether partitioning and parallelism work together. I.e. if upstream and downstream parallelism don't match it is not legal to have Forward partitioning anymore. This was not very transparent: When you went from low parallelism to high dop some downstream operators would never get any input. When you went from high parallelism to low dop you would get skew in the downstream operators because all elements that would be forwarded to an operator that is not "there" go to another operator. This requires insertion of global() or rebalance() in some places. For example with most sources which have parallelism one. This is from the Javadoc of StreamTransformation, it describes quite well how it works: ``` A {@code StreamTransformation} represents the operation that creates a * {@link org.apache.flink.streaming.api.datastream.DataStream}. Every * {@link org.apache.flink.streaming.api.datastream.DataStream} has an underlying * {@code StreamTransformation} that is the origin of said DataStream. * * <p> * API operations such as {@link org.apache.flink.streaming.api.datastream.DataStream#map} create * a tree of {@code StreamTransformation}s underneath. When the stream program is to be executed this * graph is translated to a {@link StreamGraph} using * {@link org.apache.flink.streaming.api.graph.StreamGraphGenerator}. * * <p> * A {@code StreamTransformation} does not necessarily correspond to a physical operation * at runtime. Some operations are only logical concepts. Examples of this are union, * split/select data stream, partitioning. * * <p> * The following graph of {@code StreamTransformations}: * * <pre> * â SourceââââââââââââââSourceââââââââ * ââââ+âââââââââââââââââââ+âââââââââââ * ââââ|âââââââââââââââââââ|âââââââââââ * ââââvâââââââââââââââââââvâââââââââââ * RebalanceââââââââââHashPartitionââââ * ââââ+âââââââââââââââââââ+âââââââââââ * ââââ|âââââââââââââââââââ|âââââââââââ * ââââ|âââââââââââââââââââ|âââââââââââ * ââââ+------>Union<------+âââââââââââ * ââââââââââââââ+âââââââââââââââââââââ * ââââââââââââââ|âââââââââââââââââââââ * ââââââââââââââvâââââââââââââââââââââ * ââââââââââââSplitâââââââââââââââââââ * ââââââââââââââ+âââââââââââââââââââââ * ââââââââââââââ|âââââââââââââââââââââ * ââââââââââââââvâââââââââââââââââââââ * ââââââââââââSelectââââââââââââââââââ * ââââââââââââââ+âââââââââââââââââââââ * ââââââââââââââvâââââââââââââââââââââ * âââââââââââââMapââââââââââââââââââââ * ââââââââââââââ+âââââââââââââââââââââ * ââââââââââââââ|âââââââââââââââââââââ * ââââââââââââââvâââââââââââââââââââââ * ââââââââââââSinkâ * </pre> * * Would result in this graph of operations at runtime: * * <pre> * Source Source * + + * | | * | | * +------->Map<-------+ * + * | * v * Sink * </pre> * * The information about partitioning, union, split/select end up being encoded in the edges * that connect the sources to the map operation. ``` I still have to fix the Scala examples, but you can already comment on the overall idea and implementation. You can merge this pull request into a Git repository by running: $ git pull https://github.com/aljoscha/flink stream-api-rework Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/988.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #988 ---- commit dce02be23fc98390b8b0b98f02ad1dd69be30d4c Author: Aljoscha Krettek <aljoscha.kret...@gmail.com> Date: 2015-07-23T13:12:38Z [FLINK-2398][api-breaking] Introduce StreamGraphGenerator This decouples the building of the StreamGraph from the API methods. Before the methods would build the StreamGraph as they go. Now the API methods build a hierachy of StreamTransformation nodes. From these a StreamGraph is generated upon execution. This also introduces some API breaking changes: - The result of methods that create sinks is now DataStreamSink instead of DataStream - Iterations cannot have feedback edges with differing parallelism - "Preserve partitioning" is not the default for feedback edges. The previous option for this is removed. - You can close an iteration several times, no need for a union. - Strict checking of whether partitioning and parallelism work together. I.e. if upstream and downstream parallelism don't match it is not legal to have Forward partitioning anymore. This was not very transparent: When you went from low parallelism to high dop some downstream operators would never get any input. When you went from high parallelism to low dop you would get skew in the downstream operators because all elements that would be forwarded to an operator that is not "there" go to another operator. This requires insertion of global() or rebalance() in some places. For example with most sources which have parallelism one. ---- --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---