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

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

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

    https://github.com/apache/flink/pull/988#discussion_r36347244
  
    --- Diff: 
flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.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.api.transformations;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.streaming.api.operators.StreamOperator;
    +
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +
    +/**
    + * This represents a feedback point in a topology. The type of the 
feedback elements must not match
    + * the type of the upstream {@code StreamTransformation} because the only 
allowed operations
    + * after a {@code CoFeedbackTransformation} are
    + * {@link 
org.apache.flink.streaming.api.transformations.TwoInputTransformation 
TwoInputTransformations}.
    --- End diff --
    
    what's the issue? :smile: 


> Decouple StreamGraph Building from the API
> ------------------------------------------
>
>                 Key: FLINK-2398
>                 URL: https://issues.apache.org/jira/browse/FLINK-2398
>             Project: Flink
>          Issue Type: Improvement
>          Components: Streaming
>            Reporter: Aljoscha Krettek
>            Assignee: Aljoscha Krettek
>
> Currently, the building of the StreamGraph is very intertwined with the API 
> methods. DataStream knows about the StreamGraph and keeps track of splitting, 
> selected names, unions and so on. This leads to the problem that is is very 
> hard to understand how the StreamGraph is built because the code that does it 
> is all over the place. This also makes it hard to extend/change parts of the 
> Streaming system.
> I propose to introduce "Transformations". A transformation hold information 
> about one operation: The input streams, types, names, operator and so on. An 
> API method creates a transformation instead of fiddling with the StreamGraph 
> directly. A new component, the StreamGraphGenerator creates a StreamGraph 
> from the tree of transformations that result from program specification using 
> the API methods. This would relieve DataStream from knowing about the 
> StreamGraph and makes unions, splitting, selection visible transformations 
> instead of being scattered across the different API classes as fields.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to