Github user mjsax commented on a diff in the pull request: https://github.com/apache/flink/pull/1398#discussion_r45888894 --- Diff: flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopology.java --- @@ -15,75 +16,474 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.flink.storm.api; +import backtype.storm.generated.ComponentCommon; +import backtype.storm.generated.GlobalStreamId; +import backtype.storm.generated.Grouping; import backtype.storm.generated.StormTopology; +import backtype.storm.topology.IRichBolt; +import backtype.storm.topology.IRichSpout; +import backtype.storm.topology.IRichStateSpout; +import backtype.storm.topology.TopologyBuilder; +import backtype.storm.tuple.Fields; +import com.google.common.base.Preconditions; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.storm.util.SplitStreamMapper; +import org.apache.flink.storm.util.SplitStreamType; +import org.apache.flink.storm.util.StormStreamSelector; +import org.apache.flink.storm.wrappers.BoltWrapper; +import org.apache.flink.storm.wrappers.BoltWrapperTwoInput; +import org.apache.flink.storm.wrappers.SpoutWrapper; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.datastream.SplitStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.InstantiationUtil; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; /** - * {@link FlinkTopology} mimics a {@link StormTopology} and is implemented in terms of a {@link - * StreamExecutionEnvironment} . In contrast to a regular {@link StreamExecutionEnvironment}, a {@link FlinkTopology} - * cannot be executed directly, but must be handed over to a {@link FlinkLocalCluster}, {@link FlinkSubmitter}, or - * {@link FlinkClient}. + * {@link FlinkTopology} translates a {@link TopologyBuilder} to a Flink program. + * <strong>CAUTION: {@link IRichStateSpout StateSpout}s are currently not supported.</strong> */ -public class FlinkTopology extends StreamExecutionEnvironment { +public class FlinkTopology { + + /** All declared streams and output schemas by operator ID */ + private final HashMap<String, HashMap<String, Fields>> outputStreams = new HashMap<String, HashMap<String, Fields>>(); + /** All spouts&bolts declarers by their ID */ + private final HashMap<String, FlinkOutputFieldsDeclarer> declarers = new HashMap<String, FlinkOutputFieldsDeclarer>(); + + private final HashMap<String, Set<Entry<GlobalStreamId, Grouping>>> unprocessdInputsPerBolt = + new HashMap<String, Set<Entry<GlobalStreamId, Grouping>>>(); + + final HashMap<String, HashMap<String, DataStream<Tuple>>> availableInputs = new HashMap<>(); - /** The number of declared tasks for the whole program (ie, sum over all dops) */ - private int numberOfTasks = 0; + private final TopologyBuilder builder; - public FlinkTopology() { - // Set default parallelism to 1, to mirror Storm default behavior - super.setParallelism(1); + // needs to be a class member for internal testing purpose + private final StormTopology stormTopology; + + private final Map<String, IRichSpout> spouts; + private final Map<String, IRichBolt> bolts; + + private final StreamExecutionEnvironment env; + + private FlinkTopology(TopologyBuilder builder) { + this.builder = builder; + this.stormTopology = builder.createTopology(); + // extract the spouts and bolts + this.spouts = getPrivateField("_spouts"); + this.bolts = getPrivateField("_bolts"); + + this.env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // Kick off the translation immediately + translateTopology(); } /** - * Is not supported. In order to execute use {@link FlinkLocalCluster}, {@link FlinkSubmitter}, or {@link - * FlinkClient}. * - * @throws UnsupportedOperationException - * at every invocation + * Creates a Flink program that uses the specified spouts and bolts. + * @param stormBuilder The storm topology builder to use for creating the Flink topology. + * @return A Flink Topology which may be executed. */ - @Override - public JobExecutionResult execute() throws Exception { - throw new UnsupportedOperationException( - "A FlinkTopology cannot be executed directly. Use FlinkLocalCluster, FlinkSubmitter, or FlinkClient " + - "instead."); + public static FlinkTopology createTopology(TopologyBuilder stormBuilder) { + return new FlinkTopology(stormBuilder); } /** - * Is not supported. In order to execute use {@link FlinkLocalCluster}, {@link FlinkSubmitter} or {@link - * FlinkClient}. - * - * @throws UnsupportedOperationException - * at every invocation + * Returns the underlying Flink ExecutionEnvironment for the Storm topology. --- End diff -- add JavaDoc link to `ExecutionEnvironment`
--- 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. ---